You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/08/24 21:19:07 UTC

[GitHub] [beam] rahuliyer95 opened a new issue, #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

rahuliyer95 opened a new issue, #22854:
URL: https://github.com/apache/beam/issues/22854

   ### What happened?
   
   When using expansion service with Beam SDK 2.41.0, the type inference is failing with the following exception,
   ```
   Traceback (most recent call last):
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 193, in _run_module_as_main
       "__main__", mod_spec)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 85, in _run_code
       exec(code, run_globals)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/__main__.py", line 87, in <module>
       bootstrap_pex(__entry_point__)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex_bootstrapper.py", line 591, in bootstrap_pex
       pex.PEX(entry_point).execute()
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex.py", line 528, in execute
       sys.exit(self._wrap_coverage(self._wrap_profiling, self._execute))
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex.py", line 435, in _wrap_coverage
       return runner(*args)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex.py", line 466, in _wrap_profiling
       return runner(*args)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex.py", line 556, in _execute
       EntryPoint.parse("run = {}".format(self._pex_info.entry_point))
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex.py", line 739, in execute_entry
       return self.execute_module(entry_point.module)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.bootstrap/pex/pex.py", line 747, in execute_module
       runpy.run_module(module_name, run_name="__main__", alter_sys=True)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 205, in run_module
       return _run_module_code(code, init_globals, run_name, mod_spec)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 96, in _run_module_code
       mod_name, mod_spec, pkg_name, script_name)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 85, in _run_code
       exec(code, run_globals)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/twitter/udp/integration_tests/jobs/dal_read_job.py", line 180, in <module>
       main()
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/twitter/udp/integration_tests/jobs/dal_read_job.py", line 157, in main
       >> dal_rows.ReadRows("tweetsource-public", "public_tweets", opts.date).with_output_types(beam.Row)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/ptransform.py", line 1095, in __ror__
       return self.transform.__ror__(pvalueish, self.label)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/ptransform.py", line 617, in __ror__
       result = p.apply(self, pvalueish, label)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/pipeline.py", line 663, in apply
       return self.apply(transform, pvalueish)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/pipeline.py", line 709, in apply
       pvalueish_result = self.runner.apply(transform, pvalueish, self._options)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/dataflow/dataflow_runner.py", line 141, in apply
       return super().apply(transform, input, options)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/runner.py", line 185, in apply
       return m(transform, input, options)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/runner.py", line 215, in apply_PTransform
       return transform.expand(input)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/external.py", line 547, in expand
       pcoll_id in self._expanded_transform.outputs.items()
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/external.py", line 547, in <dictcomp>
       pcoll_id in self._expanded_transform.outputs.items()
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/pipeline_context.py", line 115, in get_by_id
       self._id_to_proto[id], self._pipeline_context)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/pvalue.py", line 210, in from_runner_api
       element_type=context.element_type_from_coder_id(proto.coder_id),
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/pipeline_context.py", line 269, in element_type_from_coder_id
       self.coders[coder_id].to_type_hint())
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/pipeline_context.py", line 163, in __getitem__
       return self.get_by_id(id)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/pipeline_context.py", line 115, in get_by_id
       self._id_to_proto[id], self._pipeline_context)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/coders/coders.py", line 385, in from_runner_api
       context)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/coders/row_coder.py", line 111, in from_runner_api_parameter
       return RowCoder(schema)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/coders/row_coder.py", line 63, in __init__
       self._type_hint = named_tuple_from_schema(self.schema)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/typehints/schemas.py", line 462, in named_tuple_from_schema
       schema_registry=schema_registry)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/typehints/schemas.py", line 179, in typing_from_runner_api
       schema_registry=schema_registry).typing_from_runner_api(fieldtype_proto)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/typehints/schemas.py", line 407, in typing_from_runner_api
       field_py_type = self.typing_from_runner_api(field.type)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmps6_sxbu8/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/typehints/schemas.py", line 365, in typing_from_runner_api
       return Optional[base]
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/typing.py", line 254, in inner
       return func(*args, **kwds)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/typing.py", line 357, in __getitem__
       arg = _type_check(parameters, "Optional[t] requires a single type.")
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/typing.py", line 142, in _type_check
       raise TypeError(f"{msg} Got {arg!r:.100}.")
   TypeError: Optional[t] requires a single type. Got Row(location=typing.Union[typing.Sequence[str], NoneType], empty__=typing.Union[bool, NoneType]).
   ```
   
   This is how we are using the external transform,
   ```
   import apache_beam as beam
   from apache_beam.options.pipeline_options import GoogleCloudOptions, PipelineOptions
   from apache_beam.transforms.external import ExpansionAndArtifactRetrievalStub
   import grpc
   import typing
   
   External = typing.NamedTuple(
     "External",
     [
       ("id", str),
     ],
   )
   
   creds = grpc.ssl_channel_credentials()
   channel = grpc.secure_channel("localhost:10001", creds)
   grpc.channel_ready_future(channel).result()
   stub = ExpansionAndArtifactRetrievalStub(channel)
   with beam.Pipeline(options=PipelineOptions()) as p:
     p | "Read" >> beam.ExternalTransform("URN", NamedTupleBasedPayloadBuilder(External("123")), stub)
   ```
   
   The same code works fine with SDK 2.40.0
   
   ### Issue Priority
   
   Priority: 1
   
   ### Issue Component
   
   Component: sdk-py-core


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] rahuliyer95 commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
rahuliyer95 commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1226385572

   Trying to run with `--no_pipeline_type_check` fails with the following exception,
   ```
   Traceback (most recent call last):
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 193, in _run_module_as_main
       "__main__", mod_spec)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 85, in _run_code
       exec(code, run_globals)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/__main__.py", line 87, in <module>
       bootstrap_pex(__entry_point__)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex_bootstrapper.py", line 591, in bootstrap_pex
       pex.PEX(entry_point).execute()
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex.py", line 528, in execute
       sys.exit(self._wrap_coverage(self._wrap_profiling, self._execute))
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex.py", line 435, in _wrap_coverage
       return runner(*args)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex.py", line 466, in _wrap_profiling
       return runner(*args)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex.py", line 556, in _execute
       EntryPoint.parse("run = {}".format(self._pex_info.entry_point))
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex.py", line 739, in execute_entry
       return self.execute_module(entry_point.module)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.bootstrap/pex/pex.py", line 747, in execute_module
       runpy.run_module(module_name, run_name="__main__", alter_sys=True)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 205, in run_module
       return _run_module_code(code, init_globals, run_name, mod_spec)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 96, in _run_module_code
       mod_name, mod_spec, pkg_name, script_name)
     File "/opt/twitter_mde/package/eepython37/886055eae06dbd005860a8b21c6f43355dd92462660a6eda8e580552513a78a2/lib/python3.7/runpy.py", line 85, in _run_code
       exec(code, run_globals)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/twitter/udp/integration_tests/jobs/dal_read_job.py", line 180, in <module>
       main()
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/twitter/udp/integration_tests/jobs/dal_read_job.py", line 162, in main
       | "PublicTweets.Rows.WriteText"
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/ptransform.py", line 1095, in __ror__
       return self.transform.__ror__(pvalueish, self.label)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/ptransform.py", line 617, in __ror__
       result = p.apply(self, pvalueish, label)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/pipeline.py", line 663, in apply
       return self.apply(transform, pvalueish)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/pipeline.py", line 709, in apply
       pvalueish_result = self.runner.apply(transform, pvalueish, self._options)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/dataflow/dataflow_runner.py", line 141, in apply
       return super().apply(transform, input, options)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/runner.py", line 185, in apply
       return m(transform, input, options)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/runners/runner.py", line 215, in apply_PTransform
       return transform.expand(input)
     File "/private/var/folders/yt/m4jhm2d5197c62c7v0kx77fc0000gp/T/tmp9busrhl0/.deps/apache_beam-2.41.0-py3-none-any.whl/apache_beam/transforms/external.py", line 506, in expand
       if self._type_hints.output_types:
   AttributeError: 'ExternalTransform' object has no attribute '_type_hints'
   ```


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227707255

   cc: @TheNeuralBit 


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227729109

   Ah it looks like this issue may be specifically for an optional, nested row field, which could very well be a gap in our testing. Like:
   ```
   Row {
     foo: INT
     bar: OPTIONAL[ROW { baz: STR }]
   }
   ```
   Do you know the schema returned by the external transform in this case @rahuliyer95?
   
   Regardless, this is almost certainly caused by https://github.com/apache/beam/pull/22679, we could roll that back but we'd need to do a point release to get it in 2.41.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227771515

   Hm actually the breakage is due to https://github.com/apache/beam/pull/22066, which is tougher to rollback.
   
   The core issue here is that PR started to mix beam typehints (RowTypeConstraint) and native typehints (typing.Optional). In all our test cases this was OK because the RowTypeConstraint was either the top-level, or if nested it was not nullable/optional, so it didn't need to be wrapped in a typing.Optional.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227737494

   Thanks Brian. If this is for a very specific case, I think we can just document that as a known issues for 2.41.0 release and fix for the 2.42.0 release (cut in two weeks). Also we should add the missing test.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1252722117

   Note the fix was not merged before the 2.42.0 release cut, it won't be available until 2.43.0.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] rahuliyer95 commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
