Cluster lifecycle management
See original GitHub issueTL;DR I’m going to play around with adding lifecycle features for Dask clusters for the next couple of days. This will likely result in design discussions and eventually many PRs to many repos, so wanted to track the overall conversation here.
My primary goal is to enable this. Then expand to many other uses.
$ dask-cluster list
CLUSTER ID TYPE WORKERS CREATED STATUS
dask-123abc dask_kubernetes.KubeCluster 6 4 hours ago Running
$ dask-cluster scale dask-123abc 10
Scaled dask-123abc to 10 workers
$ dask-cluster delete dask-123abc
Are you sure Y/n: Y
Cluster dask-123abc deleted
I’ve spent a lot of time recently working on various cluster managers and feel like I’m in a good headspace to think about Dask cluster management as a whole. This issue may become a little sprawling as I try and unpack what I’ve been thinking about, so apologies!
Cluster lifecycle management is currently tricky and fractured. I would like to add some new features which will act as building blocks to improve things and gradually bring things in line.
Current state
In the Dask ecosystem LocalCluster
is the prototypical cluster implementation. It is often the first that folks come into contact with and it is used as a reference implementation for others. However it contains one core assumption that I’ve found to be problematic compared to other cluster managers; the lifecycle of the cluster is tied to the process which created it. If you create a LocalCluster
in ipython and then close ipython the cluster will close because any subprocesses will exit.
Cluster implementations such as jobqueue, kubernetes, cloudprovider, gateway, etc all create external resources which are not tied to the lifecycle of the process. Jobs are submitted to scheduling systems, cloud resources are created, etc. Many of these projects attempt to replicate the functionality of LocalCluster
by handling lifecycle within the cluster object.
- Dask Kubernetes and Dask Cloudprovider attempt to use
finalize
hooks to destroy any dangling resources. However this can be flaky and leave things running in some failure modes. - Some of the newer libraries in Dask Cloudprovider do not attempt clean up at all and instead heavily document closing the cluster and the use of context managers. Some have manual CLI clean up commands for recovery from failure.
- Dask gateway has more advanced lifecycle management, but has reimplemented many existing cluster managers in order to do this.
- The
HelmCluster
manager in Dask Kubernetes is completely divorced from the lifecycle of the cluster, leaving the user to create and destroy the cluster with thehelm
CLI tool and only provides utility features such as connection, scaling and logging. - Dask Jobqueue leaves dangling jobs to run out their allocation in case of failure.
Some cluster managers create schedulers local to the Python process and workers remotely. Some create everything remotely. Some are configurable and provide both options.
Another challenge around lifecycle is that some use cases, such as pipelines, want to create a cluster as a discrete workflow step. Subsequent steps would connect to the cluster to make use of it, and the final step would close and remove the cluster. This process is fine if all steps exist within one Python script, but in workflow engines where multiple scripts are called sequentially this doesn’t work. These use cases require explicit control over the cluster lifecycle, independent of how it was constructed.
Proposal
I would like to move forward assuming that clusters exist independently of the process that created them. To do this I would like to add new tooling to manage the lifecycle of clusters. This does mean that LocalCluster
and perhaps similar implementations like SSHCluster
would become the non-typical cases, but I can still think of situations where this would be useful for them.
In order to manage the lifecycle of the cluster we need to be able to create
, list
, scale
and delete
clusters. These operations should be atomic and not depend on state from prior steps.
Creation
We are already pretty mature in the creation space. There are a variety of ways to create clusters, mostly using Python SpecCluster
subclasses. However I propose that we move to this being an independent operation and in the case where the Python session closes the cluster remains in existence.
For Dask kubernetes this would require the removal of the finalization code and switching the scheduler mode to remote. We support remote schedulers today, but this change would make the local scheduler mode much less desirable.
For Dask Cloudprovider we would do the same for those that attempt cleanup. Newer classes like AzureVMCluster
are already in this position, which does make them a little scary to use without the rest of the lifecycle steps.
Dask Jobqueue would likely require a move to remote schedulers.
The core SSHCluster
implementation could detach the SSH subprocesses from the parent, ensuring they remain alive if Python exits.
LocalCluster
would become an outlier in this.
Listing
It would be very valuable to be able to list clusters that are available to you. This is where I expect the bulk of this effort will go as this is a non-trivial task.
Listing would involve discovery of existing clusters and reconstruction of a Cluster
subclass object for each cluster.
Discovery
In order to discover clusters they will need a unique identifier. In Dask cloudprovider we assign a uuid to each cluster object. This could be upstreamed to SpecCluster
.
Each implementation would have to handle its own discovery. Therefore I propose adding some core discovery utilites to distributed which dispatch to other libraries via registered entrypoints.
To use Dask kubernetes as an example here I would imagine that we would have some method like dask_kubernetes.discover
which would be registered as a distributed discovery entrypoint in the setup.py
file. The method would perhaps list all pods looking for ones with the Dask scheduler label. Then return a list of tuples containing the cluster ID and class that can represent that cluster (KubeCluster
in this case).
>>> from dask.distributed import discover_cluster_uuids
>>> discover_cluster_uuids()
[("dask-123abc", dask_kubernetes.KubeCluster)]
Reconstructing a cluster class
The next required component here would be for dask_kubernetes.KubeCluster
to be able to recreate a cluster object from a UUID. This could be done via a class method (probably defined on SpecCluster but not implemented) which takes a UUID and then discovers the rest of the things it needs to create the cluster from that.
>>> from dask_kubernetes import KubeCluster
>>> cluster = KubeCluster.from_uuid("dask-123abc")
To rebuild the class it would likely need to create scheduler and worker objects, these too would need to be reconstructed from their UUIDs. Therefore the ProcessInterface
class should probably also define but not implement a from_uuid
method. The KubeCluster.from_uuid
method would need to speak to the Kubernetes API to discover all the pods relating to the UUID and recreate class representations for them.
I can imagine there being state held in the cluster manager which is not necessarily possible to reconstruct. For example the worker pod spec. This kind of state could be handed to the scheduler during the creation for safe keeping, then requested back at reconstruction time. This may need a little more thinking about.
By implementing a consistent from_uuid
method this would also allow us to implement a more useful function in distributed which both discovers and reconstructs all classes. It would first discover the UUIDs and classes, then reconstruct them on our behalf returning a list of cluster managers.
>>> from dask.distributed import discover_clusters
>>> discover_clusters()
[<dask_kubernetes.KubeCluster at 0x10ac65190>]
Use in other tools
With this functionality in place dask.distributed.discover_clusters
could then be used in other tools.
One example could be a CLI utility for interacting with clusters.
$ dask-cluster list
CLUSTER ID TYPE WORKERS CREATED STATUS
dask-123abc dask_kubernetes.KubeCluster 6 4 hours ago Running
This could also be used in the Jupyter Lab Extension for listing existing clusters.
This could also be used in Dask Gateway to enable the use of existing code.
Clusters that cannot be reconstructed
For LocalCluster
it does not make sense to reconstruct the LocalCluster
object, because the scheduler and workers are subprocesses of an existing cluster object. However it could be useful to be able to create a proxy object to this cluster.
It is common when using Dask interactively in Jupyter to accidentally create multiple clusters. If you have a cell which defines a cluster object and outputs it directly and you accidentally run it multiple times you will get multiple clusters, but lose the reference to the original This is commonly seen with the warning about starting Dask on an alternative port because 8786-8787 are busy.
Being able to discover these hanging clusters and close them without restarting your notebook kernel could be very useful.
Scaling
Once we have cluster object representations we can scale them as normal with the .scale()
or .adapt()
methods.
One slight problem here is that a cluster object is required for the adaptive functionality to work. So in the use case where a workflow engine starts a cluster, uses a cluster and destroys a cluster in discrete scripts would not benefit from adaptive functionality unless the intermediate step reconstructed the cluster object. This is possible but potentially confusing to the user.
Perhaps as a longer term goal adaptivity should be handled entirely by the scheduler via a scheduler plugin.
Scaling could also be made available via the proposed dask-cluster
utility.
$ dask-cluster scale dask-123abc 10
Scaled dask-123abc to 10 workers
Destruction
Once we are able to easily reconstruct cluster objects it becomes trivial to clean them up via the .close()
method.
For libraries such as Dask cloudprovider where cluster resources can be expensive and non-trivial to clean up in failure cases this would be a game changer. Especially in terms of user anxiety. Knowing that you can easily delete your clusters and not have to worry about accidentally leaving them dangling with no route to remove them means folks would be more confident with these tools.
Again this could also be exposed via the CLI.
$ dask-cluster delete dask-123abc
Are you sure Y/n: Y
Cluster dask-123abc deleted
Cleaning broken clusters
It can also be possible to have cluster creation fail in a way where you do not have a working cluster, but you do have some resources left over. Again Dask Cloudprovider is a primary case for this where things like virtual networks, security groups, permission roles, etc can be left over after the cluster has closed. Or worse some virtual machines which are idling with no Dask processes running and still costing money.
This could be handled by discovery finding UUIDs for clusters, even if there is only one security group left over from that cluster. Then the from_uuid
method would return an instance of that cluster but with cluster.status
set to something like distributed.core.Status.broken
. Then the close()
method would handle cleanup of all resources.
$ dask-cluster list
CLUSTER ID TYPE WORKERS CREATED STATUS
dask-456def dask_cloudprovider.aws.FargateCluster 0 4 hours ago Broken
$ dask-cluster delete dask-456def
Are you sure Y/n: Y
Cluster dask-456def deleted
Next steps
I think the next thing to do is for me to play around a little with implementing some discovery and from_uuid
methods. Then implement the CLI.
I aim to have some demo ready for the weekly maintainers sync tomorrow to start a discussion on design and future plans. But in the mean time any feedback here is welcome.
Issue Analytics
- State:
- Created 3 years ago
- Reactions:5
- Comments:7 (6 by maintainers)
Top GitHub Comments
Here’s a demo of where I’m up to with this prototype functionality.
Agreed generally. I think that the API points listed above are valid without a database;