cisaacstern commented on PR #29802:
URL: https://github.com/apache/beam/pull/29802#issuecomment-1863683005
## The good news
The main objective of fixing GBK appears to be solved, as shown by the fact
that this script
```python
# gbk.py
import apache_beam as beam
from apache_beam.runners.dask.dask_runner import DaskRunner
if __name__ == "__main__":
with beam.Pipeline(runner=DaskRunner()) as p:
p | beam.Create([('a', 1), ('a', 2), ('b', 3), ('b', 4)]) |
beam.GroupByKey() | beam.Map(print)
```
gives a correctly grouped result when when with `distributed` installed from
upstream (at the pinned commit)
```console
$ python gbk.py
('b', [3, 4])
('a', [1, 2])
```
and (as expected) an _incorrect_ result with the latest release of
`distributed` from pypi
```console
$ python gbk.py
('a', [1])
('b', [3])
('b', [4])
('a', [2])
```
### The bad news
We can't actually test this, because while the linked `distributed` PR _did_
fix string keys for `dask.bag.Bag.groupby`, it turns out `None` keys are _also_
broken in `dask.bag` (And [`None` keys are required to use
`assert_that`](https://github.com/apache/beam/blob/0bbf2c3351bb83db6aeb5b89b86a81653ebb6d87/sdks/python/apache_beam/testing/util.py#L286-L292).)
This is because `hash(None)` is non-deterministic in Python < 3.12. A few
notes/thoughts:
- `hash(None)` _is_ deterministic in Python 3.12, thanks to
https://github.com/python/cpython/pull/99541 (I found the linked issue
discussion very interesting! FWIW)
- Non-deterministic behavior of `hash(None)` also means otherwise
deterministically-hashable containers (e.g. tuples, frozen dataclasses, etc)
hash non-deterministically if they contain `None`. This seems like a very
common category of key for real-world use cases.
- Only supporting GBK for > Python 3.12 seems like an unreasonable
workaround for what is ultimately an upstream brittleness issue, and should be
fixed there.
So it seems like the only viable path forward is to revisit a fix for
https://github.com/dask/dask/issues/6723, which I was hoping we could avoid. I
will re-ping that issue either later today or tomorrow with some thoughts on a
path forward.
--
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]