Skip to content

Commit

Permalink
Merge pull request #1071: [BEAM-8280] Enable type hint annotations
Browse files Browse the repository at this point in the history
  • Loading branch information
udim committed Mar 20, 2020
2 parents 56c15b9 + 636de82 commit 0351b49
Show file tree
Hide file tree
Showing 7 changed files with 54 additions and 15 deletions.
13 changes: 13 additions & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,19 @@
## I/Os

## New Features / Improvements
* Python SDK will now use Python 3 type annotations as pipeline type hints.
([#10717](https://github.com/apache/beam/pull/10717))

If you suspect that this feature is causing your pipeline to fail, calling
`apache_beam.typehints.disable_type_annotations()` before pipeline creation
will disable is completely, and decorating specific functions (such as
`process()`) with `@apache_beam.typehints.no_annotations` will disable it
for that function.

More details will be in
[Ensuring Python Type Safety](https://beam.apache.org/documentation/sdks/python-type-safety/)
and an upcoming
[blog post](https://beam.apache.org/blog/python/typing/2020/03/06/python-typing.html).

## Breaking Changes

Expand Down
16 changes: 13 additions & 3 deletions sdks/python/apache_beam/typehints/decorators.py
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,7 @@ def foo((a, b)):
funcsigs = None

__all__ = [
'disable_type_annotations',
'no_annotations',
'with_input_types',
'with_output_types',
Expand All @@ -138,8 +139,7 @@ def foo((a, b)):

_ANY_VAR_POSITIONAL = typehints.Tuple[typehints.Any, ...]
_ANY_VAR_KEYWORD = typehints.Dict[typehints.Any, typehints.Any]
# TODO(BEAM-8280): Remove this when from_callable is ready to be enabled.
_enable_from_callable = False
_disable_from_callable = False

try:
_original_getfullargspec = inspect.getfullargspec
Expand Down Expand Up @@ -231,6 +231,16 @@ def no_annotations(fn):
return fn


def disable_type_annotations():
"""Prevent Beam from using type hint annotations to determine input and output
types of transforms.
This setting applies globally.
"""
global _disable_from_callable
_disable_from_callable = True


class IOTypeHints(NamedTuple(
'IOTypeHints',
[('input_types', Optional[Tuple[Tuple[Any, ...], Dict[str, Any]]]),
Expand Down Expand Up @@ -298,7 +308,7 @@ def from_callable(cls, fn):
Returns:
A new IOTypeHints or None if no annotations found.
"""
if not _enable_from_callable or getattr(fn, '_beam_no_annotations', False):
if _disable_from_callable or getattr(fn, '_beam_no_annotations', False):
return None
signature = get_signature(fn)
if (all(param.annotation == param.empty
Expand Down
12 changes: 10 additions & 2 deletions sdks/python/apache_beam/typehints/decorators_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,6 @@
from apache_beam.typehints import decorators
from apache_beam.typehints import typehints

decorators._enable_from_callable = True


class IOTypeHintsTest(unittest.TestCase):
def test_get_signature(self):
Expand Down Expand Up @@ -221,5 +219,15 @@ def __init__(self):
Subclass().with_input_types(str)._type_hints, Subclass._type_hints)


class DecoratorsTest(unittest.TestCase):
def tearDown(self):
decorators._disable_from_callable = False

def test_disable_type_annotations(self):
self.assertFalse(decorators._disable_from_callable)
decorators.disable_type_annotations()
self.assertTrue(decorators._disable_from_callable)


if __name__ == '__main__':
unittest.main()
19 changes: 18 additions & 1 deletion sdks/python/apache_beam/typehints/decorators_test_py3.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@
from apache_beam.typehints import TypeVariable
from apache_beam.typehints import decorators

decorators._enable_from_callable = True
T = TypeVariable('T')
# Name is 'T' so it converts to a beam type with the same name.
# mypy requires that the name of the variable match, so we must ignore this.
Expand Down Expand Up @@ -194,5 +193,23 @@ def fn(a: int) -> int:
_ = ['a', 'b', 'c'] | Map(fn)


class DecoratorsTest(unittest.TestCase):
def test_no_annotations(self):
def fn(a: int) -> int:
return a

_ = [1, 2, 3] | Map(fn) # Doesn't raise - correct types.

with self.assertRaisesRegex(TypeCheckError,
r'requires .*int.* but got .*str'):
_ = ['a', 'b', 'c'] | Map(fn)

@decorators.no_annotations
def fn2(a: int) -> int:
return a

_ = ['a', 'b', 'c'] | Map(fn2) # Doesn't raise - no input type hints.


if __name__ == '__main__':
unittest.main()
3 changes: 0 additions & 3 deletions sdks/python/apache_beam/typehints/typed_pipeline_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,12 @@
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.typehints import WithTypeHints
from apache_beam.typehints import decorators
from apache_beam.typehints.decorators import get_signature

# These test often construct a pipeline as value | PTransform to test side
# effects (e.g. errors).
# pylint: disable=expression-not-assigned

decorators._enable_from_callable = True


class MainInputTest(unittest.TestCase):
def test_bad_main_input(self):
Expand Down
3 changes: 0 additions & 3 deletions sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,6 @@

import apache_beam as beam
from apache_beam import typehints
from apache_beam.typehints import decorators

decorators._enable_from_callable = True


class MainInputTest(unittest.TestCase):
Expand Down
3 changes: 0 additions & 3 deletions sdks/python/apache_beam/typehints/typehints_test_py3.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,6 @@
from apache_beam.transforms.core import DoFn
from apache_beam.typehints import KV
from apache_beam.typehints import Iterable
from apache_beam.typehints import decorators

decorators._enable_from_callable = True


class TestParDoAnnotations(unittest.TestCase):
Expand Down

0 comments on commit 0351b49

Please sign in to comment.