[ https://issues.apache.org/jira/browse/BEAM-4091?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Chuan Yu Foo updated BEAM-4091: ------------------------------- Description: 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} was: 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} > 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: Ahmet Altay > Priority: Major > > 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 (v7.6.3#76005)