Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Better error message on misspelled executor annotation #6009

Merged
merged 1 commit into from
Mar 28, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions distributed/tests/test_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -1897,6 +1897,15 @@ def get_thread_name():
assert "Dask-Foo-Threads" in gpu_result


@gen_cluster(client=True)
async def test_bad_executor_annotation(c, s, a, b):
with dask.annotate(executor="bad"):
future = c.submit(inc, 1)
with pytest.raises(ValueError, match="Invalid executor 'bad'; expected one of: "):
await future
assert future.status == "error"


@gen_cluster(client=True)
async def test_process_executor(c, s, a, b):
with ProcessPoolExecutor() as e:
Expand Down
17 changes: 11 additions & 6 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -3421,17 +3421,22 @@ async def execute(self, key: str, *, stimulus_id: str) -> None:

args2, kwargs2 = self._prepare_args_for_execution(ts, args, kwargs)

if ts.annotations is not None and "executor" in ts.annotations:
executor = ts.annotations["executor"]
else:
try:
executor = ts.annotations["executor"] # type: ignore
except (TypeError, KeyError):
executor = "default"
assert executor in self.executors
assert key == ts.key
try:
e = self.executors[executor]
except KeyError:
raise ValueError(
f"Invalid executor {executor!r}; "
f"expected one of: {sorted(self.executors)}"
)

self.active_keys.add(ts.key)

result: dict
try:
e = self.executors[executor]
ts.start_time = time()
if iscoroutinefunction(function):
result = await apply_function_async(
Expand Down