Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 2 additions & 6 deletions sdks/python/apache_beam/transforms/ptransform_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -1992,7 +1992,7 @@ def test_combine_runtime_type_check_violation_using_methods(self):
self.p._options.view_as(TypeOptions).pipeline_type_check = False
self.p._options.view_as(TypeOptions).runtime_type_check = True

with self.assertRaises(typehints.TypeCheckError) as e:
with self.assertRaises(TypeError) as e:
(
self.p
| beam.Create([0]).with_output_types(int)
Expand All @@ -2003,11 +2003,7 @@ def test_combine_runtime_type_check_violation_using_methods(self):

self.assertStartswith(
e.exception.args[0],
"Runtime type violation detected within "
"ParDo(SortJoin/KeyWithVoid): "
"Type-hint for argument: 'v' violated. "
"Expected an instance of {}, "
"instead found 0, an instance of {}.".format(str, int))
"sequence item 0: expected str instance, int found")

def test_combine_insufficient_type_hint_information(self):
self.p._options.view_as(TypeOptions).type_check_strictness = 'ALL_REQUIRED'
Expand Down
5 changes: 5 additions & 0 deletions sdks/python/apache_beam/typehints/typecheck.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,11 @@ def __init__(self, dofn):
super().__init__()
self.dofn = dofn

def __getattr__(self, attr):
if 'dofn' in self.__dict__:
return getattr(self.dofn, attr)
return super().__getattr__(attr)

def _inspect_start_bundle(self):
return self.dofn.get_function_arguments('start_bundle')

Expand Down
52 changes: 37 additions & 15 deletions sdks/python/apache_beam/typehints/typecheck_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,9 @@
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms.timeutil import TimeDomain
from apache_beam.transforms.userstate import TimerSpec
from apache_beam.transforms.userstate import on_timer
from apache_beam.typehints import TypeCheckError
from apache_beam.typehints import decorators
from apache_beam.typehints import with_input_types
Expand All @@ -48,6 +51,9 @@


class MyDoFn(beam.DoFn):

WINDOW_TIMER = TimerSpec('window_end', TimeDomain.WATERMARK)

def __init__(self, output_filename):
super().__init__()
self.output_filename = output_filename
Expand All @@ -71,13 +77,22 @@ def teardown(self):
self._output().write('teardown\n')
self._output().close()

def process(self, element: int, *args, **kwargs) -> Iterable[int]:
def process(
self,
element: Tuple[int, int],
window_timer=beam.DoFn.TimerParam(WINDOW_TIMER),
*args,
**kwargs) -> Iterable[int]:
self._output().write('process\n')
yield element
yield element[1]

@on_timer(WINDOW_TIMER)
def on_window_timer(self):
pass


class MyDoFnBadAnnotation(MyDoFn):
def process(self, element: int, *args, **kwargs) -> int:
def process(self, element: Tuple[int, int], *args, **kwargs) -> int:
# Should raise an exception about return type not being iterable.
return super().process()

Expand Down Expand Up @@ -105,21 +120,26 @@ def test_wrapper_pass_through(self):
with tempfile.TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + "tmp_filename")
dofn = MyDoFn(path)
result = self.p | beam.Create([1, 2, 3]) | beam.ParDo(dofn)
result = (
self.p | beam.Create([1, 2, 3]) | beam.WithKeys(0) | beam.ParDo(dofn))
assert_that(result, equal_to([1, 2, 3]))
self.p.run()
with open(path, mode="r") as ft:
lines = [line.strip() for line in ft]
self.assertListEqual([
'setup',
'start_bundle',
'process',
'process',
'process',
'finish_bundle',
'teardown',
],
lines)
self.assertListEqual(
[
'setup',
'start_bundle',
'process',
'process',
'process',
'finish_bundle',
# window timer
'start_bundle',
'finish_bundle',
'teardown',
],
lines)

def test_wrapper_pipeline_type_check(self):
# Verifies that type hints are not masked by the wrapper. What actually
Expand All @@ -130,7 +150,9 @@ def test_wrapper_pipeline_type_check(self):
with tempfile.NamedTemporaryFile(mode='w+t') as f:
dofn = MyDoFnBadAnnotation(f.name)
with self.assertRaisesRegex(ValueError, r'int.*is not iterable'):
_ = self.p | beam.Create([1, 2, 3]) | beam.ParDo(dofn)
_ = (
self.p
| beam.Create([1, 2, 3]) | beam.WithKeys(0) | beam.ParDo(dofn))


class PerformanceRuntimeTypeCheckTest(unittest.TestCase):
Expand Down