chuckwondo commented on issue #28558:
URL: https://github.com/apache/beam/issues/28558#issuecomment-1732318883

   After digging into this more, I see that a `_MemoizingPickleCoder` was added 
via #15357, and that coder makes use of the internal `pickler` module, which 
means that it honors any `pickle_library` option that may have been specified. 
This is precisely the approach (sans memoization) that I've identified above as 
a potential "fix" for `PickleCoder`.
   
   However, the way I'm interpreting various related parts of the code, there 
appear to be some inconsistencies:
   
   1. `_MemoizingPickleCoder` overrides `as_deterministic_coder`, but 
`DillCoder` does not do so. Since `pickle_library` uses `dill` by default, that 
means that the `_create_impl` methods of `_MemoizingPickleCoder` and 
`DillCoder` are essentially functionally equivalent (ignoring memoization), 
when `pickle_library` is `"default"` or `"dill"`, so I would expect that _both_ 
coders should implement `as_deterministic_coder` the same way (i.e., either 
_both_ override that method identically, or _neither_ would override it).
   2. `PickleCoder` overrides `as_deterministic_coder` identically to 
`_MemoizingPickleCoder` (I suspect `_MemoizingPickleCoder`'s implementation was 
copied from `PickleCoder`), but `PickleCoder` makes direct use of the standard 
`pickle` module rather than using Beam's internal `pickler` module (which is 
seemingly the crux of this bug). I don't know if this represents any sort of 
inconsistency, but they _both_ return `FastPrimitivesCoder(self, 
requires_deterministic=step_label)`, which doesn't work because 
`FastPrimitivesCoder` only takes 1 argument. It appears that this is being 
confused with `FastPrimitivesCoderImpl` which does take a 2nd argument 
(although it's named `requires_deterministic_step_label`, not 
`requires_deterministic`). Since this appears to be a separate bug, I will 
create a new issue reporting it as such.
   3. The docstring at the top of `apache_beam/coders/coders.py` states the 
following: "In Beam, we generally we use `pickle` for pipeline elements and 
`dill` for more complex types, like user functions." This is exactly the 
situation I'm facing (i.e., the need to pickle a user function), but setting 
`save_main_session` and `pickle_library` are _not_ causing my function to be 
pickled via `dill` (default) nor `cloudpickle`. Upon further digging, it 
appears that using `cloudpickle` should skip saving the main session, but using 
`dill` should still do so, but this is not the case. Instead `PickleCoder`, 
which uses the `pickle` module, is what is being invoked to pickle the 
function, rather than `dill`.
   
   So far, I have been unable to decipher _where_ there should be logic that 
uses the internal `pickler` module, particularly in the case of saving the main 
session, or pickling user functions, nor if there is something else I should be 
doing in my transforms to trigger such usage of `pickler`, in _addition_ to 
setting `save_main_session` to `True`.
   
   The only thing I've managed to do that works for this case is to modify 
`PickleCoder` to use `pickler` instead of `pickle`, but I don't know if that's 
the correct solution. I cannot tell if there are cases where we _intentionally_ 
want to use the standard `pickle` module rather than the internal `pickler` 
module, so I don't know if changing `PickleCoder` to use `pickler` instead of 
`pickle` to fix the error I'm encountering might cause failures for other cases.
   
   I've forked and cloned the repo, but for the life of me, I cannot get the 
Python test suite to run without failure. I have managed to get the word count 
example to run successfully (as mentioned in the contributing guide), but I 
cannot determine how to run a more comprehensive set of tests that covers 
testing of pickling, so I am not confident about being able test my proposed 
fix (nor add new tests, if necessary) since I cannot get a suite of tests to 
run without failure (nor without taking extremely long to run -- I've killed 
the test process after 15 minutes of running, and then probably another 15 
minutes trying to kill all of the spawned processes that seem to restart 
themselves and don't get cleaned up when I halt the main test process via 
Ctrl-C).
   
   Does anybody have some guidance here?


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