Skip to content

pylibcudf.Column: add device_buffer_size and register a dask.sizeof function for cudf-polars Column and DataFrame #18602

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

Merged
merged 21 commits into from
May 6, 2025
Merged
Show file tree
Hide file tree
Changes from 5 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
22 changes: 22 additions & 0 deletions python/cudf_polars/cudf_polars/containers/column.py
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,28 @@ def __init__(
self.name = name
self.set_sorted(is_sorted=is_sorted, order=order, null_order=null_order)

@functools.cached_property
def device_buffer_size(self) -> int:
"""
The total size of the device buffers used by the Column.

Returns
-------
Number of bytes.
"""

def _pylibcudf_device_size(col: plc.Column) -> int:
ret = 0
if col.data() is not None:
ret += col.data().nbytes
if col.null_mask() is not None:
ret += col.null_mask().nbytes
if col.children() is not None:
ret += sum(_pylibcudf_device_size(c) for c in col.children())
return ret

return _pylibcudf_device_size(self.obj)

@classmethod
def deserialize(
cls, header: ColumnHeader, frames: tuple[memoryview, plc.gpumemoryview]
Expand Down
12 changes: 12 additions & 0 deletions python/cudf_polars/cudf_polars/containers/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@

from __future__ import annotations

import functools
from functools import cached_property
from typing import TYPE_CHECKING, cast

Expand Down Expand Up @@ -49,6 +50,17 @@ def __init__(self, columns: Iterable[Column]) -> None:
self.column_map = {c.name: c for c in self.columns}
self.table = plc.Table([c.obj for c in self.columns])

@functools.cached_property
def device_buffer_size(self) -> int:
"""
The total size of the device buffers used by the Table.

Returns
-------
Number of bytes.
"""
return sum(c.device_buffer_size for c in self.columns)

def copy(self) -> Self:
"""Return a shallow copy of self."""
return type(self)(c.copy() for c in self.columns)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,12 +1,13 @@
# SPDX-FileCopyrightText: Copyright (c) 2024-2025, NVIDIA CORPORATION & AFFILIATES.
# SPDX-License-Identifier: Apache-2.0

"""Dask serialization."""
"""Dask function registrations such as serializers and dispatch implementations."""

from __future__ import annotations

from typing import TYPE_CHECKING, ClassVar, overload

from dask.sizeof import sizeof as sizeof_dispatch
from distributed.protocol import dask_deserialize, dask_serialize
from distributed.protocol.cuda import cuda_deserialize, cuda_serialize
from distributed.utils import log_errors
Expand All @@ -21,35 +22,35 @@

from cudf_polars.typing import ColumnHeader, DataFrameHeader

__all__ = ["SerializerManager", "register"]
__all__ = ["DaskRegisterManager", "register"]


class SerializerManager: # pragma: no cover; Only used with Distributed scheduler
class DaskRegisterManager: # pragma: no cover; Only used with Distributed scheduler
"""Manager to ensure ensure serializer is only registered once."""

_serializer_registered: bool = False
_registered: bool = False
_client_run_executed: ClassVar[set[str]] = set()

@classmethod
def register_serialize(cls) -> None:
def register_once(cls) -> None:
"""Register Dask/cudf-polars serializers in calling process."""
if not cls._serializer_registered:
from cudf_polars.experimental.dask_serialize import register
if not cls._registered:
from cudf_polars.experimental.dask_registers import register

register()
cls._serializer_registered = True
cls._registered = True

@classmethod
def run_on_cluster(cls, client: Client) -> None:
"""Run serializer registration on the workers and scheduler."""
"""Run register on the workers and scheduler once."""
if client.id not in cls._client_run_executed:
client.run(cls.register_serialize)
client.run_on_scheduler(cls.register_serialize)
client.run(cls.register_once)
client.run_on_scheduler(cls.register_once)
cls._client_run_executed.add(client.id)


def register() -> None:
"""Register dask serialization routines for DataFrames."""
"""Register dask serialization and dispatch functions."""

@overload
def serialize_column_or_frame(
Expand Down Expand Up @@ -128,3 +129,8 @@ def _(header: ColumnHeader, frames: tuple[memoryview, memoryview]) -> Column:
# Copy the second frame (the gpudata in host memory) back to the gpu
frames = frames[0], plc.gpumemoryview(rmm.DeviceBuffer.to_device(frames[1]))
return Column.deserialize(header, frames)

@sizeof_dispatch.register((Column, DataFrame))
def _(x: Column | DataFrame) -> int:
"""The total size of the device buffers used by the DataFrame or Column."""
return x.device_buffer_size
6 changes: 3 additions & 3 deletions python/cudf_polars/cudf_polars/experimental/parallel.py
Original file line number Diff line number Diff line change
Expand Up @@ -144,11 +144,11 @@ def get_scheduler(config_options: ConfigOptions) -> Any:
): # pragma: no cover; block depends on executor type and Distributed cluster
from distributed import get_client

from cudf_polars.experimental.dask_serialize import SerializerManager
from cudf_polars.experimental.dask_registers import DaskRegisterManager

client = get_client()
SerializerManager.register_serialize()
SerializerManager.run_on_cluster(client)
DaskRegisterManager.register_once()
DaskRegisterManager.run_on_cluster(client)
return client.get
elif scheduler == "synchronous":
from cudf_polars.experimental.scheduler import synchronous_scheduler
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
import rmm

from cudf_polars.containers import DataFrame
from cudf_polars.experimental.dask_serialize import register
from cudf_polars.experimental.dask_registers import register

# Must register serializers before running tests
register()
Expand Down
32 changes: 32 additions & 0 deletions python/cudf_polars/tests/experimental/test_dask_sizeof.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
# SPDX-FileCopyrightText: Copyright (c) 2024-2025, NVIDIA CORPORATION & AFFILIATES.
# SPDX-License-Identifier: Apache-2.0

from __future__ import annotations

import pyarrow as pa
import pytest
from dask.sizeof import sizeof

import pylibcudf as plc

from cudf_polars.containers import DataFrame
from cudf_polars.experimental.dask_registers import register

# Must register sizeof dispatch before running tests
register()


@pytest.mark.parametrize(
"arrow_tbl, size",
[
(pa.table([]), 0),
(pa.table({"a": [1, 2, 3], "b": [4, 5, 6], "c": [7, 8, 9]}), 9 * 8),
(pa.table({"a": [1, 2, 3]}), 3 * 8),
(pa.table({"a": ["a"], "b": ["bc"]}), 2 * 8 + 3),
(pa.table({"a": [1, 2, None]}), 88),
],
)
def test_dask_sizeof(arrow_tbl, size):
plc_tbl = plc.interop.from_arrow(arrow_tbl)
df = DataFrame.from_table(plc_tbl, names=arrow_tbl.column_names)
assert sizeof(df) == size
Loading