diff --git a/distributed/nanny.py b/distributed/nanny.py
index 845b73bd05..e4215f2aeb 100644
--- a/distributed/nanny.py
+++ b/distributed/nanny.py
@@ -1032,4 +1032,9 @@ def _get_env_variables(config_key: str) -> dict[str, str]:
     # Override dask config with explicitly defined env variables from the OS
     # Allow unsetting a variable in a config override by setting its value to None.
     cfg = {k: os.environ.get(k, str(v)) for k, v in cfg.items() if v is not None}
+
+    for k, v in list(cfg.items()):
+        if "_NUM_THREADS" in k and not v:
+            del cfg[k]
+
     return cfg
diff --git a/distributed/tests/test_nanny.py b/distributed/tests/test_nanny.py
index 1af40076c6..2bb1b26cb5 100644
--- a/distributed/tests/test_nanny.py
+++ b/distributed/tests/test_nanny.py
@@ -312,6 +312,19 @@ async def test_environment_variable(c, s):
     await asyncio.gather(a.close(), b.close())
 
 
+@gen_cluster(
+    nthreads=[("", 1)],
+    client=True,
+    Worker=Nanny,
+    config={
+        "distributed.nanny.pre-spawn-environ": {"OMP_NUM_THREADS": ""},
+    },
+)
+async def test_omp_num_threads_off(c, s, a):
+    results = await c.run(lambda: "OMP_NUM_THREADS" in os.environ)
+    assert results == {a.worker_address: False}
+
+
 @gen_cluster(nthreads=[], client=True)
 async def test_environment_variable_by_config(c, s, monkeypatch):
     with dask.config.set({"distributed.nanny.environ": "456"}):