[
https://issues.apache.org/jira/browse/BEAM-4091?focusedWorklogId=487557&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-487557
]
ASF GitHub Bot logged work on BEAM-4091:
----------------------------------------
Author: ASF GitHub Bot
Created on: 22/Sep/20 03:16
Start Date: 22/Sep/20 03:16
Worklog Time Spent: 10m
Work Description: aaltay commented on a change in pull request #9907:
URL: https://github.com/apache/beam/pull/9907#discussion_r492447574
##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -476,6 +499,26 @@ def _add_argparse_args(cls, parser):
'time. NOTE: only supported with portable runners '
'(including the DirectRunner)')
+ def validate(self, unused_validator):
+ errors = []
+ if beam.version.__version__ >= '3':
Review comment:
I think when/if we reach to Beam version 3, we can remove this warning,
and all and etc and and enable by default without an option to disable. So
maybe we do not need this?
##########
File path: CHANGES.md
##########
@@ -73,6 +73,10 @@
* In Interactive Beam, ib.show() and ib.collect() now have "n" and "duration"
as parameters. These mean read only up to "n" elements and up to "duration"
seconds of data read from the recording
([BEAM-10603](https://issues.apache.org/jira/browse/BEAM-10603)).
* Initial preview of
[Dataframes](https://s.apache.org/simpler-python-pipelines-2020#slide=id.g905ac9257b_1_21)
support.
See also example at apache_beam/examples/wordcount_dataframe.py
+* Fixed support for type hints on `@ptransform_fn` decorators in the Python
SDK.
Review comment:
Maybe warn that the default might change in a 2.x version later?
----------------------------------------------------------------
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: 487557)
Time Spent: 4h 50m (was: 4h 40m)
> 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
> Priority: P2
> Time Spent: 4h 50m
> 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)