Skip to content
Merged
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
51 changes: 47 additions & 4 deletions python/cuml/cuml/dask/cluster/kmeans.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,8 @@
#

import cupy as cp
import dask
import dask.array as da
from dask.distributed import get_worker
from raft_dask.common.comms import Comms, get_raft_comm_state

Expand Down Expand Up @@ -159,10 +161,51 @@ def fit(self, X, sample_weight=None):

comms.destroy()

models = [res.result() for res in kmeans_fit]
first = models[0]
first.labels_ = cp.concatenate([model.labels_ for model in models])
first.inertia_ = sum(model.inertia_ for model in models)
# Collect the full model from only the first worker (for
# cluster_centers_ etc). Since cluster centers are synchronized
# via NCCL, all workers have identical copies — pulling more
# than one would waste memory (N * n_clusters * n_features * 4B).
#
# Labels stay distributed as a dask.array to avoid transferring
# per-sample data to the client. Only the scalar inertia values
# are gathered.
first = kmeans_fit[0].result()
workers = list(data.worker_to_parts.keys())

remote_labels = [
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thank you for putting up a PR fix so quickly!

May I know how large this remote_labels variable is if dataset has 1 billion rows? Will that blow up scheduler memory?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

labels_ is a 1D array of length n_samples (in the dask case, split across N-workers). The dtype is typically int32, which brings you to 4 GiB for the array total.

In most deployments of dask the data doesn't go through the scheduler, it goes directly worker->client (also note that in cases where the scheduler runs on the same node as the client this distinction is meaningless). So you care more about the memory capacity client-side than on the scheduler itself.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thanks for clarifying! In my deployment, the scheduler does run on the same node as the client, so they share the same GPU memory. That said, since it's a 1D int32 array, 4 GB seems manageable compared to the previous issue of collecting all workers' copies of the centroid matrix.

self.client.submit(getattr, f, "labels_", workers=[w])
for f, w in zip(kmeans_fit[1:], workers[1:])
]
remote_inertias = [
self.client.submit(getattr, f, "inertia_", workers=[w])
for f, w in zip(kmeans_fit[1:], workers[1:])
]

self.inertia_ = first.inertia_ + sum(
self.client.gather(remote_inertias)
)

labels_dtype = first.labels_.dtype
label_chunks = [
da.from_delayed(
dask.delayed(first.labels_, pure=True, traverse=False),
shape=(first.labels_.shape[0],),
dtype=labels_dtype,
meta=cp.zeros(0, dtype=labels_dtype),
)
] + [
da.from_delayed(
f,
shape=(float("nan"),),
dtype=labels_dtype,
meta=cp.zeros(0, dtype=labels_dtype),
)
for f in remote_labels
]
self.labels_ = da.concatenate(
label_chunks, allow_unknown_chunksizes=True
)

self._set_internal_model(first)

return self
Expand Down
Loading