Skip to content

Conversation

@fjetter
Copy link
Member

@fjetter fjetter commented Sep 10, 2021

This uses the event forwarding system introduced in #5217 to allow for forwarding logging information. Since tornado typically logs uncaught exceptions, this allows us for a cheap way to forward exceptions to users and/or keep them on the scheduler for dashboard visualizations.

Closes #5184

Builds on top of #5217

@fjetter fjetter force-pushed the error_logs_event_system branch from 382d468 to c86e0af Compare September 13, 2021 09:54
Comment on lines +90 to +92
loggers_to_register = dask.config.get("distributed.admin.log-events")

for logger_name, level in loggers_to_register.items():
Copy link
Member

Choose a reason for hiding this comment

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

Since each Node subclass (e.g. Worker) calls self._setup_logging(logger) with their own logger as input

self._setup_logging(logger)

I think we want to only set up an EventHandler for the specific logger that's input to _setup_logging instead of looping through them all each time _setup_logging is called

Copy link
Member Author

Choose a reason for hiding this comment

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

This is currently very awkward but the loggers passed in this methods are only distributed.scheduler and distributed.worker. Most interesting exception logs actually pop up in distributed.core and tornado.application.

The proper way to do this would be to attach the EventHandler to the logger distributed and ensure that all loggers we're using are propagating their msgs https://docs.python.org/3.8/library/logging.html#logging.Logger.propagate This is the default of the stdlib but it is user configuration and we are overwriting it for some reason for "old style" logging config

logger.propagate = False

Without this knowledge, the only sane way I see is to list the registered loggers explicitly. The same goes for the deque handler, btw, but I didn't touch it, yet.

@graingert graingert force-pushed the error_logs_event_system branch from e8b7047 to 7f6714f Compare May 10, 2023 14:26
@github-actions
Copy link
Contributor

github-actions bot commented May 10, 2023

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

       20 files  ±  0         20 suites  ±0   11h 9m 51s ⏱️ + 2m 5s
  3 706 tests +  2    3 596 ✔️  -   2     106 💤 ±0  4 +4 
35 846 runs  +20  34 094 ✔️ +16  1 748 💤 ±0  4 +4 

For more details on these failures, see this check.

Results for commit 3af381b. ± Comparison against base commit 566fd1f.

♻️ This comment has been updated with latest results.

graingert added 3 commits May 11, 2023 13:44
asyncio.get_running_loop() is worker.loop.asyncio_loop is a more
reliable way of determining if we are on the worker's event loop thread

this also binds self.loop early so that log_event can be called
immediately after _setup_logging()
@graingert graingert force-pushed the error_logs_event_system branch from 37e4584 to 03b0a5d Compare June 28, 2023 14:25
@graingert
Copy link
Member

@fjetter this is working now - can you review the changes I've added, especially around the use of asyncio.get_running_loop() is worker.loop.asyncio_loop instead of worker.thread_id == threading.get_ident()

asyncio_loop = asyncio.get_running_loop()
except RuntimeError:
return False
return self._worker.loop.asyncio_loop is asyncio_loop
Copy link
Member

Choose a reason for hiding this comment

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

this wasn't covered either that's odd

Copy link
Member

Choose a reason for hiding this comment

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

I don't think this is possible to cover, if the Actor binds self._worker then it has self._client bound

Copy link
Member

Choose a reason for hiding this comment

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

This might be worthwhile simplifying then, but that's out of scope for this PR.

@graingert graingert requested a review from jrbourbeau July 26, 2023 10:44
@hendrikmakait hendrikmakait self-requested a review July 28, 2023 08:37
Copy link
Member

@hendrikmakait hendrikmakait left a comment

Choose a reason for hiding this comment

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

This code generally looks good to me. I have some small nits, and I would like us to test that the configuration works as expected.

Deadline,
DequeHandler,
EventHandler,
TimeoutError,
Copy link
Member

Choose a reason for hiding this comment

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

Why is the import of TimeoutError needed?


class SupportsLogEvent(Protocol):
def log_event(self, topic: list[str], msg: str) -> object:
...
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
...
... # pragma: nocover

asyncio_loop = asyncio.get_running_loop()
except RuntimeError:
return False
return self._worker.loop.asyncio_loop is asyncio_loop
Copy link
Member

Choose a reason for hiding this comment

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

This might be worthwhile simplifying then, but that's out of scope for this PR.

logger.addHandler(self._deque_handler)
weakref.finalize(self, logger.removeHandler, self._deque_handler)

loggers_to_register = dask.config.get("distributed.admin.log-events")
Copy link
Member

Choose a reason for hiding this comment

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

I think it would be good to have another test that checks that the configuration works as expected (e.g., changing the configuration will result in some log being propagated or not).

@fjetter fjetter closed this Jul 10, 2025
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.

[Idea/Draft/Proposal] Exception handling for server exceptions

4 participants