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

Support shared memory with Queues #539

Closed
mrocklin opened this issue Sep 20, 2016 · 5 comments
Closed

Support shared memory with Queues #539

mrocklin opened this issue Sep 20, 2016 · 5 comments
Labels
enhancement Improve existing functionality or make things work better

Comments

@mrocklin
Copy link
Member

Currently the Client, Scheduler, and Workers always communicate over Tornado IOStreams, serializing messages as defined in distributed/protocol.py and distributed/core.py::dumps(). This is necessary if workers are on different machines or if the user wants to use multiple processes, but unnecessarily wasteful if the user wants to use a single process on a single machine. In this case it would be nice to pass Python objects directly between Client/Scheduler/Workers without serialization, perhaps using Tornado Queues (or something similar).

However, this isn't as simple as optionally switching IOStreams for Queues. We choose to serialize and deserialize data in certain places. For example when the client sends functions to the scheduler it serializes on the client-side but does not deserialize on the scheduler (the scheduler never deserializes functions or data). When the time is right the scheduler sends the serialized data to a worker which then deserializes the bytes back into a function. Because of this there are dumps/loads calls are not always symmetric. They are scattered throughout the code rather than isolated to a particular location.

This problem might be easier to solve if we consider subclassing IOStream. This would allow us to push some of the protocol.py logic into the Stream class and would also let us attach .dumps and .loads methods on to the object that could be something like pickle for socket-based streams and a no-op for queue-based streams.

cc @pitrou

@pitrou
Copy link
Member

pitrou commented Oct 20, 2016

I suspect the main optimization target here is task payload and results. Are those serialized on the Client and then deserialized on the Worker (or vice-versa)? Is it common to have Client and Workers on the same machine (why wouldn't you simply use Dask in that case)?

@pitrou pitrou added the enhancement Improve existing functionality or make things work better label Oct 20, 2016
@mrocklin
Copy link
Member Author

I suspect the main optimization target here is task payload and results.

Yes. Also the serialization of complex objects like functions or locks can also be convenient to avoid.

Are those serialized on the Client and then deserialized on the Worker (or vice-versa)?

Yes. Also Worker-to-Worker communications

Is it common to have Client and Workers on the same machine (why wouldn't you simply use Dask in that case)?

People do just use the dask.async scheduler in this case today. However the distributed scheduler has some non-distributed advantages like asynchronous computing, the concurrent.futures interface, the Bokeh dashboard, etc..

This would also lay groundwork for thinking about alternative IOStream implementations. This might help with issues like #400 (GPUs), communication between same-node processes, ZeroMQ, etc..

None of this is critical though and there may be more important things to work on.

@pitrou
Copy link
Member

pitrou commented Oct 20, 2016

I'm curious about serialization of lock objects, can you point me to where / why that happens?

@mrocklin
Copy link
Member Author

What I mean is that sometimes people want to send hard-to-serialized objects as part of their function call. One such example is if they want to use a lock. This happens for example when people try to use HDF5 as in this issue: dask/dask#1683

@mrocklin
Copy link
Member Author

This relates to #604

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement Improve existing functionality or make things work better
Projects
None yet
Development

No branches or pull requests

2 participants