Skip to content

Raise when workers initialization times out#2784

Merged
mrocklin merged 4 commits intodask:masterfrom
TomAugspurger:2781-death-timer
Jun 19, 2019
Merged

Raise when workers initialization times out#2784
mrocklin merged 4 commits intodask:masterfrom
TomAugspurger:2781-death-timer

Conversation

@TomAugspurger
Copy link
Copy Markdown
Member

This changes Worker / Nanny startup to raise when they timeout.

This bubbles up to the dask-worker CLI.

Closes #2781

This changes Worker / Nanny startup to raise when they timeout.

This bubbles up to the `dask-worker` CLI.

Closes dask#2781
@TomAugspurger
Copy link
Copy Markdown
Member Author

Here's the output

bash-5.0$ dask-worker 192.168.7.26:8786 --death-timeout=2
2019-06-18 15:29:40,340 distributed.nanny[93624] INFO         Start Nanny at: 'tcp://192.168.7.20:52499'
2019-06-18 15:29:41,149 distributed.diskutils[93700] INFO Found stale lock file and directory '/Users/taugspurger/sandbox/distributed/worker-_8p_3v5j', purging
2019-06-18 15:29:41,731 distributed.worker[93700] INFO       Start worker at:   tcp://192.168.7.20:52500
2019-06-18 15:29:41,732 distributed.worker[93700] INFO          Listening to:   tcp://192.168.7.20:52500
2019-06-18 15:29:41,732 distributed.worker[93700] INFO          dashboard at:         192.168.7.20:52501
2019-06-18 15:29:41,733 distributed.worker[93700] INFO Waiting to connect to:    tcp://192.168.7.26:8786
2019-06-18 15:29:41,733 distributed.worker[93700] INFO -------------------------------------------------
2019-06-18 15:29:41,734 distributed.worker[93700] INFO               Threads:                          8
2019-06-18 15:29:41,734 distributed.worker[93700] INFO                Memory:                   17.18 GB
2019-06-18 15:29:41,735 distributed.worker[93700] INFO       Local Directory: /Users/taugspurger/sandbox/distributed/worker-wy_lygiz
2019-06-18 15:29:41,735 distributed.worker[93700] INFO -------------------------------------------------
2019-06-18 15:29:42,354 distributed.nanny[93624] INFO Closing Nanny at 'tcp://192.168.7.20:52499'
2019-06-18 15:29:42,356 distributed.worker[93700] INFO Stopping worker at tcp://192.168.7.20:52500
2019-06-18 15:29:42,411 distributed.nanny[93624] ERROR Timed out connecting Nanny '<Nanny: None, threads: 8>' to scheduler 'tcp://192.168.7.26:8786'
Traceback (most recent call last):
  File "/Users/taugspurger/sandbox/distributed/distributed/nanny.py", line 303, in instantiate
    timedelta(seconds=self.death_timeout), self.process.start()
  File "/Users/taugspurger/Envs/dask-dev/lib/python3.7/site-packages/tornado/gen.py", line 1133, in run
    value = future.result()
tornado.util.TimeoutError: Timeout
2019-06-18 15:29:42,411 distributed.dask_worker[93624] INFO End worker
Traceback (most recent call last):
  File "/Users/taugspurger/.virtualenvs/dask-dev/bin/dask-worker", line 11, in <module>
    load_entry_point('distributed', 'console_scripts', 'dask-worker')()
  File "/Users/taugspurger/sandbox/distributed/distributed/cli/dask_worker.py", line 387, in go
    main()
  File "/Users/taugspurger/Envs/dask-dev/lib/python3.7/site-packages/click/core.py", line 764, in __call__
    return self.main(*args, **kwargs)
  File "/Users/taugspurger/Envs/dask-dev/lib/python3.7/site-packages/click/core.py", line 717, in main
    rv = self.invoke(ctx)
  File "/Users/taugspurger/Envs/dask-dev/lib/python3.7/site-packages/click/core.py", line 956, in invoke
    return ctx.invoke(self.callback, **ctx.params)
  File "/Users/taugspurger/Envs/dask-dev/lib/python3.7/site-packages/click/core.py", line 555, in invoke
    return callback(*args, **kwargs)
  File "/Users/taugspurger/sandbox/distributed/distributed/cli/dask_worker.py", line 378, in main
    raise TimeoutError("Timed out starting worker.") from None
tornado.util.TimeoutError: Timed out starting worker.

Could maybe cleanup that second traceback a bit, but OK for now I hope.

logger.info("-" * 49)
while True:
if self.death_timeout and time() > start + self.death_timeout:
logger.exception(
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I can't easily comment on it here, but down in https://github.com/dask/distributed/pull/2784/files#diff-3f78e3c9a9a81b6355c8e594dce0b8f3R766 we also may handle TimeoutError. In that case we logger.info and move on.

I'm not quite sure when that is reached, but I think since we're under this while True we'll still end up here.

except (KeyboardInterrupt, TimeoutError):
except TimeoutError:
# We already log the exception in nanny / worker. Don't do it again.
raise TimeoutError("Timed out starting worker.") from None
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Whoa, raise from? New syntax for me.

Copy link
Copy Markdown
Member Author

@TomAugspurger TomAugspurger Jun 18, 2019

Choose a reason for hiding this comment

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

Welcome to the brave new world of Python 3 :)

raise ... from None specifically is useful when you think the original traceback isn't helpful, which I think it isn't here. Otherwise, you can raise from e if you want to include it.

@TomAugspurger
Copy link
Copy Markdown
Member Author

@mrocklin the previous behavior of gracefully closing the worker, without error, was tested. I've removed those tests since I changed the behavior to raise when a worker times out. Does that sound OK? Alternatively, I could somehow check in the dask-worker command that a timeout has occurred, and change the exit code there.

@gen_cluster(client=False, nthreads=[])
def test_nanny_death_timeout(s):
yield s.close()
w = yield Nanny(s.address, death_timeout=1)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I think that this test is still valid, but you just want to add something like the following:

w = Nanny(s.address, death_timeout=1
with pytest.raises(gen.TimeoutError):
    yield w

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Got it, thanks.

I've removed my new tests, since I think they're duplicative of these now.

@TomAugspurger
Copy link
Copy Markdown
Member Author

All green.

@mrocklin mrocklin merged commit eba954b into dask:master Jun 19, 2019
@mrocklin
Copy link
Copy Markdown
Member

Thanks @TomAugspurger !

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

dask-worker timeout should exit with non-zero status code?

2 participants