Open markcoletti opened 4 years ago
I suspect that the bottlenecks are going to be in the scheduler. For example, just monitoring all the worker heartbeats is a burdensome task, I imagine.
Yes, I suspect that that's exactly the kind of thing that we'll run into. Actually, we already did with previous systems. Today worker heartbeats slow down based on the number of workers. There are probably other issues that will arise though that we haven't yet accounted for.
@benjha UCX-py is a generic UCX library for Python. It is useful in dask-cuda as UCX supports NVLINK but it is also useful for pure Infiniband setups.
Yeah to complement what @jacobtomlinson already said @benjha. Dask already has UCX-Py support baked in.
Dask-CUDA is really designed for single node deployments atm. Though there may be pieces we pull from it.
That all being said, would be curious what @piprrr is able to accomplish just by adding UCX-Py.
Just in case you are not aware but the heartbeat interval is limited to a max of 5 seconds. With your 10k+ dask workers that you are trying to run your scheduler has less than 0.5ms per heartbeat before just heartbeats are causing delays for all scheduler actions.
In our fork we have limited the number of heartbeats per second to 50 which has helped quite a bit in large jobs. You can find the code for the heartbeat interval at https://github.com/dask/distributed/blob/6ee7f899d1bc686ba4910c08509c7f2e5345506d/distributed/scheduler.py#L5561 Note that not limiting the heartbeat interval requires that you also update code that relies on the heartbeat interval (e.g. https://github.com/dask/distributed/blob/6ee7f899d1bc686ba4910c08509c7f2e5345506d/distributed/scheduler.py#L5301).
Thanks for pointing that out. I opened https://github.com/dask/distributed/issues/3877 to make configuring the heartbeat easier.
Oh yeah, that's super helpful. Thanks!
On Wed, Jun 10, 2020 at 4:06 AM Tom Augspurger notifications@github.com wrote:
Thanks for pointing that out. I opened #3877 https://github.com/dask/distributed/issues/3877 to make configuring the heartbeat easier.
— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/dask/distributed/issues/3691#issuecomment-641929921, or unsubscribe https://github.com/notifications/unsubscribe-auth/AACKZTC57O7RSMT6CFLLFBDRV5SNBANCNFSM4ME452QQ .
This is a very useful feature, and I'll be happy to test it on Summit, though it may be a little while before I can do so. This does address my suspicion that it was the scheduler's heartbeat management that was causing us problems.
I don't think that this will help you. All that dask-mpi does is use MPI to communicate the address of the scheduler, and then it connects as normal. We could make a proper Dask MPI communication system, but that would require a bit of work. I'm guessing a week or two of a full-time dev who had done this before.
Could you please explain such a Dask MPI communication system? Do you have something similar to UCX-Py/TCP kind of backends that already exist in mind or is this going to be complete revamp of Dask Distributed package.
Thanks.
Dask has extensible Comms: https://distributed.dask.org/en/latest/communications.html
This is what we implement when we add things like UCX. We would use mpi4py to implement this API and then Dask would use MPI for communication. However, it would still be Dask, we're just using send/recv for point-to-point communication.
On Mon, Jun 29, 2020 at 5:15 PM aamirshafi notifications@github.com wrote:
I don't think that this will help you. All that dask-mpi does is use MPI to communicate the address of the scheduler, and then it connects as normal. We could make a proper Dask MPI communication system, but that would require a bit of work. I'm guessing a week or two of a full-time dev who had done this before.
Could you please explain such a Dask MPI communication system? Do you have something similar to UCX-Py/TCP kind of backends that already exist or is this going to be complete revamp of Dask Distributed package.
Thanks.
— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/dask/distributed/issues/3691#issuecomment-651436250, or unsubscribe https://github.com/notifications/unsubscribe-auth/AACKZTALKXQIOIYYJGXJHHTRZEVDHANCNFSM4ME452QQ .
Thanks for the response.
Dask's communication layer has connect() and listen() functions that do not have direct parallels in the MPI world. Do you have any thoughts on how to handle connection establishment with MPI? I am assuming that something like dask-mpi will be used to start schedulers and workers but in that case, MPI and all processes will already be initialized and there will be no need for connect() and listen().
Also in Dask Distributed, workers create new connections amongst themselves while the client code is executing. Is there a way to configure Dask Distributed to not make dynamic connections between workers on the fly and instead create them in the beginning only?
Yes. We would just refer to the existing connections. Is this something that you're planning on working on? If so, we should probably move this conversation to a separate issue. We're drifting a little bit off topic from Summit I think.
On Mon, Jun 29, 2020, 8:49 PM aamirshafi notifications@github.com wrote:
Thanks for the response.
Dask's communication layer has connect() and listen() functions that do not have direct parallels in the MPI world. Do you have any thoughts on how to handle connection establishment with MPI? I am assuming that something like dask-mpi will be used to start schedulers and workers but in that case, MPI and all processes will already be initialized and there will be no need for connect() and listen().
Also in Dask Distributed, workers create new connections amongst themselves while the client code is executing. Is there a way to configure Dask Distributed to not make dynamic connections between workers on the fly and instead create them in the beginning only?
— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/dask/distributed/issues/3691#issuecomment-651512191, or unsubscribe https://github.com/notifications/unsubscribe-auth/AACKZTBG2ABUWAPHZF7JJ3DRZFOFRANCNFSM4ME452QQ .
This github distributed dask issue is to track progress and resolve problems associated with scaling dask to run on all the nodes on Oak Ridge National Laboratory (ORNL)’s Summit supercomputer. Benjamin Zaitlen of Nvidia suggested that I create this GitHub issue on the RAPIDS-GoAi Slack when I first shared what we were trying to do there; he communicated with the dask development team, and they posed this approach of creating this issue in the project’s GitHub issue tracker.
At ORNL we are using distributed dask to evaluate unique driving scenarios using the CARLA driving simulator to improve a deep-learner model for driving a virtual autonomous vehicle. As part of this effort, we are stress testing dask on Summit by gradually adding more Summit nodes to submitted jobs, fixing problems at the point where a given number of nodes has caused issues, and then increasing the number of nodes to address the next set of problems.
As of April 9th, the maximum number of Summit nodes from which we’ve gotten a successful run is 402 nodes with six dask workers each, for a total of 2,412 dask workers. We have a job pending with 921 nodes that will push the total number of dask workers to 5,526 dask workers. Eventually we’d like dask running on all 4,608 Summit nodes, which would have 27,648 dask workers.
In parallel, we’re developing an open-source evolutionary algorithm framework, LEAP, with colleagues at George Mason University and MITRE that uses dask to concurrently evaluate individuals. We have used LEAP as the vehicle for stress testing dask on Summit. The first test problem was a MAX ONES instance where the fitness was the total number of ones in a given bit string. Since this problem was too trivial for stress testing, we created a new problem that had a single integer as the individual's phenome (that is, genes that have been decoded into something meaningful to the problem, in this case an integer) that would correspond to how many seconds to sleep during evaluation. This means an individual that had a genome of 10 bits of all ones would cause the dask worker to sleep for about 17 minutes during evaluation. Moreover, this was a maximization problem, so the longer the sleep periods, the fitter the individual. (We could probably call this test the UNDERGRAD problem. 😉 )
In the comments below, I’ll detail what I’ve done so far, and relate some of the problems I’ve encountered on this journey. I welcome comments, tips, suggestions, criticisms, and help with the next inevitable set of problems associated with this non-trivial task. 🙂
Cheers,
Mark Coletti, ORNL