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

Fix test nanny timeout #8847

Merged
merged 7 commits into from
Sep 3, 2024
Merged
Show file tree
Hide file tree
Changes from 3 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
43 changes: 25 additions & 18 deletions distributed/nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -745,19 +745,24 @@ async def start(self) -> Status:
os.environ.update(self.pre_spawn_env)

try:
await self.process.start()
except OSError:
logger.exception("Nanny failed to start process", exc_info=True)
# NOTE: doesn't wait for process to terminate, just for terminate signal to be sent
await self.process.terminate()
self.status = Status.failed
try:
msg = await self._wait_until_connected(uid)
except Exception:
# NOTE: doesn't wait for process to terminate, just for terminate signal to be sent
await self.process.terminate()
self.status = Status.failed
raise
try:
await self.process.start()
except OSError:
# This can only happen if the actual process creation failed, e.g.
# multiprocessing.Process.start failed. This is not tested!
logger.exception("Nanny failed to start process", exc_info=True)
# NOTE: doesn't wait for process to terminate, just for terminate signal to be sent
await self.process.terminate()
self.status = Status.failed
try:
msg = await self._wait_until_connected(uid)
except Exception:
# NOTE: doesn't wait for process to terminate, just for terminate signal to be sent
await self.process.terminate()
self.status = Status.failed
raise
finally:
self.running.set()
if not msg:
return self.status
self.worker_address = msg["address"]
Expand Down Expand Up @@ -830,22 +835,24 @@ async def kill(
"""
deadline = time() + timeout

if self.status == Status.stopped:
return
if self.status == Status.stopping:
await self.stopped.wait()
return
# If the process is not properly up it will not watch the closing queue
# and we may end up leaking this process
# Therefore wait for it to be properly started before killing it
if self.status == Status.starting:
await self.running.wait()

assert self.status in (
Status.stopping,
Status.stopped,
Status.running,
Status.failed, # process failed to start, but hasn't been joined yet
Status.closing_gracefully,
), self.status
if self.status == Status.stopped:
return
if self.status == Status.stopping:
await self.stopped.wait()
return
self.status = Status.stopping
logger.info("Nanny asking worker to close. Reason: %s", reason)

Expand Down
31 changes: 31 additions & 0 deletions distributed/tests/test_nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -582,6 +582,37 @@ async def test_worker_start_exception(s):
assert logs.getvalue().count("ValueError: broken") == 1, logs.getvalue()


@gen_cluster(nthreads=[])
async def test_worker_start_exception_while_killing(s):
nanny = Nanny(s.address, worker_class=BrokenWorker)

async def try_to_kill_nanny():
while not nanny.process or nanny.process.status not in (
Status.starting, # this is what we want
# Status.failed # we might've missed it already
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
# Status.failed # we might've missed it already

):
await asyncio.sleep(0)
await nanny.kill()

kill_task = asyncio.create_task(try_to_kill_nanny())
with captured_logger(logger="distributed.nanny", level=logging.WARNING) as logs:
with raises_with_cause(
RuntimeError,
"Nanny failed to start",
RuntimeError,
"BrokenWorker failed to start",
):
async with nanny:
pass
await kill_task
assert nanny.status == Status.failed
# ^ NOTE: `Nanny.close` sets it to `closed`, then `Server.start._close_on_failure` sets it to `failed`
assert nanny.process is None
assert "Restarting worker" not in logs.getvalue()
# Avoid excessive spewing. (It's also printed once extra within the subprocess, which is okay.)
assert logs.getvalue().count("ValueError: broken") == 1, logs.getvalue()


@gen_cluster(nthreads=[])
async def test_failure_during_worker_initialization(s):
with captured_logger(logger="distributed.nanny", level=logging.WARNING) as logs:
Expand Down
Loading