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

Log if closing an executor is not possible in thread #6644

Merged
merged 2 commits into from
Jun 29, 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
1 change: 1 addition & 0 deletions distributed/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -7516,6 +7516,7 @@ async def test_wait_for_workers_updates_info(c, s):
"""


@pytest.mark.slow
@pytest.mark.parametrize("processes", [True, False])
def test_quiet_close_process(processes, tmp_path):
with open(tmp_path / "script.py", mode="w") as f:
Expand Down
25 changes: 18 additions & 7 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@
has_arg,
import_file,
in_async_call,
is_python_shutting_down,
iscoroutinefunction,
json_load_robust,
key_split,
Expand Down Expand Up @@ -1542,21 +1543,31 @@ async def close( # type: ignore
if executor is utils._offload_executor:
continue # Never shutdown the offload executor

def _close():
def _close(wait):
if isinstance(executor, ThreadPoolExecutor):
executor._work_queue.queue.clear()
executor.shutdown(wait=executor_wait, timeout=timeout)
executor.shutdown(wait=wait, timeout=timeout)
else:
executor.shutdown(wait=executor_wait)
executor.shutdown(wait=wait)

# Waiting for the shutdown can block the event loop causing
# weird deadlocks particularly if the task that is executing in
# the thread is waiting for a server reply, e.g. when using
# worker clients, semaphores, etc.
try:
await to_thread(_close)
except RuntimeError: # Are we shutting down the process?
_close() # Just run it directly
if is_python_shutting_down():
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one is neat!

# If we're shutting down there is no need to wait for daemon
# threads to finish
_close(wait=False)
else:
try:
await to_thread(_close, wait=executor_wait)
except RuntimeError: # Are we shutting down the process?
logger.error(
"Could not close executor %r by dispatching to thread. Trying synchronously.",
executor,
exc_info=True,
)
_close(wait=executor_wait) # Just run it directly

self.stop()
await self.rpc.close()
Expand Down