diff --git a/distributed/nanny.py b/distributed/nanny.py index 021b41edfcd..2c14224d363 100644 --- a/distributed/nanny.py +++ b/distributed/nanny.py @@ -233,6 +233,13 @@ def __init__( # type: ignore[no-untyped-def] self.Worker = Worker if worker_class is None else worker_class self.pre_spawn_env = _get_env_variables("distributed.nanny.pre-spawn-environ") + # To get consistent hashing on subprocesses, we need to set a consistent seed for + # the Python hash algorithm; xref https://github.com/dask/distributed/pull/8400 + if self.pre_spawn_env.get("PYTHONHASHSEED") in (None, "0"): + # This number is arbitrary; it was chosen to commemorate + # https://github.com/dask/dask/issues/6640. + self.pre_spawn_env.update({"PYTHONHASHSEED": "6640"}) + self.env = merge( self.pre_spawn_env, _get_env_variables("distributed.nanny.environ"), diff --git a/distributed/tests/test_dask_collections.py b/distributed/tests/test_dask_collections.py index 12b9417d9ac..8a9829c5364 100644 --- a/distributed/tests/test_dask_collections.py +++ b/distributed/tests/test_dask_collections.py @@ -13,6 +13,7 @@ import dask.dataframe as dd from distributed.client import wait +from distributed.nanny import Nanny from distributed.utils_test import gen_cluster dfs = [ @@ -124,6 +125,18 @@ async def test_bag_groupby_tasks_default(c, s, a, b): assert not any("partd" in k[0] for k in b2.dask) +@gen_cluster(client=True, Worker=Nanny) +async def test_bag_groupby_key_hashing(c, s, a, b): + # https://github.com/dask/distributed/issues/4141 + dsk = {("x", 0): (range, 5), ("x", 1): (range, 5), ("x", 2): (range, 5)} + grouped = db.Bag(dsk, "x", 3).groupby(lambda x: "even" if x % 2 == 0 else "odd") + remote = c.compute(grouped) + result = await remote + assert len(result) == 2 + assert ("odd", [1, 3] * 3) in result + assert ("even", [0, 2, 4] * 3) in result + + @pytest.mark.parametrize("wait", [wait, lambda x: None]) def test_dataframe_set_index_sync(wait, client): df = dask.datasets.timeseries( diff --git a/distributed/tests/test_nanny.py b/distributed/tests/test_nanny.py index f0e824d3ed6..74d049a35d4 100644 --- a/distributed/tests/test_nanny.py +++ b/distributed/tests/test_nanny.py @@ -340,10 +340,13 @@ async def test_environment_variable_config(c, s, monkeypatch): }, ) async def test_environment_variable_pre_post_spawn(c, s, n): - assert n.env == {"PRE-SPAWN": "1", "POST-SPAWN": "2"} + assert n.env == {"PRE-SPAWN": "1", "POST-SPAWN": "2", "PYTHONHASHSEED": "6640"} results = await c.run(lambda: os.environ) assert results[n.worker_address]["PRE-SPAWN"] == "1" assert results[n.worker_address]["POST-SPAWN"] == "2" + # if unset in pre-spawn-environ config, PYTHONHASHSEED defaults to "6640" to ensure + # consistent hashing across workers; https://github.com/dask/distributed/issues/4141 + assert results[n.worker_address]["PYTHONHASHSEED"] == "6640" del os.environ["PRE-SPAWN"] assert "POST-SPAWN" not in os.environ