[
https://issues.apache.org/jira/browse/BEAM-4091?focusedWorklogId=506753&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-506753
]
ASF GitHub Bot logged work on BEAM-4091:
----------------------------------------
Author: ASF GitHub Bot
Created on: 03/Nov/20 13:42
Start Date: 03/Nov/20 13:42
Worklog Time Spent: 10m
Work Description: tvalentyn commented on a change in pull request #13246:
URL: https://github.com/apache/beam/pull/13246#discussion_r516341288
##########
File path: sdks/python/apache_beam/transforms/ptransform.py
##########
@@ -1000,17 +1000,24 @@ def expand(self, pcoll):
@wraps(fn)
def callable_ptransform_factory(*args, **kwargs):
res = _PTransformFnPTransform(fn, *args, **kwargs)
+ # Apply type hints applied before or after the ptransform_fn decorator,
+ # falling back on PTransform defaults.
+ # If the @with_{input,output}_types decorator comes before ptransform_fn,
+ # the type hints get applied to this function. If it comes after they will
+ # get applied to fn, and @wraps will copy the _type_hints attribute to
+ # this function.
+ type_hints = get_type_hints(callable_ptransform_factory)
if ptransform_fn_typehints_enabled:
- # Apply type hints applied before or after the ptransform_fn decorator,
- # falling back on PTransform defaults.
- # If the @with_{input,output}_types decorator comes before ptransform_fn,
- # the type hints get applied to this function. If it comes after they
will
- # get applied to fn, and @wraps will copy the _type_hints attribute to
- # this function.
- type_hints = get_type_hints(callable_ptransform_factory)
res._set_type_hints(type_hints.with_defaults(res.get_type_hints()))
_LOGGER.debug(
'type hints for %s: %s', res.default_label(), res.get_type_hints())
+ elif type_hints:
+ _LOGGER.warning(
Review comment:
How noisy would this log be ? Does it warrant a `warning` or should it
be an `info` instead ?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 506753)
Time Spent: 7h (was: 6h 50m)
> Typehint annotations don't work with @ptransform_fn annotation
> --------------------------------------------------------------
>
> Key: BEAM-4091
> URL: https://issues.apache.org/jira/browse/BEAM-4091
> Project: Beam
> Issue Type: Bug
> Components: sdk-py-core
> Affects Versions: 2.4.0
> Reporter: Chuan Yu Foo
> Assignee: Udi Meiri
> Priority: P2
> Labels: types
> Time Spent: 7h
> Remaining Estimate: 0h
>
> Typehint annotations don't work with functions annotated with
> {{@ptransform_fn}}, but they do work with the equivalent classes.
> The following is a minimal example illustrating this:
> {code:python}
> @beam.typehints.with_input_types(float)
> @beam.typehints.with_output_types(bytes)
> @beam.ptransform_fn
> def _DoStuffFn(pcoll):
> return pcoll | 'TimesTwo' >> beam.Map(lambda x: x * 2)
> @beam.typehints.with_input_types(float)
> @beam.typehints.with_output_types(bytes)
> class _DoStuffClass(beam.PTransform):
> def expand(self, pcoll):
> return pcoll | 'TimesTwo' >> beam.Map(lambda x: x * 2)
> {code}
> With definitions as above, the class correctly fails the typecheck:
> {code:python}
> def class_correctly_fails():
> p = beam.Pipeline(options=PipelineOptions(runtime_type_check=True))
> _ = (p
> | 'Create' >> beam.Create([1, 2, 3, 4, 5])
> | 'DoStuff1' >> _DoStuffClass()
> | 'DoStuff2' >> _DoStuffClass()
> | 'Write' >> beam.io.WriteToText('/tmp/output'))
> p.run().wait_until_finish()
> # apache_beam.typehints.decorators.TypeCheckError: Input type hint violation
> at DoStuff1: expected <type 'float'>, got <type 'int'>
> {code}
> But the {{ptransform_fn}} incorrectly passes the typecheck:
> {code:python}
> def ptransform_incorrectly_passes():
> p = beam.Pipeline(options=PipelineOptions(runtime_type_check=True))
> _ = (p
> | 'Create' >> beam.Create([1, 2, 3, 4, 5])
> | 'DoStuff1' >> _DoStuffFn()
> | 'DoStuff2' >> _DoStuffFn()
> | 'Write' >> beam.io.WriteToText('/tmp/output'))
> p.run().wait_until_finish()
> # No error
> {code}
> Note that changing the order of the {{@ptransform_fn}} and type hint
> annotations doesn't change the result, i.e. changing {{_DoStuffFn}} to the
> following still results in it incorrectly passing the typecheck:
> {code:python}
> @beam.ptransform_fn
> @beam.typehints.with_input_types(float)
> @beam.typehints.with_output_types(bytes)
> def _DoStuffFn(pcoll):
> return pcoll | 'TimesTwo' >> beam.Map(lambda x: x * 2)
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)