diff --git a/docs/source/examples/best-practices.rst b/docs/source/examples/best-practices.rst index 2de3809c..a61b40e7 100644 --- a/docs/source/examples/best-practices.rst +++ b/docs/source/examples/best-practices.rst @@ -44,6 +44,14 @@ We also recommend allocating most, though not all, of the GPU memory space. We d Additionally, when using `Accelerated Networking`_ , we only need to register a single IPC handle for the whole pool (which is expensive, but only done once) since from the IPC point of viewer there's only a single allocation. As opposed to just using RMM without a pool where each new allocation must be registered with IPC. +Spilling from Device +~~~~~~~~~~~~~~~~~~~~ + +Dask CUDA offers several different ways to enable automatic spilling from device memory. +The best method often depends on the specific workflow. For classic ETL workloads with +`Dask cuDF `_, cuDF spilling is usually +the best place to start. See `spilling`_ for more details. + Accelerated Networking ~~~~~~~~~~~~~~~~~~~~~~ diff --git a/docs/source/spilling.rst b/docs/source/spilling.rst index a237adf7..1087f4a2 100644 --- a/docs/source/spilling.rst +++ b/docs/source/spilling.rst @@ -105,3 +105,68 @@ type checking doesn't: Thus, if encountering problems remember that it is always possible to use ``unproxy()`` to access the proxied object directly, or set ``DASK_JIT_UNSPILL_COMPATIBILITY_MODE=True`` to enable compatibility mode, which automatically calls ``unproxy()`` on all function inputs. + + +cuDF Spilling +------------- + +When executing a `Dask cuDF `_ +(i.e. Dask DataFrame) ETL workflow, it is usually best to leverage `native spilling support in +cuDF `. + +Native cuDF spilling has an important advantage over the other methodologies mentioned +above. When JIT-unspill or default spilling are used, the worker is only able to spill +the input or output of a task. This means that any data that is created within the task +is completely off limits until the task is done executing. When cuDF spilling is used, +however, individual device buffers can be spilled/unspilled as needed while the task +is executing. + +When deploying a ``LocalCUDACluster``, cuDF spilling can be enabled with the ``enable_cudf_spill`` argument: + +.. code-block:: + + >>> from distributed import Client​ + >>> from dask_cuda import LocalCUDACluster​ + + >>> cluster = LocalCUDACluster(n_workers=10, enable_cudf_spill=True)​ + >>> client = Client(cluster)​ + +The same applies for ``dask cuda worker``: + +.. code-block:: + + $ dask scheduler + distributed.scheduler - INFO - Scheduler at: tcp://127.0.0.1:8786 + + $ dask cuda worker --enable-cudf-spill + + +Statistics +~~~~~~~~~~ + +When cuDF spilling is enabled, it is also possible to have cuDF collect basic +spill statistics. This information can be a useful way to understand the +performance of Dask cuDF workflows with high memory utilization: + +.. code-block:: + + $ dask cuda worker --enable-cudf-spill --cudf-spill-stats 1 + +To have each dask-cuda worker print spill statistics, do something like: + +.. code-block:: + + def spill_info(): + from cudf.core.buffer.spill_manager import get_global_manager + print(get_global_manager().statistics) + client.submit(spill_info) + + +Limitations +~~~~~~~~~~~ + +Although cuDF spilling is the best option for most Dask cuDF ETL workflows, +it will be much less effective if that workflow converts between ``cudf.DataFrame`` +and other data formats (e.g. ``cupy.ndarray``). Once the underlying device buffers +are "exposed" to external memory references, they become "unspillable" by cuDF. +In cases like this (e.g. Dask CUDA + XGBoost), JIT-Unspill is usually a better choice.