On Mon, Mar 30, 2020 at 10:40 AM Joshua B. Harrison <[email protected]>
wrote:

> Thank you for getting back to me. I would be happy to help contribute -
> has there been any discussion around this issue before?
>

Udi has been pushing the type annotation work forward lately, though I
don't know that he's looked into the multi-output much, if at all. It'd be
great if you could contribute!


> At the least, I think it be preferable to raise a not implemented error in
> Python when encountering this case.
>

Agreed.


> It seems like multi-input for CoGroupByKey is represented as a Union of
> all the component collection types. Would it make sense to do the same for
> the output types? Is this a better discussion for the dev group?
>

+1 to taking this to the dev group.


> Thanks again for your time and help.
>
> Best,
> Joshua
>
> On Mon, Mar 30, 2020 at 11:22 AM Robert Bradshaw <[email protected]>
> wrote:
>
>> That is correct, type hints unfortunately are not yet supported for
>> multiple-output PTransforms.
>>
>> On Thu, Mar 26, 2020 at 10:05 PM Joshua B. Harrison <
>> [email protected]> wrote:
>>
>>> Hello all,
>>>
>>> I am working on adding type hints to my pipeline, and ran into an issue
>>> with PTransforms that produce multiple, tagged outputs.
>>>
>>> My class looks like this:
>>>
>>> @with_input_types(mytype.Data)
>>>> @with_output_types(mytype.KeyedData)
>>>> class DenormalizeData(ptransform.PTransform):
>>>>   MAIN = 'denormalized'
>>>>   SKIPPED = functions.DenormalizeData.SKIPPED
>>>>   def expand(self, pcol: mytype.Data) -> mytype.KeyedPriceData:
>>>>     return (pcol
>>>>       | 'Denormalize PriceData' >> core.ParDo(
>>>>         functions.DenormalizeData()).with_outputs(
>>>>           self.SKIPPED, main=self.MAIN))
>>>
>>>
>>> Where functions.DenormalizeData is a core.DoFn. From what I can tell,
>>> the type checking code here at
>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/ptransform.py#L429
>>>  attempts
>>> to access the pvalue._element_type. But in this case, the pvalue is a
>>> DoOutputsTuple (
>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/pvalue.py#L239)
>>> which overrides __getattr__ to check for tag names. In this case,
>>> _element_type is not a valid tag, and I get the following partial  stack:
>>>
>>> "apache_beam_2_17_0/apache_beam/transforms/ptransform.py", line 401, in
>>>> type_check_inputs_or_outputs
>>>>     if pvalue_.element_type is None:
>>>>   File "apache_beam_2_17_0/apache_beam/pvalue.py", line 241, in
>>>> __getattr__
>>>>     return self[tag]
>>>>   File "apache_beam_2_17_0/apache_beam/pvalue.py", line 256, in
>>>> __getitem__
>>>>     tag, self._main_tag, self._tags))
>>>> ValueError: Tag 'element_type' is neither the main tag 'denormalized'
>>>> nor any of the tags ('skipped',)
>>>
>>>
>>> Is my diagnoses correct? Is this a known issue? Can type hints exist on
>>> DoOutputsTuples?
>>>
>>> Thank you for your time and help.
>>>
>>> Best,
>>> Joshua
>>>
>>> --
>>> Joshua Harrison |  Software Engineer |  [email protected]
>>> <[email protected]> |  404-433-0242 <(404)%20433-0242>
>>>
>>
>
> --
> Joshua Harrison |  Software Engineer |  [email protected]
> <[email protected]> |  404-433-0242 <(404)%20433-0242>
>

Reply via email to