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

Worker event loop performance hampered by GIL/convoy effect #6325

Open
gjoseph92 opened this issue May 11, 2022 · 9 comments
Open

Worker event loop performance hampered by GIL/convoy effect #6325

gjoseph92 opened this issue May 11, 2022 · 9 comments
Labels
networking performance stability Issue or feature related to cluster stability (e.g. deadlock)

Comments

@gjoseph92
Copy link
Collaborator

Opening this to have a clearer issue to link to in other places. In #5258 (comment), we discovered that the worker's event loop performance could be heavily impacted when:

  • User task code, running in threads, holds the GIL (even if the tasks are quick)
  • Workload involves data transfer to other workers

For example, while running tasks involving lots of np.concatenates (which doesn't release the GIL), while simultaneously trying to send data to other workers, we found that 71% of the time, the worker's event loop was blocked (not idle) by the GIL.

gil

This is the "convoy effect", a longstanding issue in CPython: https://bugs.python.org/issue7946. Basically, the non-blocking socket.send() call (running in the event loop) releases the GIL and does a non-blocking write to the socket, which is nearly instantaneous. But then it needs to re-acquire the GIL, which some other thread now holds, and will hold for potentially a long time (the default thread-switch interval is 5ms). So this "non-blocking" socket.send is, effectively, blocking. If most of what the event loop is trying to do is send data/messages to other workers/the scheduler, then most of the time, the event loop will actually be blocked.

See the "GIL Gotchas" section in https://coiled.io/blog/better-shuffling-in-dask-a-proof-of-concept/ for an intuitive explanation of this involving toothbrushes.


This is clearly bad from a performance standpoint (it massively slows down data transfer between workers).

An open question is whether it's also a stability issue. I wouldn't be surprised to see messages like Event loop was unresponsive in Worker for 3.28s. This is often caused by long-running GIL-holding functions in worker logs, even when your tasks aren't long-running GIL-holding functions, but just GIL-holding functions at all.

No async code will perform well if the event loop is gummed up. I don't think the worker is any exception. I don't think that the worker is, or should have to be, designed to work reliably when the event loop is highly unresponsive. Instead, I think we should focus on ways to protect the event loop from being blocked. (The main way to do this is run tasks in subprocesses instead of threads, which I think is a good idea for all sorts of reasons.)

I'm not sure exactly how an unresponsive event loop impacts stability. In theory, it shouldn't matter—everything should still happen, in the same order, just really, really slowly. However, any sane-seeming timeouts will go out the window if the event loop is running 1,000x slower than expected. So in practice, I'm not surprised to see a blocked event loop causing things like #6324.

@fjetter
Copy link
Member

fjetter commented May 16, 2022

Do we have a reproducer for this effect? I encourage us to think about actual real world impact. The theory is sound and intuition tells us that a "blocked event loop is bad" but do we see this matter in reality? E.g. a real world impact of this would be a task stream with a lot of white space caused by workers not communicating sufficiently. Just having a blocked loop and blocked worker communication doesn't necessarily mean that we're bottlenecked. Workers are fetching data all the time, even prematurely.

FWIW: There is huge value in this issue, just from an informational POV. I've seen SSL read pop up in profiles countless times and this offers an explanation. The question is whether this is

@ericman93
Copy link

In my case, the scheduler isn't working. I have multiple workers that are not processing anything, and the tasks are piling up
Looking at the logs, I see the unresponsive eventloop log, but with much longer unresponsive time

distributed.core - INFO - Event loop was unresponsive in Scheduler for 9409.37s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

switching from pd.DataFrame to dd.DataFrame might help?

@crusaderky
Copy link
Collaborator

distributed.core - INFO - Event loop was unresponsive in Scheduler for 9409.37s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

This looks like a laptop that went into standby to me?

@ericman93
Copy link

@crusaderky unfortunately that's a k8s pod running dask-scheduler

@fjetter
Copy link
Member

fjetter commented May 23, 2022

@ericman93 I believe what you are experiencing is something else. If your event loop is blocked for that long, it is indeed bad but I'm very certain that this does not connect to GIL convoy effect. Would you please open a new issue describing your problem, setup and ideally provide some code

@crusaderky
Copy link
Collaborator

That line can be slightly misleading.
It is accurate for sync tasks. If you schedule an async task, however, you don't necessarily need a GIL-holding function to cause that message; time.sleep(9409) will have the same effect since async tasks are run in the same event loop as everything else.

e.g.

c = distributed.Client("localhost:8786")
async def f():
    time.sleep(5)
c.submit(f).result()
2022-05-24 11:14:43,193 - distributed.core - INFO - Event loop was unresponsive in Worker for 5.01s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

You would also get the same effect with a SingleThreadExecutor (no such class exists in the distributed package AFAIK, but it would make sense for debugging - it would be basically the same as pure dask with scheduler='single-threaded')

@ericman93
Copy link

ericman93 commented May 25, 2022

@crusaderky scheduler is blocked for the sleep time? Does the scheduler wait for the tasks to finish it its sync?

@crusaderky
Copy link
Collaborator

Apologies, I misread and didn't notice it was a scheduler log and not a worker log.

@mrocklin
Copy link
Member

@crusaderky scheduler is blocked for the sleep time? Does the scheduler wait for the tasks to finish it its sync?

No. There is no reason within Dask why the scheduler would be blocked for multiple hours. This is unrelated to the GIL convoy effect. Please raise a different issue.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
networking performance stability Issue or feature related to cluster stability (e.g. deadlock)
Projects
None yet
Development

No branches or pull requests

5 participants