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

   ### What happened?
   
   When executing a pipeline with the `DirectRunner`, using 
`--runtime_type_check True`, it is impossible to use timers like in the 
examples provided by https://beam.apache.org/blog/timely-processing/
   
   It will end up in the following error: 
   
   ```bash
     File "apache_beam/runners/common.py", line 291, in 
apache_beam.runners.common.DoFnSignature.__init__
     File "apache_beam/runners/common.py", line 320, in 
apache_beam.runners.common.DoFnSignature._validate
     File "apache_beam/runners/common.py", line 378, in 
apache_beam.runners.common.DoFnSignature._validate_stateful_dofn
     File 
"/usr/local/lib/python3.11/site-packages/apache_beam/transforms/userstate.py", 
line 303, in validate_stateful_dofn
       if timer_spec._attached_callback != getattr(dofn, method_name, 
None).__func__:
                                           
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
   AttributeError: 'NoneType' object has no attribute '__func__'. Did you mean: 
'__doc__'?
   ```
   
   I did some prints to understand what is happening, it seems like the runtime 
type check wraps the DoFn with a 
`apache_beam.typehints.typecheck.OutputCheckWrapperDoFn`, preventing the 
following code to work ():
   
   ```python
           # apache_beam/transforms/userstate.py
           method_name = timer_spec._attached_callback.__name__
           if timer_spec._attached_callback != getattr(dofn, method_name, 
None).__func__:
              # ...
   ```
   
   Because the annotated method is not found on the wrapper.
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [X] Component: Python SDK
   - [ ] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to