[ 
https://issues.apache.org/jira/browse/BEAM-4132?focusedWorklogId=330691&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-330691
 ]

ASF GitHub Bot logged work on BEAM-4132:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Oct/19 17:55
            Start Date: 18/Oct/19 17:55
    Worklog Time Spent: 10m 
      Work Description: udim commented on pull request #9810: [BEAM-4132] 
Support multi-output type inference
URL: https://github.com/apache/beam/pull/9810#discussion_r336608361
 
 

 ##########
 File path: sdks/python/apache_beam/pipeline.py
 ##########
 @@ -571,6 +573,10 @@ def _infer_result_type(self, transform, inputs, 
result_pcollection):
       else:
         result_pcollection.element_type = transform.infer_output_type(
             input_element_type)
+    elif isinstance(result_pcollection, pvalue.DoOutputsTuple):
+      # Single-input, multi-output inference.
+      for pcoll in result_pcollection:
+        self._infer_result_type(transform, inputs, pcoll)
 
 Review comment:
   This indeed does set the same output type for each pcoll. Maybe there's a 
way to infer more deeply. I can look at this later if this would be of value 
(I'm thinking that we should come up with a way to specify multi-output type 
hints).
   Currently this is better that leaving pcoll.element_type as None, which 
fails type checking.
   
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 330691)
    Time Spent: 1h 10m  (was: 1h)

> Element type inference doesn't work for multi-output DoFns
> ----------------------------------------------------------
>
>                 Key: BEAM-4132
>                 URL: https://issues.apache.org/jira/browse/BEAM-4132
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>    Affects Versions: 2.4.0
>            Reporter: Chuan Yu Foo
>            Assignee: Udi Meiri
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> TLDR: if you have a multi-output DoFn, then the non-main PCollections with 
> incorrectly have their element types set to None. This affects type checking 
> for pipelines involving these PCollections.
> Minimal example:
> {code}
> import apache_beam as beam
> class TripleDoFn(beam.DoFn):
>   def process(self, elem):
>     yield_elem
>     if elem % 2 == 0:
>       yield beam.pvalue.TaggedOutput('ten_times', elem * 10)
>     if elem % 3 == 0:
>       yield beam.pvalue.TaggedOutput('hundred_times', elem * 100)
>       
> @beam.typehints.with_input_types(int)
> @beam.typehints.with_output_types(int)
> class MultiplyBy(beam.DoFn):
>   def __init__(self, multiplier):
>     self._multiplier = multiplier
>   def process(self, elem):
>     return elem * self._multiplier
>   
> def main():
>   with beam.Pipeline() as p:
>     x, a, b = (
>       p
>       | 'Create' >> beam.Create([1, 2, 3])
>       | 'TripleDo' >> beam.ParDo(TripleDoFn()).with_outputs(
>         'ten_times', 'hundred_times', main='main_output'))
>     _ = a | 'MultiplyBy2' >> beam.ParDo(MultiplyBy(2))
> if __name__ == '__main__':
>   main()    
> {code}
> Running this yields the following error:
> {noformat}
> apache_beam.typehints.decorators.TypeCheckError: Type hint violation for 
> 'MultiplyBy2': requires <type 'int'> but got None for elem
> {noformat}
> Replacing {{a}} with {{b}} yields the same error. Replacing {{a}} with {{x}} 
> instead yields the following error:
> {noformat}
> apache_beam.typehints.decorators.TypeCheckError: Type hint violation for 
> 'MultiplyBy2': requires <type 'int'> but got Union[TaggedOutput, int] for elem
> {noformat}
> I would expect Beam to correctly infer that {{a}} and {{b}} have element 
> types of {{int}} rather than {{None}}, and I would also expect Beam to 
> correctly figure out that the element types of {{x}} are compatible with 
> {{int}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to