rahuliyer95 commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227772286

   @TheNeuralBit I attached a debug point on line 362 in [schemas.py](https://github.com/apache/beam/blob/v2.41.0/sdks/python/apache_beam/typehints/schemas.py#L362) and here is the value for `fieldtype_proto`
   
   ```
   nullable: true
   row_type {
     schema {
       fields {
         name: "location"
         type {
           nullable: true
           array_type {
             element_type {
               atomic_type: STRING
             }
           }
         }
       }
       fields {
         name: "empty__"
         type {
           nullable: true
           atomic_type: BOOLEAN
         }
         id: 1
         encoding_position: 1
       }
       id: "be01b00e-d749-471b-bb17-4b4358d45070"
     }
   }
   ```


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit closed issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit closed issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row
URL: https://github.com/apache/beam/issues/22854


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227718333

   Thanks for the ping, this definitely looks bad. I'm not sure how existing tests didn't catch it. I would expect sql_test.py to trigger it at least.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1252802450

   It looks like this also affects Sequences of Rows, like the following structure:
   
   ```
   class Bar(NamedTuple):
     field: int
   
   class Foo(NamedTuple):
     sequence: Sequence[Bar]
   ```
   
   This is used in xlang KafkaIO, when `with_metadata=True` is specified. So Python's KafkaIO with with_metadata=True will fail at pipeline construction time in Beam 2.41.0 (FYI @johnjcasey).
     


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1252792043

   > Note the fix was not merged before the 2.42.0 release cut, it won't be available until 2.43.0.
   
   oops I was mistaken, this change is in fact in 2.42.0, you can see it here: https://github.com/apache/beam/blob/release-2.42.0/sdks/python/apache_beam/typehints/row_type.py
   
   Thanks @chamikaramj for double checking me.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #22854: [Bug]: Type inference failing for Python SDK with External transforms and beam.Row

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #22854:
URL: https://github.com/apache/beam/issues/22854#issuecomment-1227778332

   It looks like a possible forward fix is to make Python recognize `RowTypeConstraint` as a typehint, which just requires making it callable.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org