baeminbo opened a new issue, #35062: URL: https://github.com/apache/beam/issues/35062
### What happened? If an object of dynamic class is serialized and deserialized multiple times through `cloudpickle`'s `dump` and `load`, the subsequent deserialization overwrite the class states of the class for the deserialized objects. This behavior was confirmed with [pickle_dump.py](https://gist.github.com/baeminbo/3272b083d1be01063af067d8ad7a2b99#file-pickle_dump-py) and [pickle_load.py](https://gist.github.com/baeminbo/3272b083d1be01063af067d8ad7a2b99#file-pickle_load-py). The dump script serializes an object of a dynamic `DoFn` class defined within a function. The load script then deserializes these bytes twice, and prints the original function information of the method `on_window_end_timer` for the **first** object. The [result](https://gist.github.com/baeminbo/3272b083d1be01063af067d8ad7a2b99#file-result) shows the methods in the class was changed. I believe this is because `cloudpickle` [reuses the class at `_lookup_class_or_track`](https://github.com/apache/beam/blob/v2.65.0/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py#L120-L126) for the same class tracker id, however [`_class_setstate`](https://github.com/apache/beam/blob/v2.65.0/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py#L1153-L1188) always updates the class states although the class is a reused one. Note that the `_class_setstate` returned as 6th tuple item of [`reducer_override`](https://github.com/apache/beam/blob/v2.65.0/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py#L1380), which is called at pickle loading instead of `__setstate__`. See [`reducer_override`](https://docs.python.org/3/library/pickle.html#reducer-override) and [`__reduce__`](https://docs.python.org/3/library/pickle.html#object.__reduce__) IIUC, This issue can cause unexpected `KeyError` or `ValueError` with `TimerSpec` in Dataflow Python jobs with Apache Beam 2.65.0. ### 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 YAML - [ ] Component: Beam examples - [ ] Component: Beam playground - [ ] Component: Beam katas - [ ] Component: Website - [ ] Component: Infrastructure - [ ] Component: Spark Runner - [ ] Component: Flink Runner - [ ] Component: Samza Runner - [ ] Component: Twister2 Runner - [ ] Component: Hazelcast Jet Runner - [x] 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: github-unsubscr...@beam.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org