Hi all! Currently, in the Python SDK, we don't support annotation-style type hints for PTransforms.
This email includes a proposal to support PEP 484 annotations on PTransform's expand() function, and would allow you to write something like the following: class MapStrToInt(beam.PTransform): def expand(pcoll: PCollection[str]) -> PCollection[int]: return pcoll | beam.Map(lambda elem: int(elem) You can view the proposal here <https://docs.google.com/document/d/1B91mV_KSJvkQopvIqa1LhV0pkPfMEeavKHb6Ql9GFaM/edit?usp=sharing>, the WIP PR here <https://github.com/apache/beam/pull/12009>, or the full text of the proposal below. Let me know what you think, or if you have any suggestions for improvements. Thanks! Best, Saavan --------------- Supporting Python Annotations on PTransforms https://issues.apache.org/jira/browse/BEAM-10258 Author: Saavan Nanavati (saa...@google.com <autho...@example.org>) Reviewer: Udi Meiri (eh...@google.com) Overview For any developer working in a dynamically-typed language such as Python, type safety is an important consideration that can help reduce runtime bugs, speed up the development process with IDE-assisted code completion and type hints using static type-checkers such as MyPy, and aid investigations into otherwise complex error messages, with regards to Coder objects, for instance. Due to the complex nature of pipelines in Beam, an internal type-checking system is employed by Beam to fill in the gaps that static type-checkers have (such as failing to match type hints across PTransforms or type-checking at runtime). These type hints are used to raise errors during pipeline construction and, optionally, at runtime as well (but at the expense of pipeline performance). This design document outlines a proposed improvement that would allow Beam’s internal typing system to integrate and utilize type hint annotations (that conform to PEP 484 <https://www.python.org/dev/peps/pep-0484/>) on PTransform’s primary method, expand(...). Table of Contents Background <https://docs.google.com/document/d/1B91mV_KSJvkQopvIqa1LhV0pkPfMEeavKHb6Ql9GFaM/edit#heading=h.drvuif6t4bls> Goals <https://docs.google.com/document/d/1B91mV_KSJvkQopvIqa1LhV0pkPfMEeavKHb6Ql9GFaM/edit#heading=h.wlzpbos00ni> Design Details <https://docs.google.com/document/d/1B91mV_KSJvkQopvIqa1LhV0pkPfMEeavKHb6Ql9GFaM/edit#heading=h.qds1titcbwpu> Estimate of Work <https://docs.google.com/document/d/1B91mV_KSJvkQopvIqa1LhV0pkPfMEeavKHb6Ql9GFaM/edit#heading=h.7bp0u7a4fgs> Background There are currently 3 ways to declare type hints in Beam pipelines. 1. Inline 1. with_input_types(...) 2. with_input_types(...) 2. Decorators 1. @beam.typehints.with_input_types(...) 2. @beam.typehints.with_output_types(...) 3. Annotations 1. def func(elem: int) -> str: ... Since inline hints are not associated with PTransforms themselves, decorators and annotations are the preferred methods for code that is going to be reused, however annotations are not currently supported for PTransforms. This proposed feature would fix that, allowing you to write code akin to the following: import apache_beam as beam from apache_beam.pvalue import PCollection class MapStrToInt(beam.PTransform): def expand(pcoll: PCollection[str]) -> PCollection[int]: return pcoll | beam.Map(lambda elem: int(elem) This is in contrast to the recommended way of type-hinting PTransforms in the status quo, shown below: import apache_beam as beam @beam.typehints.with_input_types(str) @beam.typehints.with_input_types(int) class MapStrToInt(beam.PTransform): def expand(pcoll): return pcoll | beam.Map(lambda elem: int(elem) In addition to providing the end-user a number of approaches for how they can type-hint their pipelines, thereby expanding coverage for Beam’s internal typing system, there is potentially an additional benefit for static type checkers like MyPy to be able to catch errors, since they are only capable of working with PEP 484 annotations. Goals - Support PEP 484 annotations on the input/output PCollection of the expand(...) method of PTransforms - Update the documentation on Ensuring Python Type Safety <https://beam.apache.org/documentation/sdks/python-type-safety/> to reflect these changes Design Details First, for Beam’s internal typing system to work with PCollections, PCollection must be capable of being used as a type. This is currently already the case since PCollection inherits from Generic[T]. Second, Beam’s typing system must be able to work with annotations like PCollection[T]. This can be done by modelling the approach that DoFn takes for analyzing annotations on its’ process(...) method. To elaborate, DoFn does this by implementing the default_type_hints(...) method from the inherited WithTypeHints mixin class, which both DoFn and PTransform inherit from, and using IOTypeHints.from_callable(...) to convert the annotations on process(...) into an IOTypeHints object that the internal Beam typing system can then use. A similar method can be used by PTransform on its’ expand(...) method, however the outer PCollection would need to be stripped from the input and output types before the annotations can be compatible with the rest of the Beam typing system. There are some additional considerations to keep in mind however. First, PBegin[T] and PDone[T] must also be supported as valid outer types for PTransform annotations. This requires both classes to inherit from Generic[T], which they do not currently. Second, PCollection by itself, without any internal type hint T, must be supported. In this case, T will be presumed to be of type any. This is currently the only design approach under consideration. Estimate of Work The majority of the work is already complete, and available to view in this PR <https://github.com/apache/beam/pull/12009>. There is still some work TBD - more tests need to be added in order to confirm edge-case scenarios work properly. These include: - Tests for PBegin/PDone annotations - Tests for PCollection annotations with multiple internal types - Tests for PCollection annotations with a nested PCollection Additionally, more error checking is needed throughout the PR. Lastly, the website documentation needs to be updated. With the holidays coming up this week (July 4 weekend), this work can be estimated to be complete around July 8. This is not including the time it takes to run against internal tests in Google, some of which may or may not be broken due to this change, therefore the time-to-merge will likely be longer (possibly by a week or two).