Skip to content

Commit

Permalink
Check 'distributed.scheduler.pickle' in Scheduler.run_function (#4838)
Browse files Browse the repository at this point in the history
  • Loading branch information
jrbourbeau authored May 24, 2021
1 parent 7c3ea36 commit d41169b
Show file tree
Hide file tree
Showing 2 changed files with 16 additions and 0 deletions.
7 changes: 7 additions & 0 deletions distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -6278,6 +6278,13 @@ def run_function(self, stream, function, args=(), kwargs={}, wait=True):
"""
from .worker import run

if not dask.config.get("distributed.scheduler.pickle"):
raise ValueError(
"Cannot run function as the scheduler has been explicitly disallowed from "
"deserializing arbitrary bytestrings using pickle via the "
"'distributed.scheduler.pickle' configuration setting."
)

self.log_event("all", {"action": "run-function", "function": function})
return run(self, stream, function=function, args=args, kwargs=kwargs, wait=wait)

Expand Down
9 changes: 9 additions & 0 deletions distributed/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -1100,6 +1100,15 @@ def f(dask_scheduler=None):
assert response == s.address


@gen_cluster(client=True, config={"distributed.scheduler.pickle": False})
async def test_run_on_scheduler_disabled(c, s, a, b):
def f(dask_scheduler=None):
return dask_scheduler.address

with pytest.raises(ValueError, match="disallowed from deserializing"):
await c._run_on_scheduler(f)


@gen_cluster(client=True)
async def test_close_worker(c, s, a, b):
assert len(s.workers) == 2
Expand Down

0 comments on commit d41169b

Please sign in to comment.