From 5044a73b890571282492bae04621c9209f73e4f0 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Mon, 11 Dec 2023 14:55:50 -0800 Subject: [PATCH 1/4] failing test for dask.bag non-numeric groupby keys --- distributed/tests/test_dask_collections.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/distributed/tests/test_dask_collections.py b/distributed/tests/test_dask_collections.py index 12b9417d9ac..2303859499b 100644 --- a/distributed/tests/test_dask_collections.py +++ b/distributed/tests/test_dask_collections.py @@ -12,6 +12,7 @@ import dask.bag as db import dask.dataframe as dd +from distributed.nanny import Nanny from distributed.client import wait from distributed.utils_test import gen_cluster @@ -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( From d2491704c6ec7198571bfd2c5a9da33825ddb003 Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Tue, 12 Dec 2023 06:30:23 -0800 Subject: [PATCH 2/4] pre-commit fixes --- distributed/tests/test_dask_collections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distributed/tests/test_dask_collections.py b/distributed/tests/test_dask_collections.py index 2303859499b..8a9829c5364 100644 --- a/distributed/tests/test_dask_collections.py +++ b/distributed/tests/test_dask_collections.py @@ -12,8 +12,8 @@ import dask.bag as db import dask.dataframe as dd -from distributed.nanny import Nanny from distributed.client import wait +from distributed.nanny import Nanny from distributed.utils_test import gen_cluster dfs = [ From 6f2b00497a0d0e07d007c6126319bfa2dc9d365b Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Tue, 12 Dec 2023 09:29:24 -0800 Subject: [PATCH 3/4] if unset, set PYTHONHASHSEED in pre_spawn_env --- distributed/nanny.py | 7 +++++++ 1 file changed, 7 insertions(+) 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"), From acc507510796e4608bed92695974ae5999254a3f Mon Sep 17 00:00:00 2001 From: Charles Stern <62192187+cisaacstern@users.noreply.github.com> Date: Thu, 14 Dec 2023 09:36:48 -0800 Subject: [PATCH 4/4] fix test_environment_variable_pre_post_spawn --- distributed/tests/test_nanny.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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