From 124d3e353eeebd595da113dbef3d5bad842a791d Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Mon, 16 Sep 2024 12:17:58 -0500 Subject: [PATCH 01/15] Migrate dask-cudf README improvements to dask-cudf sphinx docs (#16765) Follow up to https://github.com/rapidsai/cudf/pull/16671 - Moves most of the information recently added to the dask-cudf README into the dask-cudf Sphinx documentation - Adds a "Quick-start" example to the simplified dask-cudf README Authors: - Richard (Rick) Zamora (https://github.com/rjzamora) Approvers: - Bradley Dice (https://github.com/bdice) - Benjamin Zaitlen (https://github.com/quasiben) - Peter Andreas Entschev (https://github.com/pentschev) URL: https://github.com/rapidsai/cudf/pull/16765 --- docs/cudf/source/user_guide/10min.ipynb | 31 ++-- docs/dask_cudf/source/index.rst | 210 ++++++++++++++++++------ python/dask_cudf/README.md | 148 +++++------------ 3 files changed, 213 insertions(+), 176 deletions(-) diff --git a/docs/cudf/source/user_guide/10min.ipynb b/docs/cudf/source/user_guide/10min.ipynb index 2eaa75b3189..95f5f9734dd 100644 --- a/docs/cudf/source/user_guide/10min.ipynb +++ b/docs/cudf/source/user_guide/10min.ipynb @@ -5,9 +5,9 @@ "id": "4c6c548b", "metadata": {}, "source": [ - "# 10 Minutes to cuDF and Dask-cuDF\n", + "# 10 Minutes to cuDF and Dask cuDF\n", "\n", - "Modelled after 10 Minutes to Pandas, this is a short introduction to cuDF and Dask-cuDF, geared mainly towards new users.\n", + "Modelled after 10 Minutes to Pandas, this is a short introduction to cuDF and Dask cuDF, geared mainly towards new users.\n", "\n", "## What are these Libraries?\n", "\n", @@ -18,13 +18,14 @@ "[Dask cuDF](https://github.com/rapidsai/cudf/tree/main/python/dask_cudf) extends Dask where necessary to allow its DataFrame partitions to be processed using cuDF GPU DataFrames instead of Pandas DataFrames. For instance, when you call `dask_cudf.read_csv(...)`, your cluster's GPUs do the work of parsing the CSV file(s) by calling [`cudf.read_csv()`](https://docs.rapids.ai/api/cudf/stable/api_docs/api/cudf.read_csv.html).\n", "\n", "\n", - "> [!NOTE] \n", - "> This notebook uses the explicit Dask cuDF API (`dask_cudf`) for clarity. However, we strongly recommend that you use Dask's [configuration infrastructure](https://docs.dask.org/en/latest/configuration.html) to set the `\"dataframe.backend\"` to `\"cudf\"`, and work with the `dask.dataframe` API directly. Please see the [Dask cuDF documentation](https://github.com/rapidsai/cudf/tree/main/python/dask_cudf) for more information.\n", + "
\n", + "Note: This notebook uses the explicit Dask cuDF API (dask_cudf) for clarity. However, we strongly recommend that you use Dask's configuration infrastructure to set the \"dataframe.backend\" option to \"cudf\", and work with the Dask DataFrame API directly. Please see the Dask cuDF documentation for more information.\n", + "
\n", "\n", "\n", - "## When to use cuDF and Dask-cuDF\n", + "## When to use cuDF and Dask cuDF\n", "\n", - "If your workflow is fast enough on a single GPU or your data comfortably fits in memory on a single GPU, you would want to use cuDF. If you want to distribute your workflow across multiple GPUs, have more data than you can fit in memory on a single GPU, or want to analyze data spread across many files at once, you would want to use Dask-cuDF." + "If your workflow is fast enough on a single GPU or your data comfortably fits in memory on a single GPU, you would want to use cuDF. If you want to distribute your workflow across multiple GPUs, have more data than you can fit in memory on a single GPU, or want to analyze data spread across many files at once, you would want to use Dask cuDF." ] }, { @@ -115,7 +116,7 @@ "source": [ "ds = dask_cudf.from_cudf(s, npartitions=2)\n", "# Note the call to head here to show the first few entries, unlike\n", - "# cuDF objects, dask-cuDF objects do not have a printing\n", + "# cuDF objects, Dask-cuDF objects do not have a printing\n", "# representation that shows values since they may not be in local\n", "# memory.\n", "ds.head(n=3)" @@ -331,11 +332,11 @@ "id": "b17db919", "metadata": {}, "source": [ - "Now we will convert our cuDF dataframe into a dask-cuDF equivalent. Here we call out a key difference: to inspect the data we must call a method (here `.head()` to look at the first few values). In the general case (see the end of this notebook), the data in `ddf` will be distributed across multiple GPUs.\n", + "Now we will convert our cuDF dataframe into a Dask-cuDF equivalent. Here we call out a key difference: to inspect the data we must call a method (here `.head()` to look at the first few values). In the general case (see the end of this notebook), the data in `ddf` will be distributed across multiple GPUs.\n", "\n", - "In this small case, we could call `ddf.compute()` to obtain a cuDF object from the dask-cuDF object. In general, we should avoid calling `.compute()` on large dataframes, and restrict ourselves to using it when we have some (relatively) small postprocessed result that we wish to inspect. Hence, throughout this notebook we will generally call `.head()` to inspect the first few values of a dask-cuDF dataframe, occasionally calling out places where we use `.compute()` and why.\n", + "In this small case, we could call `ddf.compute()` to obtain a cuDF object from the Dask-cuDF object. In general, we should avoid calling `.compute()` on large dataframes, and restrict ourselves to using it when we have some (relatively) small postprocessed result that we wish to inspect. Hence, throughout this notebook we will generally call `.head()` to inspect the first few values of a Dask-cuDF dataframe, occasionally calling out places where we use `.compute()` and why.\n", "\n", - "*To understand more of the differences between how cuDF and dask-cuDF behave here, visit the [10 Minutes to Dask](https://docs.dask.org/en/stable/10-minutes-to-dask.html) tutorial after this one.*" + "*To understand more of the differences between how cuDF and Dask cuDF behave here, visit the [10 Minutes to Dask](https://docs.dask.org/en/stable/10-minutes-to-dask.html) tutorial after this one.*" ] }, { @@ -1680,7 +1681,7 @@ "id": "7aa0089f", "metadata": {}, "source": [ - "Note here we call `compute()` rather than `head()` on the dask-cuDF dataframe since we are happy that the number of matching rows will be small (and hence it is reasonable to bring the entire result back)." + "Note here we call `compute()` rather than `head()` on the Dask-cuDF dataframe since we are happy that the number of matching rows will be small (and hence it is reasonable to bring the entire result back)." ] }, { @@ -2393,7 +2394,7 @@ "id": "f6094cbe", "metadata": {}, "source": [ - "Applying functions to a `Series`. Note that applying user defined functions directly with Dask-cuDF is not yet implemented. For now, you can use [map_partitions](http://docs.dask.org/en/stable/generated/dask.dataframe.DataFrame.map_partitions.html) to apply a function to each partition of the distributed dataframe." + "Applying functions to a `Series`. Note that applying user defined functions directly with Dask cuDF is not yet implemented. For now, you can use [map_partitions](http://docs.dask.org/en/stable/generated/dask.dataframe.DataFrame.map_partitions.html) to apply a function to each partition of the distributed dataframe." ] }, { @@ -3492,7 +3493,7 @@ "id": "5ac3b004", "metadata": {}, "source": [ - "Transposing a dataframe, using either the `transpose` method or `T` property. Currently, all columns must have the same type. Transposing is not currently implemented in Dask-cuDF." + "Transposing a dataframe, using either the `transpose` method or `T` property. Currently, all columns must have the same type. Transposing is not currently implemented in Dask cuDF." ] }, { @@ -4181,7 +4182,7 @@ "id": "aa8a445b", "metadata": {}, "source": [ - "To convert the first few entries to pandas, we similarly call `.head()` on the dask-cuDF dataframe to obtain a local cuDF dataframe, which we can then convert." + "To convert the first few entries to pandas, we similarly call `.head()` on the Dask-cuDF dataframe to obtain a local cuDF dataframe, which we can then convert." ] }, { @@ -4899,7 +4900,7 @@ "id": "787eae14", "metadata": {}, "source": [ - "Note that for the dask-cuDF case, we use `dask_cudf.read_csv` in preference to `dask_cudf.from_cudf(cudf.read_csv)` since the former can parallelize across multiple GPUs and handle larger CSV files that would fit in memory on a single GPU." + "Note that for the Dask-cuDF case, we use `dask_cudf.read_csv` in preference to `dask_cudf.from_cudf(cudf.read_csv)` since the former can parallelize across multiple GPUs and handle larger CSV files that would fit in memory on a single GPU." ] }, { diff --git a/docs/dask_cudf/source/index.rst b/docs/dask_cudf/source/index.rst index 9a216690384..7fe6cbd45fa 100644 --- a/docs/dask_cudf/source/index.rst +++ b/docs/dask_cudf/source/index.rst @@ -3,39 +3,42 @@ You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. -Welcome to dask-cudf's documentation! +Welcome to Dask cuDF's documentation! ===================================== -**Dask-cuDF** (pronounced "DASK KOO-dee-eff") is an extension +**Dask cuDF** (pronounced "DASK KOO-dee-eff") is an extension library for the `Dask `__ parallel computing -framework that provides a `cuDF -`__-backed distributed -dataframe with the same API as `Dask dataframes -`__. +framework. When installed, Dask cuDF is automatically registered +as the ``"cudf"`` dataframe backend for +`Dask DataFrame `__. + +.. note:: + Neither Dask cuDF nor Dask DataFrame provide support for multi-GPU + or multi-node execution on their own. You must also deploy a + `dask.distributed ` cluster + to leverage multiple GPUs. We strongly recommend using `Dask-CUDA + `__ to simplify the + setup of the cluster, taking advantage of all features of the GPU + and networking hardware. If you are familiar with Dask and `pandas `__ or -`cuDF `__, then Dask-cuDF +`cuDF `__, then Dask cuDF should feel familiar to you. If not, we recommend starting with `10 minutes to Dask `__ followed -by `10 minutes to cuDF and Dask-cuDF +by `10 minutes to cuDF and Dask cuDF `__. -When running on multi-GPU systems, `Dask-CUDA -`__ is recommended to -simplify the setup of the cluster, taking advantage of all features of -the GPU and networking hardware. -Using Dask-cuDF +Using Dask cuDF --------------- -When installed, Dask-cuDF registers itself as a dataframe backend for -Dask. This means that in many cases, using cuDF-backed dataframes requires -only small changes to an existing workflow. The minimal change is to -select cuDF as the dataframe backend in :doc:`Dask's -configuration `. To do so, we must set the option -``dataframe.backend`` to ``cudf``. From Python, this can be achieved -like so:: +The Dask DataFrame API (Recommended) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Simply use the `Dask configuration ` system to +set the ``"dataframe.backend"`` option to ``"cudf"``. From Python, +this can be achieved like so:: import dask @@ -44,52 +47,157 @@ like so:: Alternatively, you can set ``DASK_DATAFRAME__BACKEND=cudf`` in the environment before running your code. -Dataframe creation from on-disk formats -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -If your workflow creates Dask dataframes from on-disk formats -(for example using :func:`dask.dataframe.read_parquet`), then setting -the backend may well be enough to migrate your workflow. - -For example, consider reading a dataframe from parquet:: +Once this is done, the public Dask DataFrame API will leverage +``cudf`` automatically when a new DataFrame collection is created +from an on-disk format using any of the following ``dask.dataframe`` +functions:: - import dask.dataframe as dd +* :func:`dask.dataframe.read_parquet` +* :func:`dask.dataframe.read_json` +* :func:`dask.dataframe.read_csv` +* :func:`dask.dataframe.read_orc` +* :func:`dask.dataframe.read_hdf` +* :func:`dask.dataframe.from_dict` - # By default, we obtain a pandas-backed dataframe - df = dd.read_parquet("data.parquet", ...) +For example:: + import dask.dataframe as dd -To obtain a cuDF-backed dataframe, we must set the -``dataframe.backend`` configuration option:: + # By default, we obtain a pandas-backed dataframe + df = dd.read_parquet("data.parquet", ...) import dask - import dask.dataframe as dd dask.config.set({"dataframe.backend": "cudf"}) - # This gives us a cuDF-backed dataframe + # This now gives us a cuDF-backed dataframe df = dd.read_parquet("data.parquet", ...) -This code will use cuDF's GPU-accelerated :func:`parquet reader -` to read partitions of the data. +When other functions are used to create a new collection +(e.g. :func:`from_map`, :func:`from_pandas`, :func:`from_delayed`, +and :func:`from_array`), the backend of the new collection will +depend on the inputs to those functions. For example:: + + import pandas as pd + import cudf + + # This gives us a pandas-backed dataframe + dd.from_pandas(pd.DataFrame({"a": range(10)})) + + # This gives us a cuDF-backed dataframe + dd.from_pandas(cudf.DataFrame({"a": range(10)})) + +An existing collection can always be moved to a specific backend +using the :func:`dask.dataframe.DataFrame.to_backend` API:: + + # This ensures that we have a cuDF-backed dataframe + df = df.to_backend("cudf") + + # This ensures that we have a pandas-backed dataframe + df = df.to_backend("pandas") + +The explicit Dask cuDF API +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +In addition to providing the ``"cudf"`` backend for Dask DataFrame, +Dask cuDF also provides an explicit ``dask_cudf`` API:: + + import dask_cudf + + # This always gives us a cuDF-backed dataframe + df = dask_cudf.read_parquet("data.parquet", ...) + +This API is used implicitly by the Dask DataFrame API when the ``"cudf"`` +backend is enabled. Therefore, using it directly will not provide any +performance benefit over the CPU/GPU-portable ``dask.dataframe`` API. +Also, using some parts of the explicit API are incompatible with +automatic query planning (see the next section). + +The explicit Dask cuDF API +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Dask cuDF now provides automatic query planning by default (RAPIDS 24.06+). +As long as the ``"dataframe.query-planning"`` configuration is set to +``True`` (the default) when ``dask.dataframe`` is first imported, `Dask +Expressions `__ will be used under the hood. + +For example, the following code will automatically benefit from predicate +pushdown when the result is computed:: + + df = dd.read_parquet("/my/parquet/dataset/") + result = df.sort_values('B')['A'] + +Unoptimized expression graph (``df.pprint()``):: + + Projection: columns='A' + SortValues: by=['B'] shuffle_method='tasks' options={} + ReadParquetFSSpec: path='/my/parquet/dataset/' ... + +Simplified expression graph (``df.simplify().pprint()``):: + + Projection: columns='A' + SortValues: by=['B'] shuffle_method='tasks' options={} + ReadParquetFSSpec: path='/my/parquet/dataset/' columns=['A', 'B'] ... + +.. note:: + Dask will automatically simplify the expression graph (within + :func:`optimize`) when the result is converted to a task graph + (via :func:`compute` or :func:`persist`). You do not need to call + :func:`simplify` yourself. + + +Using Multiple GPUs and Multiple Nodes +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Whenever possible, Dask cuDF (i.e. Dask DataFrame) will automatically try +to partition your data into small-enough tasks to fit comfortably in the +memory of a single GPU. This means the necessary compute tasks needed to +compute a query can often be streamed to a single GPU process for +out-of-core computing. This also means that the compute tasks can be +executed in parallel over a multi-GPU cluster. + +In order to execute your Dask workflow on multiple GPUs, you will +typically need to use `Dask-CUDA `__ +to deploy distributed Dask cluster, and +`Distributed `__ +to define a client object. For example:: + + from dask_cuda import LocalCUDACluster + from distributed import Client + + if __name__ == "__main__": + + client = Client( + LocalCUDACluster( + CUDA_VISIBLE_DEVICES="0,1", # Use two workers (on devices 0 and 1) + rmm_pool_size=0.9, # Use 90% of GPU memory as a pool for faster allocations + enable_cudf_spill=True, # Improve device memory stability + local_directory="/fast/scratch/", # Use fast local storage for spilling + ) + ) + + df = dd.read_parquet("/my/parquet/dataset/") + agg = df.groupby('B').sum() + agg.compute() # This will use the cluster defined above + +.. note:: + This example uses :func:`compute` to materialize a concrete + ``cudf.DataFrame`` object in local memory. Never call :func:`compute` + on a large collection that cannot fit comfortably in the memory of a + single GPU! See Dask's `documentation on managing computation + `__ + for more details. -Dataframe creation from in-memory formats -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +Please see the `Dask-CUDA `__ +documentation for more information about deploying GPU-aware clusters +(including `best practices +`__). -If you already have a dataframe in memory and want to convert it to a -cuDF-backend one, there are two options depending on whether the -dataframe is already a Dask one or not. If you have a Dask dataframe, -then you can call :func:`dask.dataframe.to_backend` passing ``"cudf"`` -as the backend; if you have a pandas dataframe then you can either -call :func:`dask.dataframe.from_pandas` followed by -:func:`~dask.dataframe.to_backend` or first convert the dataframe with -:func:`cudf.from_pandas` and then parallelise this with -:func:`dask_cudf.from_cudf`. API Reference ------------- -Generally speaking, Dask-cuDF tries to offer exactly the same API as -Dask itself. There are, however, some minor differences mostly because +Generally speaking, Dask cuDF tries to offer exactly the same API as +Dask DataFrame. There are, however, some minor differences mostly because cuDF does not :doc:`perfectly mirror ` the pandas API, or because cuDF provides additional configuration flags (these mostly occur in data reading and writing interfaces). @@ -97,7 +205,7 @@ flags (these mostly occur in data reading and writing interfaces). As a result, straightforward workflows can be migrated without too much trouble, but more complex ones that utilise more features may need a bit of tweaking. The API documentation describes details of the -differences and all functionality that Dask-cuDF supports. +differences and all functionality that Dask cuDF supports. .. toctree:: :maxdepth: 2 diff --git a/python/dask_cudf/README.md b/python/dask_cudf/README.md index 6edb9f87d48..4655d2165f0 100644 --- a/python/dask_cudf/README.md +++ b/python/dask_cudf/README.md @@ -1,135 +1,63 @@ #
 Dask cuDF - A GPU Backend for Dask DataFrame
-Dask cuDF (a.k.a. dask-cudf or `dask_cudf`) is an extension library for [Dask DataFrame](https://docs.dask.org/en/stable/dataframe.html). When installed, Dask cuDF is automatically registered as the `"cudf"` [dataframe backend](https://docs.dask.org/en/stable/how-to/selecting-the-collection-backend.html) for Dask DataFrame. - -## Using Dask cuDF - -### The Dask DataFrame API (Recommended) - -Simply set the `"dataframe.backend"` [configuration](https://docs.dask.org/en/stable/configuration.html) to `"cudf"` in Dask, and the public Dask DataFrame API will leverage `cudf` automatically: - -```python -import dask -dask.config.set({"dataframe.backend": "cudf"}) - -import dask.dataframe as dd -# This gives us a cuDF-backed dataframe -df = dd.read_parquet("data.parquet", ...) -``` +Dask cuDF (a.k.a. dask-cudf or `dask_cudf`) is an extension library for [Dask DataFrame](https://docs.dask.org/en/stable/dataframe.html) that provides a Pandas-like API for parallel and larger-than-memory DataFrame computing on GPUs. When installed, Dask cuDF is automatically registered as the `"cudf"` [dataframe backend](https://docs.dask.org/en/stable/how-to/selecting-the-collection-backend.html) for Dask DataFrame. > [!IMPORTANT] -> The `"dataframe.backend"` configuration will only be used for collection creation when the following APIs are used: `read_parquet`, `read_json`, `read_csv`, `read_orc`, `read_hdf`, and `from_dict`. For example, if `from_map`, `from_pandas`, `from_delayed`, or `from_array` are used, the backend of the new collection will depend on the input to the function: - -```python -import pandas as pd -import cudf - -# This gives us a Pandas-backed dataframe -dd.from_pandas(pd.DataFrame({"a": range(10)})) - -# This gives us a cuDF-backed dataframe -dd.from_pandas(cudf.DataFrame({"a": range(10)})) -``` - -A cuDF-backed DataFrame collection can be moved to the `"pandas"` backend: - -```python -df = df.to_backend("pandas") -``` - -Similarly, a Pandas-backed DataFrame collection can be moved to the `"cudf"` backend: - -```python -df = df.to_backend("cudf") -``` - -### The Explicit Dask cuDF API - -In addition to providing the `"cudf"` backend for Dask DataFrame, Dask cuDF also provides an explicit `dask_cudf` API: - -```python -import dask_cudf - -# This always gives us a cuDF-backed dataframe -df = dask_cudf.read_parquet("data.parquet", ...) -``` - -> [!NOTE] -> This API is used implicitly by the Dask DataFrame API when the `"cudf"` backend is enabled. Therefore, using it directly will not provide any performance benefit over the CPU/GPU-portable `dask.dataframe` API. Also, using some parts of the explicit API are incompatible with automatic query planning (see the next section). +> Dask cuDF does not provide support for multi-GPU or multi-node execution on its own. You must also deploy a distributed cluster (ideally with [Dask-CUDA](https://docs.rapids.ai/api/dask-cuda/stable/)) to leverage multiple GPUs efficiently. -See the [Dask cuDF's API documentation](https://docs.rapids.ai/api/dask-cudf/stable/) for further information. - -## Query Planning - -Dask cuDF now provides automatic query planning by default (RAPIDS 24.06+). As long as the `"dataframe.query-planning"` configuration is set to `True` (the default) when `dask.dataframe` is first imported, [Dask Expressions](https://github.com/dask/dask-expr) will be used under the hood. - -For example, the following user code will automatically benefit from predicate pushdown when the result is computed. - -```python -df = dd.read_parquet("/my/parquet/dataset/") -result = df.sort_values('B')['A'] -``` - -Unoptimized expression graph (`df.pprint()`): -``` -Projection: columns='A' - SortValues: by=['B'] shuffle_method='tasks' options={} - ReadParquetFSSpec: path='/my/parquet/dataset/' ... -``` +## Using Dask cuDF -Simplified expression graph (`df.simplify().pprint()`): -``` -Projection: columns='A' - SortValues: by=['B'] shuffle_method='tasks' options={} - ReadParquetFSSpec: path='/my/parquet/dataset/' columns=['A', 'B'] ... -``` +Please visit [the official documentation page](https://docs.rapids.ai/api/dask-cudf/stable/) for detailed information about using Dask cuDF. -> [!NOTE] -> Dask will automatically simplify the expression graph (within `optimize`) when the result is converted to a task graph (via `compute` or `persist`). The user does not need to call `simplify` themself. +## Installation +See the [RAPIDS install page](https://docs.rapids.ai/install) for the most up-to-date information and commands for installing Dask cuDF and other RAPIDS packages. -## Using Multiple GPUs and Multiple Nodes +## Resources -Whenever possible, Dask cuDF (i.e. Dask DataFrame) will automatically try to partition your data into small-enough tasks to fit comfortably in the memory of a single GPU. This means the necessary compute tasks needed to compute a query can often be streamed to a single GPU process for out-of-core computing. This also means that the compute tasks can be executed in parallel over a multi-GPU cluster. +- [Dask cuDF documentation](https://docs.rapids.ai/api/dask-cudf/stable/) +- [cuDF documentation](https://docs.rapids.ai/api/cudf/stable/) +- [10 Minutes to cuDF and Dask cuDF](https://docs.rapids.ai/api/cudf/stable/user_guide/10min/) +- [Dask-CUDA documentation](https://docs.rapids.ai/api/dask-cuda/stable/) +- [Deployment](https://docs.rapids.ai/deployment/stable/) +- [RAPIDS Community](https://rapids.ai/learn-more/#get-involved): Get help, contribute, and collaborate. -> [!IMPORTANT] -> Neither Dask cuDF nor Dask DataFrame provide support for multi-GPU or multi-node execution on their own. You must deploy a distributed cluster (ideally with [Dask CUDA](https://docs.rapids.ai/api/dask-cuda/stable/)) to leverage multiple GPUs. +### Quick-start example -In order to execute your Dask workflow on multiple GPUs, you will typically need to use [Dask CUDA](https://docs.rapids.ai/api/dask-cuda/stable/) to deploy distributed Dask cluster, and [Distributed](https://distributed.dask.org/en/stable/client.html) to define a `client` object. For example: +A very common Dask cuDF use case is single-node multi-GPU data processing. These workflows typically use the following pattern: ```python - +import dask +import dask.dataframe as dd from dask_cuda import LocalCUDACluster from distributed import Client -client = Client( +if __name__ == "__main__": + + # Define a GPU-aware cluster to leverage multiple GPUs + client = Client( LocalCUDACluster( - CUDA_VISIBLE_DEVICES="0,1", # Use two workers (on devices 0 and 1) - rmm_pool_size=0.9, # Use 90% of GPU memory as a pool for faster allocations - enable_cudf_spill=True, # Improve device memory stability - local_directory="/fast/scratch/", # Use fast local storage for spilling + CUDA_VISIBLE_DEVICES="0,1", # Use two workers (on devices 0 and 1) + rmm_pool_size=0.9, # Use 90% of GPU memory as a pool for faster allocations + enable_cudf_spill=True, # Improve device memory stability + local_directory="/fast/scratch/", # Use fast local storage for spilling ) -) + ) -df = dd.read_parquet("/my/parquet/dataset/") -agg = df.groupby('B').sum() -agg.compute() # This will use the cluster defined above -``` + # Set the default dataframe backend to "cudf" + dask.config.set({"dataframe.backend": "cudf"}) -> [!NOTE] -> This example uses `compute` to materialize a concrete `cudf.DataFrame` object in local memory. Never call `compute` on a large collection that cannot fit comfortably in the memory of a single GPU! See Dask's [documentation on managing computation](https://distributed.dask.org/en/stable/manage-computation.html) for more details. + # Create your DataFrame collection from on-disk + # or in-memory data + df = dd.read_parquet("/my/parquet/dataset/") -Please see the [Dask CUDA](https://docs.rapids.ai/api/dask-cuda/stable/) documentation for more information about deploying GPU-aware clusters (including [best practices](https://docs.rapids.ai/api/dask-cuda/stable/examples/best-practices/)). + # Use cudf-like syntax to transform and/or query your data + query = df.groupby('item')['price'].mean() -## Install - -See the [RAPIDS install page](https://docs.rapids.ai/install) for the most up-to-date information and commands for installing Dask cuDF and other RAPIDS packages. + # Compute, persist, or write out the result + query.head() +``` -## Resources +If you do not have multiple GPUs available, using `LocalCUDACluster` is optional. However, it is still a good idea to [enable cuDF spilling](https://docs.rapids.ai/api/cudf/stable/developer_guide/library_design/#spilling-to-host-memory). -- [Dask cuDF API documentation](https://docs.rapids.ai/api/dask-cudf/stable/) -- [cuDF API documentation](https://docs.rapids.ai/api/cudf/stable/) -- [10 Minutes to cuDF and Dask cuDF](https://docs.rapids.ai/api/cudf/stable/user_guide/10min/) -- [Dask CUDA documentation](https://docs.rapids.ai/api/dask-cuda/stable/) -- [Deployment](https://docs.rapids.ai/deployment/stable/) -- [RAPIDS Community](https://rapids.ai/learn-more/#get-involved): Get help, contribute, and collaborate. +If you wish to scale across multiple nodes, you will need to use a different mechanism to deploy your Dask-CUDA workers. Please see [the RAPIDS deployment documentation](https://docs.rapids.ai/deployment/stable/) for more instructions. From 40333854b5efadb5b482ec80663b837680af1598 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 16 Sep 2024 17:04:47 -0500 Subject: [PATCH 02/15] Java: Make ColumnVector.fromViewWithContiguousAllocation public (#16784) Exposes ColumnVector's fromViewWithContiguousAllocation method so code outside of cudf that builds contiguous table views can expose those columns in Java. Authors: - Jason Lowe (https://github.com/jlowe) Approvers: - Alessandro Bellina (https://github.com/abellina) URL: https://github.com/rapidsai/cudf/pull/16784 --- java/src/main/java/ai/rapids/cudf/ColumnVector.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/java/src/main/java/ai/rapids/cudf/ColumnVector.java b/java/src/main/java/ai/rapids/cudf/ColumnVector.java index 5a0fbd224ad..6a0f0f6f169 100644 --- a/java/src/main/java/ai/rapids/cudf/ColumnVector.java +++ b/java/src/main/java/ai/rapids/cudf/ColumnVector.java @@ -218,7 +218,13 @@ static long initViewHandle(DType type, int numRows, int nullCount, od, vd, nullCount, numRows, childHandles); } - static ColumnVector fromViewWithContiguousAllocation(long columnViewAddress, DeviceMemoryBuffer buffer) { + /** + * Creates a ColumnVector from a native column_view using a contiguous device allocation. + * + * @param columnViewAddress address of the native column_view + * @param buffer device buffer containing the data referenced by the column view + */ + public static ColumnVector fromViewWithContiguousAllocation(long columnViewAddress, DeviceMemoryBuffer buffer) { return new ColumnVector(columnViewAddress, buffer); } From 86861e08d9f7b1ae0a61d6b05bbfc6690107ca0f Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Mon, 16 Sep 2024 19:14:18 -0500 Subject: [PATCH 03/15] Fix `cov`/`corr` bug in dask-cudf (#16786) Closes https://github.com/rapidsai/cudf/issues/14935 Overrides `_prepare_cov_corr` method to avoid cudf compatibility issues in dask-cudf. Authors: - Richard (Rick) Zamora (https://github.com/rjzamora) Approvers: - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/16786 --- python/dask_cudf/dask_cudf/expr/_collection.py | 18 +++++++++++++++++- python/dask_cudf/dask_cudf/tests/test_core.py | 17 +++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index f60e4ff81ef..97e1dffc65b 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -49,8 +49,24 @@ def to_dask_dataframe(self, **kwargs): return self.to_backend("pandas", **kwargs) + def _prepare_cov_corr(self, min_periods, numeric_only): + # Upstream version of this method sets min_periods + # to 2 by default (which is not supported by cudf) + # TODO: Remove when cudf supports both min_periods + # and numeric_only + # See: https://github.com/rapidsai/cudf/issues/12626 + # See: https://github.com/rapidsai/cudf/issues/9009 + self._meta.cov(min_periods=min_periods) + + frame = self + if numeric_only: + numerics = self._meta._get_numeric_data() + if len(numerics.columns) != len(self.columns): + frame = frame[list(numerics.columns)] + return frame, min_periods + # var can be removed if cudf#15179 is addressed. - # See: https://github.com/rapidsai/cudf/issues/15179 + # See: https://github.com/rapidsai/cudf/issues/14935 def var( self, axis=0, diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index 905d8c08135..7aa0f6320f2 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -1007,3 +1007,20 @@ def test_to_backend_simplify(): df2 = df.to_backend("cudf")[["y"]].simplify() df3 = df[["y"]].to_backend("cudf").to_backend("cudf").simplify() assert df2._name == df3._name + + +@pytest.mark.parametrize("numeric_only", [True, False]) +@pytest.mark.parametrize("op", ["corr", "cov"]) +def test_cov_corr(op, numeric_only): + df = cudf.DataFrame.from_dict( + { + "x": np.random.randint(0, 5, size=10), + "y": np.random.normal(size=10), + } + ) + ddf = dd.from_pandas(df, npartitions=2) + res = getattr(ddf, op)(numeric_only=numeric_only) + # Use to_pandas until cudf supports numeric_only + # (See: https://github.com/rapidsai/cudf/issues/12626) + expect = getattr(df.to_pandas(), op)(numeric_only=numeric_only) + dd.assert_eq(res, expect) From f8d50639fffb541dee3b860c19756af2c4a5a850 Mon Sep 17 00:00:00 2001 From: Paul Mattione <156858817+pmattione-nvidia@users.noreply.github.com> Date: Mon, 16 Sep 2024 21:27:38 -0400 Subject: [PATCH 04/15] Add ability to set parquet row group max #rows and #bytes in java (#16805) Adds the ability to set the max # rows per row group and max # bytes per row group for parquet files. Authors: - Paul Mattione (https://github.com/pmattione-nvidia) Approvers: - Robert (Bobby) Evans (https://github.com/revans2) - Muhammad Haseeb (https://github.com/mhaseeb123) - Nghia Truong (https://github.com/ttnghia) URL: https://github.com/rapidsai/cudf/pull/16805 --- .../ai/rapids/cudf/ParquetWriterOptions.java | 26 ++++++- java/src/main/java/ai/rapids/cudf/Table.java | 68 +++++++++++-------- java/src/main/native/src/TableJni.cpp | 8 +++ .../test/java/ai/rapids/cudf/TableTest.java | 8 ++- 4 files changed, 80 insertions(+), 30 deletions(-) diff --git a/java/src/main/java/ai/rapids/cudf/ParquetWriterOptions.java b/java/src/main/java/ai/rapids/cudf/ParquetWriterOptions.java index 7b58817550d..8c8180436e6 100644 --- a/java/src/main/java/ai/rapids/cudf/ParquetWriterOptions.java +++ b/java/src/main/java/ai/rapids/cudf/ParquetWriterOptions.java @@ -1,6 +1,6 @@ /* * - * Copyright (c) 2019-2021, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,9 +24,13 @@ */ public final class ParquetWriterOptions extends CompressionMetadataWriterOptions { private final StatisticsFrequency statsGranularity; + private int rowGroupSizeRows; + private long rowGroupSizeBytes; private ParquetWriterOptions(Builder builder) { super(builder); + this.rowGroupSizeRows = builder.rowGroupSizeRows; + this.rowGroupSizeBytes = builder.rowGroupSizeBytes; this.statsGranularity = builder.statsGranularity; } @@ -51,18 +55,38 @@ public static Builder builder() { return new Builder(); } + public int getRowGroupSizeRows() { + return rowGroupSizeRows; + } + + public long getRowGroupSizeBytes() { + return rowGroupSizeBytes; + } + public StatisticsFrequency getStatisticsFrequency() { return statsGranularity; } public static class Builder extends CompressionMetadataWriterOptions.Builder { + private int rowGroupSizeRows = 1000000; //Max of 1 million rows per row group + private long rowGroupSizeBytes = 128 * 1024 * 1024; //Max of 128MB per row group private StatisticsFrequency statsGranularity = StatisticsFrequency.ROWGROUP; public Builder() { super(); } + public Builder withRowGroupSizeRows(int rowGroupSizeRows) { + this.rowGroupSizeRows = rowGroupSizeRows; + return this; + } + + public Builder withRowGroupSizeBytes(long rowGroupSizeBytes) { + this.rowGroupSizeBytes = rowGroupSizeBytes; + return this; + } + public Builder withStatisticsFrequency(StatisticsFrequency statsGranularity) { this.statsGranularity = statsGranularity; return this; diff --git a/java/src/main/java/ai/rapids/cudf/Table.java b/java/src/main/java/ai/rapids/cudf/Table.java index cbb126d7ee5..09da43374ae 100644 --- a/java/src/main/java/ai/rapids/cudf/Table.java +++ b/java/src/main/java/ai/rapids/cudf/Table.java @@ -332,20 +332,22 @@ private static native long[] readAvroFromDataSource(String[] filterColumnNames, /** * Setup everything to write parquet formatted data to a file. - * @param columnNames names that correspond to the table columns - * @param numChildren Children of the top level - * @param flatNumChildren flattened list of children per column - * @param nullable true if the column can have nulls else false - * @param metadataKeys Metadata key names to place in the Parquet file - * @param metadataValues Metadata values corresponding to metadataKeys - * @param compression native compression codec ID - * @param statsFreq native statistics frequency ID - * @param isInt96 true if timestamp type is int96 - * @param precisions precision list containing all the precisions of the decimal types in - * the columns - * @param isMapValues true if a column is a map - * @param isBinaryValues true if a column is a binary - * @param filename local output path + * @param columnNames names that correspond to the table columns + * @param numChildren Children of the top level + * @param flatNumChildren flattened list of children per column + * @param nullable true if the column can have nulls else false + * @param metadataKeys Metadata key names to place in the Parquet file + * @param metadataValues Metadata values corresponding to metadataKeys + * @param compression native compression codec ID + * @param rowGroupSizeRows max #rows in a row group + * @param rowGroupSizeBytes max #bytes in a row group + * @param statsFreq native statistics frequency ID + * @param isInt96 true if timestamp type is int96 + * @param precisions precision list containing all the precisions of the decimal types in + * the columns + * @param isMapValues true if a column is a map + * @param isBinaryValues true if a column is a binary + * @param filename local output path * @return a handle that is used in later calls to writeParquetChunk and writeParquetEnd. */ private static native long writeParquetFileBegin(String[] columnNames, @@ -355,6 +357,8 @@ private static native long writeParquetFileBegin(String[] columnNames, String[] metadataKeys, String[] metadataValues, int compression, + int rowGroupSizeRows, + long rowGroupSizeBytes, int statsFreq, boolean[] isInt96, int[] precisions, @@ -366,20 +370,22 @@ private static native long writeParquetFileBegin(String[] columnNames, /** * Setup everything to write parquet formatted data to a buffer. - * @param columnNames names that correspond to the table columns - * @param numChildren Children of the top level - * @param flatNumChildren flattened list of children per column - * @param nullable true if the column can have nulls else false - * @param metadataKeys Metadata key names to place in the Parquet file - * @param metadataValues Metadata values corresponding to metadataKeys - * @param compression native compression codec ID - * @param statsFreq native statistics frequency ID - * @param isInt96 true if timestamp type is int96 - * @param precisions precision list containing all the precisions of the decimal types in - * the columns - * @param isMapValues true if a column is a map - * @param isBinaryValues true if a column is a binary - * @param consumer consumer of host buffers produced. + * @param columnNames names that correspond to the table columns + * @param numChildren Children of the top level + * @param flatNumChildren flattened list of children per column + * @param nullable true if the column can have nulls else false + * @param metadataKeys Metadata key names to place in the Parquet file + * @param metadataValues Metadata values corresponding to metadataKeys + * @param compression native compression codec ID + * @param rowGroupSizeRows max #rows in a row group + * @param rowGroupSizeBytes max #bytes in a row group + * @param statsFreq native statistics frequency ID + * @param isInt96 true if timestamp type is int96 + * @param precisions precision list containing all the precisions of the decimal types in + * the columns + * @param isMapValues true if a column is a map + * @param isBinaryValues true if a column is a binary + * @param consumer consumer of host buffers produced. * @return a handle that is used in later calls to writeParquetChunk and writeParquetEnd. */ private static native long writeParquetBufferBegin(String[] columnNames, @@ -389,6 +395,8 @@ private static native long writeParquetBufferBegin(String[] columnNames, String[] metadataKeys, String[] metadataValues, int compression, + int rowGroupSizeRows, + long rowGroupSizeBytes, int statsFreq, boolean[] isInt96, int[] precisions, @@ -1820,6 +1828,8 @@ private ParquetTableWriter(ParquetWriterOptions options, File outputFile) { options.getMetadataKeys(), options.getMetadataValues(), options.getCompressionType().nativeId, + options.getRowGroupSizeRows(), + options.getRowGroupSizeBytes(), options.getStatisticsFrequency().nativeId, options.getFlatIsTimeTypeInt96(), options.getFlatPrecision(), @@ -1840,6 +1850,8 @@ private ParquetTableWriter(ParquetWriterOptions options, HostBufferConsumer cons options.getMetadataKeys(), options.getMetadataValues(), options.getCompressionType().nativeId, + options.getRowGroupSizeRows(), + options.getRowGroupSizeBytes(), options.getStatisticsFrequency().nativeId, options.getFlatIsTimeTypeInt96(), options.getFlatPrecision(), diff --git a/java/src/main/native/src/TableJni.cpp b/java/src/main/native/src/TableJni.cpp index 40a111209b0..92e213bcb60 100644 --- a/java/src/main/native/src/TableJni.cpp +++ b/java/src/main/native/src/TableJni.cpp @@ -2150,6 +2150,8 @@ Java_ai_rapids_cudf_Table_writeParquetBufferBegin(JNIEnv* env, jobjectArray j_metadata_keys, jobjectArray j_metadata_values, jint j_compression, + jint j_row_group_size_rows, + jlong j_row_group_size_bytes, jint j_stats_freq, jbooleanArray j_isInt96, jintArray j_precisions, @@ -2205,6 +2207,8 @@ Java_ai_rapids_cudf_Table_writeParquetBufferBegin(JNIEnv* env, chunked_parquet_writer_options::builder(sink) .metadata(std::move(metadata)) .compression(static_cast(j_compression)) + .row_group_size_rows(j_row_group_size_rows) + .row_group_size_bytes(j_row_group_size_bytes) .stats_level(static_cast(j_stats_freq)) .key_value_metadata({kv_metadata}) .compression_statistics(stats) @@ -2227,6 +2231,8 @@ Java_ai_rapids_cudf_Table_writeParquetFileBegin(JNIEnv* env, jobjectArray j_metadata_keys, jobjectArray j_metadata_values, jint j_compression, + jint j_row_group_size_rows, + jlong j_row_group_size_bytes, jint j_stats_freq, jbooleanArray j_isInt96, jintArray j_precisions, @@ -2280,6 +2286,8 @@ Java_ai_rapids_cudf_Table_writeParquetFileBegin(JNIEnv* env, chunked_parquet_writer_options::builder(sink) .metadata(std::move(metadata)) .compression(static_cast(j_compression)) + .row_group_size_rows(j_row_group_size_rows) + .row_group_size_bytes(j_row_group_size_bytes) .stats_level(static_cast(j_stats_freq)) .key_value_metadata({kv_metadata}) .compression_statistics(stats) diff --git a/java/src/test/java/ai/rapids/cudf/TableTest.java b/java/src/test/java/ai/rapids/cudf/TableTest.java index 56fe63598d9..830f2b33b32 100644 --- a/java/src/test/java/ai/rapids/cudf/TableTest.java +++ b/java/src/test/java/ai/rapids/cudf/TableTest.java @@ -9122,7 +9122,11 @@ void testParquetWriteToBufferChunked() { columns.add(Columns.STRUCT.name); WriteUtils.buildWriterOptions(optBuilder, columns); ParquetWriterOptions options = optBuilder.build(); - ParquetWriterOptions optionsNoCompress = optBuilder.withCompressionType(CompressionType.NONE).build(); + ParquetWriterOptions optionsNoCompress = + optBuilder.withCompressionType(CompressionType.NONE) + .withRowGroupSizeRows(10000) + .withRowGroupSizeBytes(10000) + .build(); try (Table table0 = getExpectedFileTable(columns); MyBufferConsumer consumer = new MyBufferConsumer()) { try (TableWriter writer = Table.writeParquetChunked(options, consumer)) { @@ -9208,6 +9212,8 @@ void testParquetWriteToFileUncompressedNoStats() throws IOException { .withDecimalColumn("_c7", 4) .withDecimalColumn("_c8", 6) .withCompressionType(CompressionType.NONE) + .withRowGroupSizeRows(10000) + .withRowGroupSizeBytes(10000) .withStatisticsFrequency(ParquetWriterOptions.StatisticsFrequency.NONE) .build(); try (TableWriter writer = Table.writeParquetChunked(options, tempFile.getAbsoluteFile())) { From 7285efbeee12fa7f327933bcf6a52726bfa07790 Mon Sep 17 00:00:00 2001 From: Matthew Roeschke <10647082+mroeschke@users.noreply.github.com> Date: Mon, 16 Sep 2024 18:41:27 -1000 Subject: [PATCH 05/15] Support drop_first in get_dummies (#16795) closes #16791 Authors: - Matthew Roeschke (https://github.com/mroeschke) Approvers: - Matthew Murray (https://github.com/Matt711) URL: https://github.com/rapidsai/cudf/pull/16795 --- python/cudf/cudf/core/reshape.py | 11 +++++++---- python/cudf/cudf/tests/test_onehot.py | 17 +++++++++++++++++ 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/python/cudf/cudf/core/reshape.py b/python/cudf/cudf/core/reshape.py index 3d205957126..c026579b8b5 100644 --- a/python/cudf/cudf/core/reshape.py +++ b/python/cudf/cudf/core/reshape.py @@ -738,7 +738,8 @@ def get_dummies( sparse : boolean, optional Right now this is NON-FUNCTIONAL argument in rapids. drop_first : boolean, optional - Right now this is NON-FUNCTIONAL argument in rapids. + Whether to get k-1 dummies out of k categorical levels by removing the + first level. columns : sequence of str, optional Names of columns to encode. If not provided, will attempt to encode all columns. Note this is different from pandas default behavior, which @@ -806,9 +807,6 @@ def get_dummies( if sparse: raise NotImplementedError("sparse is not supported yet") - if drop_first: - raise NotImplementedError("drop_first is not supported yet") - if isinstance(data, cudf.DataFrame): encode_fallback_dtypes = ["object", "category"] @@ -862,6 +860,7 @@ def get_dummies( prefix=prefix_map.get(name, prefix), prefix_sep=prefix_sep_map.get(name, prefix_sep), dtype=dtype, + drop_first=drop_first, ) result_data.update(col_enc_data) return cudf.DataFrame._from_data(result_data, index=data.index) @@ -874,6 +873,7 @@ def get_dummies( prefix=prefix, prefix_sep=prefix_sep, dtype=dtype, + drop_first=drop_first, ) return cudf.DataFrame._from_data(data, index=ser.index) @@ -1256,6 +1256,7 @@ def _one_hot_encode_column( prefix: str | None, prefix_sep: str | None, dtype: Dtype | None, + drop_first: bool, ) -> dict[str, ColumnBase]: """Encode a single column with one hot encoding. The return dictionary contains pairs of (category, encodings). The keys may be prefixed with @@ -1276,6 +1277,8 @@ def _one_hot_encode_column( ) data = one_hot_encode(column, categories) + if drop_first and len(data): + data.pop(next(iter(data))) if prefix is not None and prefix_sep is not None: data = {f"{prefix}{prefix_sep}{col}": enc for col, enc in data.items()} if dtype: diff --git a/python/cudf/cudf/tests/test_onehot.py b/python/cudf/cudf/tests/test_onehot.py index cc17dc46e0a..e054143b438 100644 --- a/python/cudf/cudf/tests/test_onehot.py +++ b/python/cudf/cudf/tests/test_onehot.py @@ -161,3 +161,20 @@ def test_get_dummies_cats_deprecated(): df = cudf.DataFrame(range(3)) with pytest.warns(FutureWarning): cudf.get_dummies(df, cats={0: [0, 1, 2]}) + + +def test_get_dummies_drop_first_series(): + result = cudf.get_dummies(cudf.Series(list("abcaa")), drop_first=True) + expected = pd.get_dummies(pd.Series(list("abcaa")), drop_first=True) + assert_eq(result, expected) + + +def test_get_dummies_drop_first_dataframe(): + result = cudf.get_dummies( + cudf.DataFrame({"A": list("abcaa"), "B": list("bcaab")}), + drop_first=True, + ) + expected = pd.get_dummies( + pd.DataFrame({"A": list("abcaa"), "B": list("bcaab")}), drop_first=True + ) + assert_eq(result, expected) From 27c29ebd81864d1662dd8a3e8e807955bd8fd9c5 Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Tue, 17 Sep 2024 09:17:43 -0500 Subject: [PATCH 06/15] Use cupy 12.2.0 as oldest dependency pinning on CUDA 12 ARM (#16808) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Uses cupy 12.2.0 as oldest dependency pinning on ARM to ensure CUDA 12 support. This will fix nightly CI failures that look like: ``` LibMambaUnsatisfiableError: Encountered problems while solving: - package cupy-12.0.0-py311h308989c_2 requires python_abi 3.11.* *_cp311, but none of the providers can be installed Could not solve for environment specs The following packages are incompatible ├─ cuda-version 12.2** is installable and it requires │ └─ cudatoolkit 12.2|12.2.* , which can be installed; ├─ cupy 12.0.0 is installable with the potential options │ ├─ cupy 12.0.0 would require │ │ └─ cudatoolkit >=11.2,<12 , which conflicts with any installable versions previously reported; ... ``` Authors: - Bradley Dice (https://github.com/bdice) Approvers: - Kyle Edwards (https://github.com/KyleFromNVIDIA) URL: https://github.com/rapidsai/cudf/pull/16808 --- dependencies.yaml | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/dependencies.yaml b/dependencies.yaml index 483335c02ff..7a13043cc5f 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -710,7 +710,16 @@ dependencies: - numpy==1.23.* - pandas==2.0.* - pyarrow==14.0.0 - - cupy==12.0.0 # ignored as pip constraint + - matrix: + packages: + - output_types: conda + matrices: + - matrix: {dependencies: "oldest", arch: "aarch64", cuda: "12.*"} + packages: + - cupy==12.2.0 # cupy 12.2.0 is the earliest with CUDA 12 ARM packages. + - matrix: {dependencies: "oldest"} + packages: + - cupy==12.0.0 - matrix: packages: - output_types: requirements From 23351aa15f5334b7582c53d4cb6b7421c5c2fd74 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Tue, 17 Sep 2024 13:14:32 -0400 Subject: [PATCH 07/15] Word-based nvtext::minhash function (#15368) Experimental implementation for #15055 The input is a lists column of strings where each string in each row is expected as a word to be hashed. The minimum hash for that row is returned in a lists column where each row contains a minhash per input hash seed. Here the caller is expected to produce the words to be hashed. ``` std::unique_ptr word_minhash( cudf::lists_column_view const& input, cudf::device_span seeds, rmm::cuda_stream_view stream, rmm::device_async_resource_ref mr); ``` Authors: - David Wendt (https://github.com/davidwendt) Approvers: - Bradley Dice (https://github.com/bdice) - Nghia Truong (https://github.com/ttnghia) - GALI PREM SAGAR (https://github.com/galipremsagar) URL: https://github.com/rapidsai/cudf/pull/15368 --- cpp/benchmarks/CMakeLists.txt | 2 +- cpp/benchmarks/text/word_minhash.cpp | 77 +++++++++ cpp/include/nvtext/minhash.hpp | 61 +++++++- cpp/src/text/minhash.cu | 147 +++++++++++++++++- cpp/tests/text/minhash_tests.cpp | 35 +++++ python/cudf/cudf/_lib/nvtext/minhash.pyx | 38 +++++ python/cudf/cudf/_lib/strings/__init__.py | 9 +- python/cudf/cudf/core/column/string.py | 70 +++++++++ .../cudf/cudf/tests/text/test_text_methods.py | 60 +++++++ .../pylibcudf/libcudf/nvtext/minhash.pxd | 10 ++ 10 files changed, 498 insertions(+), 11 deletions(-) create mode 100644 cpp/benchmarks/text/word_minhash.cpp diff --git a/cpp/benchmarks/CMakeLists.txt b/cpp/benchmarks/CMakeLists.txt index 3bf9d02b384..6c5f4a68a4c 100644 --- a/cpp/benchmarks/CMakeLists.txt +++ b/cpp/benchmarks/CMakeLists.txt @@ -337,7 +337,7 @@ ConfigureBench(TEXT_BENCH text/ngrams.cpp text/subword.cpp) ConfigureNVBench( TEXT_NVBENCH text/edit_distance.cpp text/hash_ngrams.cpp text/jaccard.cpp text/minhash.cpp - text/normalize.cpp text/replace.cpp text/tokenize.cpp text/vocab.cpp + text/normalize.cpp text/replace.cpp text/tokenize.cpp text/vocab.cpp text/word_minhash.cpp ) # ################################################################################################## diff --git a/cpp/benchmarks/text/word_minhash.cpp b/cpp/benchmarks/text/word_minhash.cpp new file mode 100644 index 00000000000..adc3dddc59c --- /dev/null +++ b/cpp/benchmarks/text/word_minhash.cpp @@ -0,0 +1,77 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include + +#include +#include +#include +#include + +#include + +#include + +#include + +static void bench_word_minhash(nvbench::state& state) +{ + auto const num_rows = static_cast(state.get_int64("num_rows")); + auto const row_width = static_cast(state.get_int64("row_width")); + auto const seed_count = static_cast(state.get_int64("seed_count")); + auto const base64 = state.get_int64("hash_type") == 64; + + data_profile const strings_profile = + data_profile_builder().distribution(cudf::type_id::STRING, distribution_id::NORMAL, 0, 5); + auto strings_table = + create_random_table({cudf::type_id::STRING}, row_count{num_rows}, strings_profile); + + auto const num_offsets = (num_rows / row_width) + 1; + auto offsets = cudf::sequence(num_offsets, + cudf::numeric_scalar(0), + cudf::numeric_scalar(row_width)); + + auto source = cudf::make_lists_column(num_offsets - 1, + std::move(offsets), + std::move(strings_table->release().front()), + 0, + rmm::device_buffer{}); + + data_profile const seeds_profile = data_profile_builder().no_validity().distribution( + cudf::type_to_id(), distribution_id::NORMAL, 0, 256); + auto const seed_type = base64 ? cudf::type_id::UINT64 : cudf::type_id::UINT32; + auto const seeds_table = create_random_table({seed_type}, row_count{seed_count}, seeds_profile); + auto seeds = seeds_table->get_column(0); + + state.set_cuda_stream(nvbench::make_cuda_stream_view(cudf::get_default_stream().value())); + + cudf::strings_column_view input(cudf::lists_column_view(source->view()).child()); + auto chars_size = input.chars_size(cudf::get_default_stream()); + state.add_global_memory_reads(chars_size); + state.add_global_memory_writes(num_rows); // output are hashes + + state.exec(nvbench::exec_tag::sync, [&](nvbench::launch& launch) { + auto result = base64 ? nvtext::word_minhash64(source->view(), seeds.view()) + : nvtext::word_minhash(source->view(), seeds.view()); + }); +} + +NVBENCH_BENCH(bench_word_minhash) + .set_name("word_minhash") + .add_int64_axis("num_rows", {131072, 262144, 524288, 1048576, 2097152}) + .add_int64_axis("row_width", {10, 100, 1000}) + .add_int64_axis("seed_count", {2, 25}) + .add_int64_axis("hash_type", {32, 64}); diff --git a/cpp/include/nvtext/minhash.hpp b/cpp/include/nvtext/minhash.hpp index c83a4260c19..7c909f1a948 100644 --- a/cpp/include/nvtext/minhash.hpp +++ b/cpp/include/nvtext/minhash.hpp @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -72,7 +73,7 @@ std::unique_ptr minhash( * * @throw std::invalid_argument if the width < 2 * @throw std::invalid_argument if seeds is empty - * @throw std::overflow_error if `seeds * input.size()` exceeds the column size limit + * @throw std::overflow_error if `seeds.size() * input.size()` exceeds the column size limit * * @param input Strings column to compute minhash * @param seeds Seed values used for the hash algorithm @@ -133,7 +134,7 @@ std::unique_ptr minhash64( * * @throw std::invalid_argument if the width < 2 * @throw std::invalid_argument if seeds is empty - * @throw std::overflow_error if `seeds * input.size()` exceeds the column size limit + * @throw std::overflow_error if `seeds.size() * input.size()` exceeds the column size limit * * @param input Strings column to compute minhash * @param seeds Seed values used for the hash algorithm @@ -150,5 +151,61 @@ std::unique_ptr minhash64( rmm::cuda_stream_view stream = cudf::get_default_stream(), rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); +/** + * @brief Returns the minhash values for each row of strings per seed + * + * Hash values are computed from each string in each row and the + * minimum hash value is returned for each row for each seed. + * Each row of the output list column are seed results for the corresponding + * input row. The order of the elements in each row match the order of + * the seeds provided in the `seeds` parameter. + * + * This function uses MurmurHash3_x86_32 for the hash algorithm. + * + * Any null row entries result in corresponding null output rows. + * + * @throw std::invalid_argument if seeds is empty + * @throw std::overflow_error if `seeds.size() * input.size()` exceeds the column size limit + * + * @param input Lists column of strings to compute minhash + * @param seeds Seed values used for the hash algorithm + * @param stream CUDA stream used for device memory operations and kernel launches + * @param mr Device memory resource used to allocate the returned column's device memory + * @return List column of minhash values for each string per seed + */ +std::unique_ptr word_minhash( + cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); + +/** + * @brief Returns the minhash values for each row of strings per seed + * + * Hash values are computed from each string in each row and the + * minimum hash value is returned for each row for each seed. + * Each row of the output list column are seed results for the corresponding + * input row. The order of the elements in each row match the order of + * the seeds provided in the `seeds` parameter. + * + * This function uses MurmurHash3_x64_128 for the hash algorithm though + * only the first 64-bits of the hash are used in computing the output. + * + * Any null row entries result in corresponding null output rows. + * + * @throw std::invalid_argument if seeds is empty + * @throw std::overflow_error if `seeds.size() * input.size()` exceeds the column size limit + * + * @param input Lists column of strings to compute minhash + * @param seeds Seed values used for the hash algorithm + * @param stream CUDA stream used for device memory operations and kernel launches + * @param mr Device memory resource used to allocate the returned column's device memory + * @return List column of minhash values for each string per seed + */ +std::unique_ptr word_minhash64( + cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream = cudf::get_default_stream(), + rmm::device_async_resource_ref mr = cudf::get_current_device_resource_ref()); /** @} */ // end of group } // namespace CUDF_EXPORT nvtext diff --git a/cpp/src/text/minhash.cu b/cpp/src/text/minhash.cu index 605582f28a6..a03a34f5fa7 100644 --- a/cpp/src/text/minhash.cu +++ b/cpp/src/text/minhash.cu @@ -25,6 +25,8 @@ #include #include #include +#include +#include #include #include #include @@ -151,15 +153,111 @@ std::unique_ptr minhash_fn(cudf::strings_column_view const& input, mr); auto d_hashes = hashes->mutable_view().data(); - constexpr int block_size = 256; - cudf::detail::grid_1d grid{input.size() * cudf::detail::warp_size, block_size}; + constexpr cudf::thread_index_type block_size = 256; + cudf::detail::grid_1d grid{ + static_cast(input.size()) * cudf::detail::warp_size, block_size}; minhash_kernel<<>>( *d_strings, seeds, width, d_hashes); return hashes; } -std::unique_ptr build_list_result(cudf::strings_column_view const& input, +/** + * @brief Compute the minhash of each list row of strings for each seed + * + * This is a warp-per-row algorithm where parallel threads within a warp + * work on strings in a single list row. + * + * @tparam HashFunction hash function to use on each string + * + * @param d_input List of strings to process + * @param seeds Seeds for hashing each string + * @param d_hashes Minhash output values (one per row) + */ +template < + typename HashFunction, + typename hash_value_type = std:: + conditional_t, uint32_t, uint64_t>> +CUDF_KERNEL void minhash_word_kernel(cudf::detail::lists_column_device_view const d_input, + cudf::device_span seeds, + hash_value_type* d_hashes) +{ + auto const idx = cudf::detail::grid_1d::global_thread_id(); + auto const row_idx = idx / cudf::detail::warp_size; + + if (row_idx >= d_input.size()) { return; } + if (d_input.is_null(row_idx)) { return; } + + auto const d_row = cudf::list_device_view(d_input, row_idx); + auto const d_output = d_hashes + (row_idx * seeds.size()); + + // initialize hashes output for this row + auto const lane_idx = static_cast(idx % cudf::detail::warp_size); + if (lane_idx == 0) { + auto const init = d_row.size() == 0 ? 0 : std::numeric_limits::max(); + thrust::fill(thrust::seq, d_output, d_output + seeds.size(), init); + } + __syncwarp(); + + // each lane hashes a string from the input row + for (auto str_idx = lane_idx; str_idx < d_row.size(); str_idx += cudf::detail::warp_size) { + auto const hash_str = + d_row.is_null(str_idx) ? cudf::string_view{} : d_row.element(str_idx); + for (std::size_t seed_idx = 0; seed_idx < seeds.size(); ++seed_idx) { + auto const hasher = HashFunction(seeds[seed_idx]); + // hash string and store the min value + hash_value_type hv; + if constexpr (std::is_same_v) { + hv = hasher(hash_str); + } else { + // This code path assumes the use of MurmurHash3_x64_128 which produces 2 uint64 values + // but only uses the first uint64 value as requested by the LLM team. + hv = thrust::get<0>(hasher(hash_str)); + } + cuda::atomic_ref ref{*(d_output + seed_idx)}; + ref.fetch_min(hv, cuda::std::memory_order_relaxed); + } + } +} + +template < + typename HashFunction, + typename hash_value_type = std:: + conditional_t, uint32_t, uint64_t>> +std::unique_ptr word_minhash_fn(cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_EXPECTS(!seeds.empty(), "Parameter seeds cannot be empty", std::invalid_argument); + CUDF_EXPECTS((static_cast(input.size()) * seeds.size()) < + static_cast(std::numeric_limits::max()), + "The number of seeds times the number of input rows exceeds the column size limit", + std::overflow_error); + + auto const output_type = cudf::data_type{cudf::type_to_id()}; + if (input.is_empty()) { return cudf::make_empty_column(output_type); } + + auto const d_input = cudf::column_device_view::create(input.parent(), stream); + + auto hashes = cudf::make_numeric_column(output_type, + input.size() * static_cast(seeds.size()), + cudf::mask_state::UNALLOCATED, + stream, + mr); + auto d_hashes = hashes->mutable_view().data(); + auto lcdv = cudf::detail::lists_column_device_view(*d_input); + + constexpr cudf::thread_index_type block_size = 256; + cudf::detail::grid_1d grid{ + static_cast(input.size()) * cudf::detail::warp_size, block_size}; + minhash_word_kernel + <<>>(lcdv, seeds, d_hashes); + + return hashes; +} + +std::unique_ptr build_list_result(cudf::column_view const& input, std::unique_ptr&& hashes, cudf::size_type seeds_size, rmm::cuda_stream_view stream, @@ -176,7 +274,7 @@ std::unique_ptr build_list_result(cudf::strings_column_view const& std::move(offsets), std::move(hashes), input.null_count(), - cudf::detail::copy_bitmask(input.parent(), stream, mr), + cudf::detail::copy_bitmask(input, stream, mr), stream, mr); // expect this condition to be very rare @@ -208,7 +306,7 @@ std::unique_ptr minhash(cudf::strings_column_view const& input, { using HashFunction = cudf::hashing::detail::MurmurHash3_x86_32; auto hashes = detail::minhash_fn(input, seeds, width, stream, mr); - return build_list_result(input, std::move(hashes), seeds.size(), stream, mr); + return build_list_result(input.parent(), std::move(hashes), seeds.size(), stream, mr); } std::unique_ptr minhash64(cudf::strings_column_view const& input, @@ -232,7 +330,27 @@ std::unique_ptr minhash64(cudf::strings_column_view const& input, { using HashFunction = cudf::hashing::detail::MurmurHash3_x64_128; auto hashes = detail::minhash_fn(input, seeds, width, stream, mr); - return build_list_result(input, std::move(hashes), seeds.size(), stream, mr); + return build_list_result(input.parent(), std::move(hashes), seeds.size(), stream, mr); +} + +std::unique_ptr word_minhash(cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + using HashFunction = cudf::hashing::detail::MurmurHash3_x86_32; + auto hashes = detail::word_minhash_fn(input, seeds, stream, mr); + return build_list_result(input.parent(), std::move(hashes), seeds.size(), stream, mr); +} + +std::unique_ptr word_minhash64(cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + using HashFunction = cudf::hashing::detail::MurmurHash3_x64_128; + auto hashes = detail::word_minhash_fn(input, seeds, stream, mr); + return build_list_result(input.parent(), std::move(hashes), seeds.size(), stream, mr); } } // namespace detail @@ -276,4 +394,21 @@ std::unique_ptr minhash64(cudf::strings_column_view const& input, return detail::minhash64(input, seeds, width, stream, mr); } +std::unique_ptr word_minhash(cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_FUNC_RANGE(); + return detail::word_minhash(input, seeds, stream, mr); +} + +std::unique_ptr word_minhash64(cudf::lists_column_view const& input, + cudf::device_span seeds, + rmm::cuda_stream_view stream, + rmm::device_async_resource_ref mr) +{ + CUDF_FUNC_RANGE(); + return detail::word_minhash64(input, seeds, stream, mr); +} } // namespace nvtext diff --git a/cpp/tests/text/minhash_tests.cpp b/cpp/tests/text/minhash_tests.cpp index 7575a3ba846..e23f3f6e7d8 100644 --- a/cpp/tests/text/minhash_tests.cpp +++ b/cpp/tests/text/minhash_tests.cpp @@ -139,6 +139,41 @@ TEST_F(MinHashTest, MultiSeedWithNullInputRow) CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results64, expected64); } +TEST_F(MinHashTest, WordsMinHash) +{ + using LCWS = cudf::test::lists_column_wrapper; + auto validity = cudf::test::iterators::null_at(1); + + LCWS input( + {LCWS({"hello", "abcdéfgh"}), + LCWS{}, + LCWS({"rapids", "moré", "test", "text"}), + LCWS({"The", "quick", "brown", "fox", "jumpéd", "over", "the", "lazy", "brown", "dog"})}, + validity); + + auto view = cudf::lists_column_view(input); + + auto seeds = cudf::test::fixed_width_column_wrapper({1, 2}); + auto results = nvtext::word_minhash(view, cudf::column_view(seeds)); + using LCW32 = cudf::test::lists_column_wrapper; + LCW32 expected({LCW32{2069617641u, 1975382903u}, + LCW32{}, + LCW32{657297235u, 1010955999u}, + LCW32{644643885u, 310002789u}}, + validity); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + + auto seeds64 = cudf::test::fixed_width_column_wrapper({11, 22}); + auto results64 = nvtext::word_minhash64(view, cudf::column_view(seeds64)); + using LCW64 = cudf::test::lists_column_wrapper; + LCW64 expected64({LCW64{1940333969930105370ul, 272615362982418219ul}, + LCW64{}, + LCW64{5331949571924938590ul, 2088583894581919741ul}, + LCW64{3400468157617183341ul, 2398577492366130055ul}}, + validity); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results64, expected64); +} + TEST_F(MinHashTest, EmptyTest) { auto input = cudf::make_empty_column(cudf::data_type{cudf::type_id::STRING}); diff --git a/python/cudf/cudf/_lib/nvtext/minhash.pyx b/python/cudf/cudf/_lib/nvtext/minhash.pyx index 5ee15d0e409..59cb8d51440 100644 --- a/python/cudf/cudf/_lib/nvtext/minhash.pyx +++ b/python/cudf/cudf/_lib/nvtext/minhash.pyx @@ -10,6 +10,8 @@ from pylibcudf.libcudf.column.column_view cimport column_view from pylibcudf.libcudf.nvtext.minhash cimport ( minhash as cpp_minhash, minhash64 as cpp_minhash64, + word_minhash as cpp_word_minhash, + word_minhash64 as cpp_word_minhash64, ) from pylibcudf.libcudf.types cimport size_type @@ -54,3 +56,39 @@ def minhash64(Column strings, Column seeds, int width): ) return Column.from_unique_ptr(move(c_result)) + + +@acquire_spill_lock() +def word_minhash(Column input, Column seeds): + + cdef column_view c_input = input.view() + cdef column_view c_seeds = seeds.view() + cdef unique_ptr[column] c_result + + with nogil: + c_result = move( + cpp_word_minhash( + c_input, + c_seeds + ) + ) + + return Column.from_unique_ptr(move(c_result)) + + +@acquire_spill_lock() +def word_minhash64(Column input, Column seeds): + + cdef column_view c_input = input.view() + cdef column_view c_seeds = seeds.view() + cdef unique_ptr[column] c_result + + with nogil: + c_result = move( + cpp_word_minhash64( + c_input, + c_seeds + ) + ) + + return Column.from_unique_ptr(move(c_result)) diff --git a/python/cudf/cudf/_lib/strings/__init__.py b/python/cudf/cudf/_lib/strings/__init__.py index 47a194c4fda..4bf8a9b1a8f 100644 --- a/python/cudf/cudf/_lib/strings/__init__.py +++ b/python/cudf/cudf/_lib/strings/__init__.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. from cudf._lib.nvtext.edit_distance import edit_distance, edit_distance_matrix from cudf._lib.nvtext.generate_ngrams import ( generate_character_ngrams, @@ -6,7 +6,12 @@ hash_character_ngrams, ) from cudf._lib.nvtext.jaccard import jaccard_index -from cudf._lib.nvtext.minhash import minhash, minhash64 +from cudf._lib.nvtext.minhash import ( + minhash, + minhash64, + word_minhash, + word_minhash64, +) from cudf._lib.nvtext.ngrams_tokenize import ngrams_tokenize from cudf._lib.nvtext.normalize import normalize_characters, normalize_spaces from cudf._lib.nvtext.replace import filter_tokens, replace_tokens diff --git a/python/cudf/cudf/core/column/string.py b/python/cudf/cudf/core/column/string.py index 16e6908f308..e059917b0b8 100644 --- a/python/cudf/cudf/core/column/string.py +++ b/python/cudf/cudf/core/column/string.py @@ -5349,6 +5349,76 @@ def minhash64( libstrings.minhash64(self._column, seeds_column, width) ) + def word_minhash(self, seeds: ColumnLike | None = None) -> SeriesOrIndex: + """ + Compute the minhash of a list column of strings. + This uses the MurmurHash3_x86_32 algorithm for the hash function. + + Parameters + ---------- + seeds : ColumnLike + The seeds used for the hash algorithm. + Must be of type uint32. + + Examples + -------- + >>> import cudf + >>> import numpy as np + >>> ls = cudf.Series([["this", "is", "my"], ["favorite", "book"]]) + >>> seeds = cudf.Series([0, 1, 2], dtype=np.uint32) + >>> ls.str.word_minhash(seeds=seeds) + 0 [21141582, 1232889953, 1268336794] + 1 [962346254, 2321233602, 1354839212] + dtype: list + """ + if seeds is None: + seeds_column = column.as_column(0, dtype=np.uint32, length=1) + else: + seeds_column = column.as_column(seeds) + if seeds_column.dtype != np.uint32: + raise ValueError( + f"Expecting a Series with dtype uint32, got {type(seeds)}" + ) + return self._return_or_inplace( + libstrings.word_minhash(self._column, seeds_column) + ) + + def word_minhash64(self, seeds: ColumnLike | None = None) -> SeriesOrIndex: + """ + Compute the minhash of a list column of strings. + This uses the MurmurHash3_x64_128 algorithm for the hash function. + This function generates 2 uint64 values but only the first + uint64 value is used. + + Parameters + ---------- + seeds : ColumnLike + The seeds used for the hash algorithm. + Must be of type uint64. + + Examples + -------- + >>> import cudf + >>> import numpy as np + >>> ls = cudf.Series([["this", "is", "my"], ["favorite", "book"]]) + >>> seeds = cudf.Series([0, 1, 2], dtype=np.uint64) + >>> ls.str.word_minhash64(seeds) + 0 [2603139454418834912, 8644371945174847701, 5541030711534384340] + 1 [5240044617220523711, 5847101123925041457, 153762819128779913] + dtype: list + """ + if seeds is None: + seeds_column = column.as_column(0, dtype=np.uint64, length=1) + else: + seeds_column = column.as_column(seeds) + if seeds_column.dtype != np.uint64: + raise ValueError( + f"Expecting a Series with dtype uint64, got {type(seeds)}" + ) + return self._return_or_inplace( + libstrings.word_minhash64(self._column, seeds_column) + ) + def jaccard_index(self, input: cudf.Series, width: int) -> SeriesOrIndex: """ Compute the Jaccard index between this column and the given diff --git a/python/cudf/cudf/tests/text/test_text_methods.py b/python/cudf/cudf/tests/text/test_text_methods.py index 52179f55da3..997ca357986 100644 --- a/python/cudf/cudf/tests/text/test_text_methods.py +++ b/python/cudf/cudf/tests/text/test_text_methods.py @@ -946,6 +946,66 @@ def test_minhash(): strings.str.minhash64(seeds=seeds) +def test_word_minhash(): + ls = cudf.Series([["this", "is", "my"], ["favorite", "book"]]) + + expected = cudf.Series( + [ + cudf.Series([21141582], dtype=np.uint32), + cudf.Series([962346254], dtype=np.uint32), + ] + ) + actual = ls.str.word_minhash() + assert_eq(expected, actual) + seeds = cudf.Series([0, 1, 2], dtype=np.uint32) + expected = cudf.Series( + [ + cudf.Series([21141582, 1232889953, 1268336794], dtype=np.uint32), + cudf.Series([962346254, 2321233602, 1354839212], dtype=np.uint32), + ] + ) + actual = ls.str.word_minhash(seeds=seeds) + assert_eq(expected, actual) + + expected = cudf.Series( + [ + cudf.Series([2603139454418834912], dtype=np.uint64), + cudf.Series([5240044617220523711], dtype=np.uint64), + ] + ) + actual = ls.str.word_minhash64() + assert_eq(expected, actual) + seeds = cudf.Series([0, 1, 2], dtype=np.uint64) + expected = cudf.Series( + [ + cudf.Series( + [ + 2603139454418834912, + 8644371945174847701, + 5541030711534384340, + ], + dtype=np.uint64, + ), + cudf.Series( + [5240044617220523711, 5847101123925041457, 153762819128779913], + dtype=np.uint64, + ), + ] + ) + actual = ls.str.word_minhash64(seeds=seeds) + assert_eq(expected, actual) + + # test wrong seed types + with pytest.raises(ValueError): + ls.str.word_minhash(seeds="a") + with pytest.raises(ValueError): + seeds = cudf.Series([0, 1, 2], dtype=np.int32) + ls.str.word_minhash(seeds=seeds) + with pytest.raises(ValueError): + seeds = cudf.Series([0, 1, 2], dtype=np.uint32) + ls.str.word_minhash64(seeds=seeds) + + def test_jaccard_index(): str1 = cudf.Series(["the brown dog", "jumped about"]) str2 = cudf.Series(["the black cat", "jumped around"]) diff --git a/python/pylibcudf/pylibcudf/libcudf/nvtext/minhash.pxd b/python/pylibcudf/pylibcudf/libcudf/nvtext/minhash.pxd index 0c352a5068b..f2dd22f43aa 100644 --- a/python/pylibcudf/pylibcudf/libcudf/nvtext/minhash.pxd +++ b/python/pylibcudf/pylibcudf/libcudf/nvtext/minhash.pxd @@ -19,3 +19,13 @@ cdef extern from "nvtext/minhash.hpp" namespace "nvtext" nogil: const column_view &seeds, const size_type width, ) except + + + cdef unique_ptr[column] word_minhash( + const column_view &input, + const column_view &seeds + ) except + + + cdef unique_ptr[column] word_minhash64( + const column_view &input, + const column_view &seeds + ) except + From e98e10981fc245a6837a51e9b6c2b933a5d7acd8 Mon Sep 17 00:00:00 2001 From: David Wendt <45795991+davidwendt@users.noreply.github.com> Date: Tue, 17 Sep 2024 13:19:40 -0400 Subject: [PATCH 08/15] Support multiple new-line characters in regex APIs (#15961) Add support for multiple new-line characters for BOL (`^` / `\A`) and EOL (`$` / `\Z`): - `\n` line-feed (already supported) - `\r` carriage-return - `\u0085` next line (NEL) - `\u2028` line separator - `\u2029` paragraph separator Reference #15746 Authors: - David Wendt (https://github.com/davidwendt) Approvers: - Vukasin Milovanovic (https://github.com/vuule) - Nghia Truong (https://github.com/ttnghia) - Navin Kumar (https://github.com/NVnavkumar) URL: https://github.com/rapidsai/cudf/pull/15961 --- cpp/doxygen/regex.md | 6 +++ cpp/include/cudf/strings/regex/flags.hpp | 20 ++++++-- cpp/include/cudf/strings/string_view.cuh | 11 +++-- cpp/src/strings/regex/regcomp.cpp | 21 ++++++-- cpp/src/strings/regex/regex.inl | 46 +++++++++++++----- cpp/tests/strings/contains_tests.cpp | 59 +++++++++++++++++++++++ cpp/tests/strings/extract_tests.cpp | 40 +++++++++++++++ cpp/tests/strings/findall_tests.cpp | 28 +++++++++++ cpp/tests/strings/replace_regex_tests.cpp | 49 +++++++++++++++++++ cpp/tests/strings/special_chars.h | 25 ++++++++++ 10 files changed, 281 insertions(+), 24 deletions(-) create mode 100644 cpp/tests/strings/special_chars.h diff --git a/cpp/doxygen/regex.md b/cpp/doxygen/regex.md index 8d206f245dc..6d1c91a5752 100644 --- a/cpp/doxygen/regex.md +++ b/cpp/doxygen/regex.md @@ -17,6 +17,12 @@ The details are based on features documented at https://www.regular-expressions. **Note:** The alternation character is the pipe character `|` and not the character included in the tables on this page. There is an issue including the pipe character inside the table markdown that is rendered by doxygen. +By default, only the `\n` character is recognized as a line break. The [cudf::strings::regex_flags::EXT_NEWLINE](@ref cudf::strings::regex_flags) increases the set of line break characters to include: +- Paragraph separator (Unicode: `2029`, UTF-8: `E280A9`) +- Line separator (Unicode: `2028`, UTF-8: `E280A8`) +- Next line (Unicode: `0085`, UTF-8: `C285`) +- Carriage return (Unicode: `000D`, UTF-8: `0D`) + **Invalid regex patterns will result in undefined behavior**. This includes but is not limited to the following: - Unescaped special characters (listed in the third row of the Characters table below) when they are intended to match as literals. - Unmatched paired special characters like `()`, `[]`, and `{}`. diff --git a/cpp/include/cudf/strings/regex/flags.hpp b/cpp/include/cudf/strings/regex/flags.hpp index f7108129dee..4f3fc7086f2 100644 --- a/cpp/include/cudf/strings/regex/flags.hpp +++ b/cpp/include/cudf/strings/regex/flags.hpp @@ -35,10 +35,11 @@ namespace strings { * and to match the Python flag values. */ enum regex_flags : uint32_t { - DEFAULT = 0, ///< default - MULTILINE = 8, ///< the '^' and '$' honor new-line characters - DOTALL = 16, ///< the '.' matching includes new-line characters - ASCII = 256 ///< use only ASCII when matching built-in character classes + DEFAULT = 0, ///< default + MULTILINE = 8, ///< the '^' and '$' honor new-line characters + DOTALL = 16, ///< the '.' matching includes new-line characters + ASCII = 256, ///< use only ASCII when matching built-in character classes + EXT_NEWLINE = 512 ///< new-line matches extended characters }; /** @@ -74,6 +75,17 @@ constexpr bool is_ascii(regex_flags const f) return (f & regex_flags::ASCII) == regex_flags::ASCII; } +/** + * @brief Returns true if the given flags contain EXT_NEWLINE + * + * @param f Regex flags to check + * @return true if `f` includes EXT_NEWLINE + */ +constexpr bool is_ext_newline(regex_flags const f) +{ + return (f & regex_flags::EXT_NEWLINE) == regex_flags::EXT_NEWLINE; +} + /** * @brief Capture groups setting * diff --git a/cpp/include/cudf/strings/string_view.cuh b/cpp/include/cudf/strings/string_view.cuh index abb26d7ccb4..14695c3bb27 100644 --- a/cpp/include/cudf/strings/string_view.cuh +++ b/cpp/include/cudf/strings/string_view.cuh @@ -191,9 +191,14 @@ __device__ inline string_view::const_iterator& string_view::const_iterator::oper __device__ inline string_view::const_iterator& string_view::const_iterator::operator--() { - if (byte_pos > 0) - while (strings::detail::bytes_in_utf8_byte(static_cast(p[--byte_pos])) == 0) - ; + if (byte_pos > 0) { + if (byte_pos == char_pos) { + --byte_pos; + } else { + while (strings::detail::bytes_in_utf8_byte(static_cast(p[--byte_pos])) == 0) + ; + } + } --char_pos; return *this; } diff --git a/cpp/src/strings/regex/regcomp.cpp b/cpp/src/strings/regex/regcomp.cpp index adf650a4f27..7c4c89bd3fb 100644 --- a/cpp/src/strings/regex/regcomp.cpp +++ b/cpp/src/strings/regex/regcomp.cpp @@ -539,15 +539,26 @@ class regex_parser { : static_cast(LBRA); case ')': return RBRA; case '^': { - _chr = is_multiline(_flags) ? chr : '\n'; + if (is_ext_newline(_flags)) { + _chr = is_multiline(_flags) ? 'S' : 'N'; + } else { + _chr = is_multiline(_flags) ? chr : '\n'; + } return BOL; } case '$': { - _chr = is_multiline(_flags) ? chr : '\n'; + if (is_ext_newline(_flags)) { + _chr = is_multiline(_flags) ? 'S' : 'N'; + } else { + _chr = is_multiline(_flags) ? chr : '\n'; + } return EOL; } case '[': return build_cclass(); - case '.': return dot_type; + case '.': { + _chr = is_ext_newline(_flags) ? 'N' : chr; + return dot_type; + } } if (std::find(quantifiers.begin(), quantifiers.end(), static_cast(chr)) == @@ -959,7 +970,7 @@ class regex_compiler { _prog.inst_at(inst_id).u1.cls_id = class_id; } else if (token == CHAR) { _prog.inst_at(inst_id).u1.c = yy; - } else if (token == BOL || token == EOL) { + } else if (token == BOL || token == EOL || token == ANY) { _prog.inst_at(inst_id).u1.c = yy; } push_and(inst_id, inst_id); @@ -1194,7 +1205,7 @@ void reprog::print(regex_flags const flags) case STAR: printf(" STAR next=%d", inst.u2.next_id); break; case PLUS: printf(" PLUS next=%d", inst.u2.next_id); break; case QUEST: printf(" QUEST next=%d", inst.u2.next_id); break; - case ANY: printf(" ANY next=%d", inst.u2.next_id); break; + case ANY: printf(" ANY '%c', next=%d", inst.u1.c, inst.u2.next_id); break; case ANYNL: printf(" ANYNL next=%d", inst.u2.next_id); break; case NOP: printf(" NOP next=%d", inst.u2.next_id); break; case BOL: { diff --git a/cpp/src/strings/regex/regex.inl b/cpp/src/strings/regex/regex.inl index 3b899e4edc1..e34a1e12015 100644 --- a/cpp/src/strings/regex/regex.inl +++ b/cpp/src/strings/regex/regex.inl @@ -126,6 +126,16 @@ __device__ __forceinline__ void reprog_device::reljunk::swaplist() list2 = tmp; } +/** + * @brief Check for supported new-line characters + * + * '\n, \r, \u0085, \u2028, or \u2029' + */ +constexpr bool is_newline(char32_t const ch) +{ + return (ch == '\n' || ch == '\r' || ch == 0x00c285 || ch == 0x00e280a8 || ch == 0x00e280a9); +} + /** * @brief Utility to check a specific character against this class instance. * @@ -258,11 +268,14 @@ __device__ __forceinline__ match_result reprog_device::regexec(string_view const if (checkstart) { auto startchar = static_cast(jnk.startchar); switch (jnk.starttype) { - case BOL: - if (pos == 0) break; - if (jnk.startchar != '^') { return cuda::std::nullopt; } + case BOL: { + if (pos == 0) { break; } + if (startchar != '^' && startchar != 'S') { return cuda::std::nullopt; } + if (startchar != '\n') { break; } --itr; startchar = static_cast('\n'); + [[fallthrough]]; + } case CHAR: { auto const find_itr = find_char(startchar, dstr, itr); if (find_itr.byte_offset() >= dstr.size_bytes()) { return cuda::std::nullopt; } @@ -312,26 +325,34 @@ __device__ __forceinline__ match_result reprog_device::regexec(string_view const id_activate = inst.u2.next_id; expanded = true; break; - case BOL: - if ((pos == 0) || ((inst.u1.c == '^') && (dstr[pos - 1] == '\n'))) { + case BOL: { + auto titr = itr; + auto const prev_c = pos > 0 ? *(--titr) : 0; + if ((pos == 0) || ((inst.u1.c == '^') && (prev_c == '\n')) || + ((inst.u1.c == 'S') && (is_newline(prev_c)))) { id_activate = inst.u2.next_id; expanded = true; } break; - case EOL: + } + case EOL: { // after the last character OR: // - for MULTILINE, if current character is new-line // - for non-MULTILINE, the very last character of the string can also be a new-line + bool const nl = (inst.u1.c == 'S' || inst.u1.c == 'N') ? is_newline(c) : (c == '\n'); if (last_character || - ((c == '\n') && (inst.u1.c != 'Z') && - ((inst.u1.c == '$') || (itr.byte_offset() + 1 == dstr.size_bytes())))) { + (nl && (inst.u1.c != 'Z') && + ((inst.u1.c == '$' || inst.u1.c == 'S') || + (itr.byte_offset() + bytes_in_char_utf8(c) == dstr.size_bytes())))) { id_activate = inst.u2.next_id; expanded = true; } break; + } case BOW: case NBOW: { - auto const prev_c = pos > 0 ? dstr[pos - 1] : 0; + auto titr = itr; + auto const prev_c = pos > 0 ? *(--titr) : 0; auto const word_class = reclass_device{CCLASS_W}; bool const curr_is_word = word_class.is_match(c, _codepoint_flags); bool const prev_is_word = word_class.is_match(prev_c, _codepoint_flags); @@ -366,9 +387,10 @@ __device__ __forceinline__ match_result reprog_device::regexec(string_view const case CHAR: if (inst.u1.c == c) id_activate = inst.u2.next_id; break; - case ANY: - if (c != '\n') id_activate = inst.u2.next_id; - break; + case ANY: { + if ((c == '\n') || ((inst.u1.c == 'N') && is_newline(c))) { break; } + [[fallthrough]]; + } case ANYNL: id_activate = inst.u2.next_id; break; case NCCLASS: case CCLASS: { diff --git a/cpp/tests/strings/contains_tests.cpp b/cpp/tests/strings/contains_tests.cpp index c816316d0ff..acf850c7a66 100644 --- a/cpp/tests/strings/contains_tests.cpp +++ b/cpp/tests/strings/contains_tests.cpp @@ -14,6 +14,8 @@ * limitations under the License. */ +#include "special_chars.h" + #include #include #include @@ -613,6 +615,63 @@ TEST_F(StringsContainsTests, MultiLine) CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(*results, expected_count); } +TEST_F(StringsContainsTests, SpecialNewLines) +{ + auto input = cudf::test::strings_column_wrapper({"zzé" LINE_SEPARATOR "qqq" NEXT_LINE "zzé", + "qqq\rzzé" LINE_SEPARATOR "lll", + "zzé", + "", + "zzé" PARAGRAPH_SEPARATOR, + "abc\nzzé" NEXT_LINE}); + auto view = cudf::strings_column_view(input); + + auto pattern = std::string("^zzé$"); + auto prog = + cudf::strings::regex_program::create(pattern, cudf::strings::regex_flags::EXT_NEWLINE); + auto ml_flags = static_cast(cudf::strings::regex_flags::EXT_NEWLINE | + cudf::strings::regex_flags::MULTILINE); + auto prog_ml = cudf::strings::regex_program::create(pattern, ml_flags); + + auto expected = cudf::test::fixed_width_column_wrapper({0, 0, 1, 0, 1, 0}); + auto results = cudf::strings::contains_re(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + expected = cudf::test::fixed_width_column_wrapper({1, 1, 1, 0, 1, 1}); + results = cudf::strings::contains_re(view, *prog_ml); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + + expected = cudf::test::fixed_width_column_wrapper({0, 0, 1, 0, 1, 0}); + results = cudf::strings::matches_re(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + expected = cudf::test::fixed_width_column_wrapper({1, 0, 1, 0, 1, 0}); + results = cudf::strings::matches_re(view, *prog_ml); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + + auto counts = cudf::test::fixed_width_column_wrapper({0, 0, 1, 0, 1, 0}); + results = cudf::strings::count_re(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, counts); + counts = cudf::test::fixed_width_column_wrapper({2, 1, 1, 0, 1, 1}); + results = cudf::strings::count_re(view, *prog_ml); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, counts); + + pattern = std::string("q.*l"); + prog = cudf::strings::regex_program::create(pattern); + expected = cudf::test::fixed_width_column_wrapper({0, 1, 0, 0, 0, 0}); + results = cudf::strings::contains_re(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + // inst ANY will stop matching on first 'newline' and so should not match anything here + prog = cudf::strings::regex_program::create(pattern, cudf::strings::regex_flags::EXT_NEWLINE); + expected = cudf::test::fixed_width_column_wrapper({0, 0, 0, 0, 0, 0}); + results = cudf::strings::contains_re(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); + // including the DOTALL flag accepts the newline characters + auto dot_flags = static_cast(cudf::strings::regex_flags::EXT_NEWLINE | + cudf::strings::regex_flags::DOTALL); + prog = cudf::strings::regex_program::create(pattern, dot_flags); + expected = cudf::test::fixed_width_column_wrapper({0, 1, 0, 0, 0, 0}); + results = cudf::strings::contains_re(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(*results, expected); +} + TEST_F(StringsContainsTests, EndOfString) { auto input = cudf::test::strings_column_wrapper( diff --git a/cpp/tests/strings/extract_tests.cpp b/cpp/tests/strings/extract_tests.cpp index b26cbd5a549..1491da758d5 100644 --- a/cpp/tests/strings/extract_tests.cpp +++ b/cpp/tests/strings/extract_tests.cpp @@ -14,9 +14,12 @@ * limitations under the License. */ +#include "special_chars.h" + #include #include #include +#include #include #include @@ -200,6 +203,43 @@ TEST_F(StringsExtractTests, DotAll) CUDF_TEST_EXPECT_TABLES_EQUAL(*results, expected); } +TEST_F(StringsExtractTests, SpecialNewLines) +{ + auto input = cudf::test::strings_column_wrapper({"zzé" NEXT_LINE "qqq" LINE_SEPARATOR "zzé", + "qqq" LINE_SEPARATOR "zzé\rlll", + "zzé", + "", + "zzé" NEXT_LINE, + "abc" PARAGRAPH_SEPARATOR "zzé\n"}); + auto view = cudf::strings_column_view(input); + + auto prog = + cudf::strings::regex_program::create("(^zzé$)", cudf::strings::regex_flags::EXT_NEWLINE); + auto results = cudf::strings::extract(view, *prog); + auto expected = + cudf::test::strings_column_wrapper({"", "", "zzé", "", "zzé", ""}, {0, 0, 1, 0, 1, 0}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view().column(0), expected); + + auto both_flags = static_cast( + cudf::strings::regex_flags::EXT_NEWLINE | cudf::strings::regex_flags::MULTILINE); + auto prog_ml = cudf::strings::regex_program::create("^(zzé)$", both_flags); + results = cudf::strings::extract(view, *prog_ml); + expected = + cudf::test::strings_column_wrapper({"zzé", "zzé", "zzé", "", "zzé", "zzé"}, {1, 1, 1, 0, 1, 1}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view().column(0), expected); + + prog = cudf::strings::regex_program::create("q(q.*l)l"); + expected = cudf::test::strings_column_wrapper({"", "qq" LINE_SEPARATOR "zzé\rll", "", "", "", ""}, + {0, 1, 0, 0, 0, 0}); + results = cudf::strings::extract(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view().column(0), expected); + // expect no matches here since the newline(s) interrupts the pattern + prog = cudf::strings::regex_program::create("q(q.*l)l", cudf::strings::regex_flags::EXT_NEWLINE); + expected = cudf::test::strings_column_wrapper({"", "", "", "", "", ""}, {0, 0, 0, 0, 0, 0}); + results = cudf::strings::extract(view, *prog); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view().column(0), expected); +} + TEST_F(StringsExtractTests, EmptyExtractTest) { std::vector h_strings{nullptr, "AAA", "AAA_A", "AAA_AAA_", "A__", ""}; diff --git a/cpp/tests/strings/findall_tests.cpp b/cpp/tests/strings/findall_tests.cpp index 4582dcb1e38..47606b9b3ed 100644 --- a/cpp/tests/strings/findall_tests.cpp +++ b/cpp/tests/strings/findall_tests.cpp @@ -14,6 +14,8 @@ * limitations under the License. */ +#include "special_chars.h" + #include #include #include @@ -80,6 +82,32 @@ TEST_F(StringsFindallTests, DotAll) CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(results->view(), expected); } +TEST_F(StringsFindallTests, SpecialNewLines) +{ + auto input = cudf::test::strings_column_wrapper({"zzé" PARAGRAPH_SEPARATOR "qqq\nzzé", + "qqq\nzzé" PARAGRAPH_SEPARATOR "lll", + "zzé", + "", + "zzé\r", + "zzé" LINE_SEPARATOR "zzé" NEXT_LINE}); + auto view = cudf::strings_column_view(input); + + auto prog = + cudf::strings::regex_program::create("(^zzé$)", cudf::strings::regex_flags::EXT_NEWLINE); + auto results = cudf::strings::findall(view, *prog); + using LCW = cudf::test::lists_column_wrapper; + LCW expected({LCW{}, LCW{}, LCW{"zzé"}, LCW{}, LCW{"zzé"}, LCW{}}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected); + + auto both_flags = static_cast( + cudf::strings::regex_flags::EXT_NEWLINE | cudf::strings::regex_flags::MULTILINE); + auto prog_ml = cudf::strings::regex_program::create("^(zzé)$", both_flags); + results = cudf::strings::findall(view, *prog_ml); + LCW expected_ml( + {LCW{"zzé", "zzé"}, LCW{"zzé"}, LCW{"zzé"}, LCW{}, LCW{"zzé"}, LCW{"zzé", "zzé"}}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected_ml); +} + TEST_F(StringsFindallTests, MediumRegex) { // This results in 15 regex instructions and falls in the 'medium' range. diff --git a/cpp/tests/strings/replace_regex_tests.cpp b/cpp/tests/strings/replace_regex_tests.cpp index 8c0482653fb..9847d8d6bb5 100644 --- a/cpp/tests/strings/replace_regex_tests.cpp +++ b/cpp/tests/strings/replace_regex_tests.cpp @@ -14,6 +14,8 @@ * limitations under the License. */ +#include "special_chars.h" + #include #include #include @@ -245,6 +247,53 @@ TEST_F(StringsReplaceRegexTest, Multiline) CUDF_TEST_EXPECT_COLUMNS_EQUIVALENT(*results, br_expected); } +TEST_F(StringsReplaceRegexTest, SpecialNewLines) +{ + auto input = cudf::test::strings_column_wrapper({"zzé" NEXT_LINE "qqq" NEXT_LINE "zzé", + "qqq" NEXT_LINE "zzé" NEXT_LINE "lll", + "zzé", + "", + "zzé" PARAGRAPH_SEPARATOR, + "abc\rzzé\r"}); + auto view = cudf::strings_column_view(input); + auto repl = cudf::string_scalar("_"); + auto pattern = std::string("^zzé$"); + auto prog = + cudf::strings::regex_program::create(pattern, cudf::strings::regex_flags::EXT_NEWLINE); + auto results = cudf::strings::replace_re(view, *prog, repl); + auto expected = cudf::test::strings_column_wrapper({"zzé" NEXT_LINE "qqq" NEXT_LINE "zzé", + "qqq" NEXT_LINE "zzé" NEXT_LINE "lll", + "_", + "", + "_" PARAGRAPH_SEPARATOR, + "abc\rzzé\r"}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected); + + auto both_flags = static_cast( + cudf::strings::regex_flags::EXT_NEWLINE | cudf::strings::regex_flags::MULTILINE); + auto prog_ml = cudf::strings::regex_program::create(pattern, both_flags); + results = cudf::strings::replace_re(view, *prog_ml, repl); + expected = cudf::test::strings_column_wrapper({"_" NEXT_LINE "qqq" NEXT_LINE "_", + "qqq" NEXT_LINE "_" NEXT_LINE "lll", + "_", + "", + "_" PARAGRAPH_SEPARATOR, + "abc\r_\r"}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected); + + auto repl_template = std::string("[\\1]"); + pattern = std::string("(^zzé$)"); + prog = cudf::strings::regex_program::create(pattern, both_flags); + results = cudf::strings::replace_with_backrefs(view, *prog, repl_template); + expected = cudf::test::strings_column_wrapper({"[zzé]" NEXT_LINE "qqq" NEXT_LINE "[zzé]", + "qqq" NEXT_LINE "[zzé]" NEXT_LINE "lll", + "[zzé]", + "", + "[zzé]" PARAGRAPH_SEPARATOR, + "abc\r[zzé]\r"}); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(results->view(), expected); +} + TEST_F(StringsReplaceRegexTest, ReplaceBackrefsRegexTest) { std::vector h_strings{"the quick brown fox jumps over the lazy dog", diff --git a/cpp/tests/strings/special_chars.h b/cpp/tests/strings/special_chars.h new file mode 100644 index 00000000000..0d630f6bb52 --- /dev/null +++ b/cpp/tests/strings/special_chars.h @@ -0,0 +1,25 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +namespace cudf::test { + +// special new-line characters for use with regex_flags::EXT_NEWLINE +#define NEXT_LINE "\xC2\x85" +#define LINE_SEPARATOR "\xE2\x80\xA8" +#define PARAGRAPH_SEPARATOR "\xE2\x80\xA9" + +} // namespace cudf::test From a112f684318e24b2321df48004ca58180f169410 Mon Sep 17 00:00:00 2001 From: Muhammad Haseeb <14217455+mhaseeb123@users.noreply.github.com> Date: Tue, 17 Sep 2024 11:31:38 -0700 Subject: [PATCH 09/15] Add io_type axis with default `PINNED_BUFFER` to nvbench PQ multithreaded reader (#16809) Closes #16758 This PR adds an `io_type` axis to the benchmarks in `PARQUET_MULTITHREAD_READER_NVBENCH` with `PINNED_BUFFER` as default value. More description at #16758. Authors: - Muhammad Haseeb (https://github.com/mhaseeb123) Approvers: - Yunsong Wang (https://github.com/PointKernel) - David Wendt (https://github.com/davidwendt) - Tianyu Liu (https://github.com/kingcrimsontianyu) URL: https://github.com/rapidsai/cudf/pull/16809 --- .../io/parquet/parquet_reader_multithread.cpp | 36 ++++++++++++------- 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/cpp/benchmarks/io/parquet/parquet_reader_multithread.cpp b/cpp/benchmarks/io/parquet/parquet_reader_multithread.cpp index 3abd4280081..7121cb9f034 100644 --- a/cpp/benchmarks/io/parquet/parquet_reader_multithread.cpp +++ b/cpp/benchmarks/io/parquet/parquet_reader_multithread.cpp @@ -50,7 +50,7 @@ std::string get_label(std::string const& test_name, nvbench::state const& state) } std::tuple, size_t, size_t> write_file_data( - nvbench::state& state, std::vector const& d_types) + nvbench::state& state, std::vector const& d_types, io_type io_source_type) { cudf::size_type const cardinality = state.get_int64("cardinality"); cudf::size_type const run_length = state.get_int64("run_length"); @@ -63,7 +63,7 @@ std::tuple, size_t, size_t> write_file_data( size_t total_file_size = 0; for (size_t i = 0; i < num_files; ++i) { - cuio_source_sink_pair source_sink{io_type::HOST_BUFFER}; + cuio_source_sink_pair source_sink{io_source_type}; auto const tbl = create_random_table( cycle_dtypes(d_types, num_cols), @@ -92,11 +92,13 @@ void BM_parquet_multithreaded_read_common(nvbench::state& state, { size_t const data_size = state.get_int64("total_data_size"); auto const num_threads = state.get_int64("num_threads"); + auto const source_type = retrieve_io_type_enum(state.get_string("io_type")); auto streams = cudf::detail::fork_streams(cudf::get_default_stream(), num_threads); BS::thread_pool threads(num_threads); - auto [source_sink_vector, total_file_size, num_files] = write_file_data(state, d_types); + auto [source_sink_vector, total_file_size, num_files] = + write_file_data(state, d_types, source_type); std::vector source_info_vector; std::transform(source_sink_vector.begin(), source_sink_vector.end(), @@ -173,10 +175,12 @@ void BM_parquet_multithreaded_read_chunked_common(nvbench::state& state, auto const num_threads = state.get_int64("num_threads"); size_t const input_limit = state.get_int64("input_limit"); size_t const output_limit = state.get_int64("output_limit"); + auto const source_type = retrieve_io_type_enum(state.get_string("io_type")); auto streams = cudf::detail::fork_streams(cudf::get_default_stream(), num_threads); BS::thread_pool threads(num_threads); - auto [source_sink_vector, total_file_size, num_files] = write_file_data(state, d_types); + auto [source_sink_vector, total_file_size, num_files] = + write_file_data(state, d_types, source_type); std::vector source_info_vector; std::transform(source_sink_vector.begin(), source_sink_vector.end(), @@ -264,7 +268,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_mixed) .add_int64_axis("total_data_size", {512 * 1024 * 1024, 1024 * 1024 * 1024}) .add_int64_axis("num_threads", {1, 2, 4, 8}) .add_int64_axis("num_cols", {4}) - .add_int64_axis("run_length", {8}); + .add_int64_axis("run_length", {8}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); NVBENCH_BENCH(BM_parquet_multithreaded_read_fixed_width) .set_name("parquet_multithreaded_read_decode_fixed_width") @@ -273,7 +278,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_fixed_width) .add_int64_axis("total_data_size", {512 * 1024 * 1024, 1024 * 1024 * 1024}) .add_int64_axis("num_threads", {1, 2, 4, 8}) .add_int64_axis("num_cols", {4}) - .add_int64_axis("run_length", {8}); + .add_int64_axis("run_length", {8}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); NVBENCH_BENCH(BM_parquet_multithreaded_read_string) .set_name("parquet_multithreaded_read_decode_string") @@ -282,7 +288,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_string) .add_int64_axis("total_data_size", {512 * 1024 * 1024, 1024 * 1024 * 1024}) .add_int64_axis("num_threads", {1, 2, 4, 8}) .add_int64_axis("num_cols", {4}) - .add_int64_axis("run_length", {8}); + .add_int64_axis("run_length", {8}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); NVBENCH_BENCH(BM_parquet_multithreaded_read_list) .set_name("parquet_multithreaded_read_decode_list") @@ -291,7 +298,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_list) .add_int64_axis("total_data_size", {512 * 1024 * 1024, 1024 * 1024 * 1024}) .add_int64_axis("num_threads", {1, 2, 4, 8}) .add_int64_axis("num_cols", {4}) - .add_int64_axis("run_length", {8}); + .add_int64_axis("run_length", {8}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); // mixed data types: fixed width, strings NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_mixed) @@ -303,7 +311,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_mixed) .add_int64_axis("num_cols", {4}) .add_int64_axis("run_length", {8}) .add_int64_axis("input_limit", {640 * 1024 * 1024}) - .add_int64_axis("output_limit", {640 * 1024 * 1024}); + .add_int64_axis("output_limit", {640 * 1024 * 1024}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_fixed_width) .set_name("parquet_multithreaded_read_decode_chunked_fixed_width") @@ -314,7 +323,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_fixed_width) .add_int64_axis("num_cols", {4}) .add_int64_axis("run_length", {8}) .add_int64_axis("input_limit", {640 * 1024 * 1024}) - .add_int64_axis("output_limit", {640 * 1024 * 1024}); + .add_int64_axis("output_limit", {640 * 1024 * 1024}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_string) .set_name("parquet_multithreaded_read_decode_chunked_string") @@ -325,7 +335,8 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_string) .add_int64_axis("num_cols", {4}) .add_int64_axis("run_length", {8}) .add_int64_axis("input_limit", {640 * 1024 * 1024}) - .add_int64_axis("output_limit", {640 * 1024 * 1024}); + .add_int64_axis("output_limit", {640 * 1024 * 1024}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_list) .set_name("parquet_multithreaded_read_decode_chunked_list") @@ -336,4 +347,5 @@ NVBENCH_BENCH(BM_parquet_multithreaded_read_chunked_list) .add_int64_axis("num_cols", {4}) .add_int64_axis("run_length", {8}) .add_int64_axis("input_limit", {640 * 1024 * 1024}) - .add_int64_axis("output_limit", {640 * 1024 * 1024}); + .add_int64_axis("output_limit", {640 * 1024 * 1024}) + .add_string_axis("io_type", {"PINNED_BUFFER"}); From 57ae3e372e93a16db8aef143759ef58392c4215f Mon Sep 17 00:00:00 2001 From: Bradley Dice Date: Wed, 18 Sep 2024 02:10:58 -0500 Subject: [PATCH 10/15] Enable cudf.pandas REPL and -c command support (#16428) This PR enables support for two features: - `python -m cudf.pandas` gives a REPL experience (previously it raised an error) - `python -m cudf.pandas -c ""` runs the provided commands (previously unsupported) Authors: - Bradley Dice (https://github.com/bdice) - Matthew Murray (https://github.com/Matt711) Approvers: - Matthew Murray (https://github.com/Matt711) URL: https://github.com/rapidsai/cudf/pull/16428 --- docs/cudf/source/cudf_pandas/usage.md | 20 +++++ python/cudf/cudf/pandas/__main__.py | 36 +++++++- python/cudf/cudf_pandas_tests/test_main.py | 100 +++++++++++++++++++++ 3 files changed, 154 insertions(+), 2 deletions(-) create mode 100644 python/cudf/cudf_pandas_tests/test_main.py diff --git a/docs/cudf/source/cudf_pandas/usage.md b/docs/cudf/source/cudf_pandas/usage.md index 0398a8d7086..41838e01dd9 100644 --- a/docs/cudf/source/cudf_pandas/usage.md +++ b/docs/cudf/source/cudf_pandas/usage.md @@ -120,3 +120,23 @@ To profile a script being run from the command line, pass the ```bash python -m cudf.pandas --profile script.py ``` + +### cudf.pandas CLI Features + +Several of the ways to provide input to the `python` interpreter also work with `python -m cudf.pandas`, such as the REPL, the `-c` flag, and reading from stdin. + +Executing `python -m cudf.pandas` with no script name will enter a REPL (read-eval-print loop) similar to the behavior of the normal `python` interpreter. + +The `-c` flag accepts a code string to run, like this: + +```bash +$ python -m cudf.pandas -c "import pandas; print(pandas)" + +``` + +Users can also provide code to execute from stdin, like this: + +```bash +$ echo "import pandas; print(pandas)" | python -m cudf.pandas + +``` diff --git a/python/cudf/cudf/pandas/__main__.py b/python/cudf/cudf/pandas/__main__.py index 3a82829eb7a..e0d3d9101a9 100644 --- a/python/cudf/cudf/pandas/__main__.py +++ b/python/cudf/cudf/pandas/__main__.py @@ -10,6 +10,7 @@ """ import argparse +import code import runpy import sys import tempfile @@ -21,6 +22,8 @@ @contextmanager def profile(function_profile, line_profile, fn): + if fn is None and (line_profile or function_profile): + raise RuntimeError("Enabling the profiler requires a script name.") if line_profile: with open(fn) as f: lines = f.readlines() @@ -54,6 +57,11 @@ def main(): dest="module", nargs=1, ) + parser.add_argument( + "-c", + dest="cmd", + nargs=1, + ) parser.add_argument( "--profile", action="store_true", @@ -72,9 +80,18 @@ def main(): args = parser.parse_args() + if args.cmd: + f = tempfile.NamedTemporaryFile(mode="w+b", suffix=".py") + f.write(args.cmd[0].encode()) + f.seek(0) + args.args.insert(0, f.name) + install() - with profile(args.profile, args.line_profile, args.args[0]) as fn: - args.args[0] = fn + + script_name = args.args[0] if len(args.args) > 0 else None + with profile(args.profile, args.line_profile, script_name) as fn: + if script_name is not None: + args.args[0] = fn if args.module: (module,) = args.module # run the module passing the remaining arguments @@ -85,6 +102,21 @@ def main(): # Remove ourself from argv and continue sys.argv[:] = args.args runpy.run_path(args.args[0], run_name="__main__") + else: + if sys.stdin.isatty(): + banner = f"Python {sys.version} on {sys.platform}" + site_import = not sys.flags.no_site + if site_import: + cprt = 'Type "help", "copyright", "credits" or "license" for more information.' + banner += "\n" + cprt + else: + # Don't show prompts or banners if stdin is not a TTY + sys.ps1 = "" + sys.ps2 = "" + banner = "" + + # Launch an interactive interpreter + code.interact(banner=banner, exitmsg="") if __name__ == "__main__": diff --git a/python/cudf/cudf_pandas_tests/test_main.py b/python/cudf/cudf_pandas_tests/test_main.py new file mode 100644 index 00000000000..326224c8fc0 --- /dev/null +++ b/python/cudf/cudf_pandas_tests/test_main.py @@ -0,0 +1,100 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# All rights reserved. +# SPDX-License-Identifier: Apache-2.0 + +import subprocess +import tempfile +import textwrap + + +def _run_python(*, cudf_pandas, command): + executable = "python " + if cudf_pandas: + executable += "-m cudf.pandas " + return subprocess.run( + executable + command, + shell=True, + capture_output=True, + check=True, + text=True, + ) + + +def test_run_cudf_pandas_with_script(): + with tempfile.NamedTemporaryFile(mode="w", suffix=".py", delete=True) as f: + code = textwrap.dedent( + """ + import pandas as pd + df = pd.DataFrame({'a': [1, 2, 3]}) + print(df['a'].sum()) + """ + ) + f.write(code) + f.flush() + + res = _run_python(cudf_pandas=True, command=f.name) + expect = _run_python(cudf_pandas=False, command=f.name) + + assert res.stdout != "" + assert res.stdout == expect.stdout + + +def test_run_cudf_pandas_with_script_with_cmd_args(): + input_args_and_code = """-c 'import pandas as pd; df = pd.DataFrame({"a": [1, 2, 3]}); print(df["a"].sum())'""" + + res = _run_python(cudf_pandas=True, command=input_args_and_code) + expect = _run_python(cudf_pandas=False, command=input_args_and_code) + + assert res.stdout != "" + assert res.stdout == expect.stdout + + +def test_run_cudf_pandas_with_script_with_cmd_args_check_cudf(): + """Verify that cudf is active with -m cudf.pandas.""" + input_args_and_code = """-c 'import pandas as pd; print(pd)'""" + + res = _run_python(cudf_pandas=True, command=input_args_and_code) + expect = _run_python(cudf_pandas=False, command=input_args_and_code) + + assert "cudf" in res.stdout + assert "cudf" not in expect.stdout + + +def test_cudf_pandas_script_repl(): + def start_repl_process(cmd): + return subprocess.Popen( + cmd.split(), + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + text=True, + ) + + def get_repl_output(process, commands): + for command in commands: + process.stdin.write(command) + process.stdin.flush() + return process.communicate() + + p1 = start_repl_process("python -m cudf.pandas") + p2 = start_repl_process("python") + commands = [ + "import pandas as pd\n", + "print(pd.Series(range(2)).sum())\n", + "print(pd.Series(range(5)).sum())\n", + "import sys\n", + "print(pd.Series(list('abcd')), out=sys.stderr)\n", + ] + + res = get_repl_output(p1, commands) + expect = get_repl_output(p2, commands) + + # Check stdout + assert res[0] != "" + assert res[0] == expect[0] + + # Check stderr + assert res[1] != "" + assert res[1] == expect[1] + + p1.kill() + p2.kill() From 44a9c10105ab06538264e727188a04d623b0811e Mon Sep 17 00:00:00 2001 From: Muhammad Haseeb <14217455+mhaseeb123@users.noreply.github.com> Date: Wed, 18 Sep 2024 01:25:59 -0700 Subject: [PATCH 11/15] Add a benchmark to study Parquet reader's performance for wide tables (#16751) Related to #16750 This PR adds a benchmark to study read throughput of Parquet reader for wide tables. Authors: - Muhammad Haseeb (https://github.com/mhaseeb123) Approvers: - Paul Mattione (https://github.com/pmattione-nvidia) - Vukasin Milovanovic (https://github.com/vuule) URL: https://github.com/rapidsai/cudf/pull/16751 --- .../io/parquet/parquet_reader_input.cpp | 87 ++++++++++++++++++- 1 file changed, 85 insertions(+), 2 deletions(-) diff --git a/cpp/benchmarks/io/parquet/parquet_reader_input.cpp b/cpp/benchmarks/io/parquet/parquet_reader_input.cpp index 7563c823454..ce115fd7723 100644 --- a/cpp/benchmarks/io/parquet/parquet_reader_input.cpp +++ b/cpp/benchmarks/io/parquet/parquet_reader_input.cpp @@ -32,7 +32,8 @@ constexpr cudf::size_type num_cols = 64; void parquet_read_common(cudf::size_type num_rows_to_read, cudf::size_type num_cols_to_read, cuio_source_sink_pair& source_sink, - nvbench::state& state) + nvbench::state& state, + size_t table_data_size = data_size) { cudf::io::parquet_reader_options read_opts = cudf::io::parquet_reader_options::builder(source_sink.make_source_info()); @@ -52,7 +53,7 @@ void parquet_read_common(cudf::size_type num_rows_to_read, }); auto const time = state.get_summary("nv/cold/time/gpu/mean").get_float64("value"); - state.add_element_count(static_cast(data_size) / time, "bytes_per_second"); + state.add_element_count(static_cast(table_data_size) / time, "bytes_per_second"); state.add_buffer_size( mem_stats_logger.peak_memory_usage(), "peak_memory_usage", "peak_memory_usage"); state.add_buffer_size(source_sink.size(), "encoded_file_size", "encoded_file_size"); @@ -231,6 +232,70 @@ void BM_parquet_read_chunks(nvbench::state& state, nvbench::type_list +void BM_parquet_read_wide_tables(nvbench::state& state, + nvbench::type_list> type_list) +{ + auto const d_type = get_type_or_group(static_cast(DataType)); + + auto const n_col = static_cast(state.get_int64("num_cols")); + auto const data_size_bytes = static_cast(state.get_int64("data_size_mb") << 20); + auto const cardinality = static_cast(state.get_int64("cardinality")); + auto const run_length = static_cast(state.get_int64("run_length")); + auto const source_type = io_type::DEVICE_BUFFER; + cuio_source_sink_pair source_sink(source_type); + + auto const num_rows_written = [&]() { + auto const tbl = create_random_table( + cycle_dtypes(d_type, n_col), + table_size_bytes{data_size_bytes}, + data_profile_builder().cardinality(cardinality).avg_run_length(run_length)); + auto const view = tbl->view(); + + cudf::io::parquet_writer_options write_opts = + cudf::io::parquet_writer_options::builder(source_sink.make_sink_info(), view) + .compression(cudf::io::compression_type::NONE); + cudf::io::write_parquet(write_opts); + return view.num_rows(); + }(); + + parquet_read_common(num_rows_written, n_col, source_sink, state, data_size_bytes); +} + +void BM_parquet_read_wide_tables_mixed(nvbench::state& state) +{ + auto const d_type = []() { + auto d_type1 = get_type_or_group(static_cast(data_type::INTEGRAL)); + auto d_type2 = get_type_or_group(static_cast(data_type::FLOAT)); + d_type1.reserve(d_type1.size() + d_type2.size()); + std::move(d_type2.begin(), d_type2.end(), std::back_inserter(d_type1)); + return d_type1; + }(); + + auto const n_col = static_cast(state.get_int64("num_cols")); + auto const data_size_bytes = static_cast(state.get_int64("data_size_mb") << 20); + auto const cardinality = static_cast(state.get_int64("cardinality")); + auto const run_length = static_cast(state.get_int64("run_length")); + auto const source_type = io_type::DEVICE_BUFFER; + cuio_source_sink_pair source_sink(source_type); + + auto const num_rows_written = [&]() { + auto const tbl = create_random_table( + cycle_dtypes(d_type, n_col), + table_size_bytes{data_size_bytes}, + data_profile_builder().cardinality(cardinality).avg_run_length(run_length)); + auto const view = tbl->view(); + + cudf::io::parquet_writer_options write_opts = + cudf::io::parquet_writer_options::builder(source_sink.make_sink_info(), view) + .compression(cudf::io::compression_type::NONE); + cudf::io::write_parquet(write_opts); + return view.num_rows(); + }(); + + parquet_read_common(num_rows_written, n_col, source_sink, state, data_size_bytes); +} + using d_type_list = nvbench::enum_type_list; +NVBENCH_BENCH_TYPES(BM_parquet_read_wide_tables, NVBENCH_TYPE_AXES(d_type_list_wide_table)) + .set_name("parquet_read_wide_tables") + .set_min_samples(4) + .set_type_axes_names({"data_type"}) + .add_int64_axis("data_size_mb", {1024, 2048, 4096}) + .add_int64_axis("num_cols", {256, 512, 1024}) + .add_int64_axis("cardinality", {0, 1000}) + .add_int64_axis("run_length", {1, 32}); + +NVBENCH_BENCH(BM_parquet_read_wide_tables_mixed) + .set_name("parquet_read_wide_tables_mixed") + .set_min_samples(4) + .add_int64_axis("data_size_mb", {1024, 2048, 4096}) + .add_int64_axis("num_cols", {256, 512, 1024}) + .add_int64_axis("cardinality", {0, 1000}) + .add_int64_axis("run_length", {1, 32}); + // a benchmark for structs that only contain fixed-width types using d_type_list_struct_only = nvbench::enum_type_list; NVBENCH_BENCH_TYPES(BM_parquet_read_fixed_width_struct, NVBENCH_TYPE_AXES(d_type_list_struct_only)) From 2a9a8f5b95ea62824147f1629de1fe52fdbf1254 Mon Sep 17 00:00:00 2001 From: Jake Awe <50372925+AyodeAwe@users.noreply.github.com> Date: Wed, 18 Sep 2024 09:02:41 -0500 Subject: [PATCH 12/15] use get-pr-info from nv-gha-runners (#16819) There are two implementations of the same action; one in [rapidsai/shared-actions](https://github.com/rapidsai/shared-actions/tree/main/get-pr-info) and [the other](https://github.com/nv-gha-runners/get-pr-info) in the nv-gha-runners org. This PR switches to the implementation in the nv-gha-runners group in order to keep a single source of truth. Tested in https://github.com/rapidsai/cudf/actions/runs/10906617425/job/30268277178?pr=16819#step:4:5 --- .github/workflows/pr.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index a4a8f036174..d7d14ea12ff 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -52,7 +52,7 @@ jobs: steps: - name: Get PR info id: get-pr-info - uses: rapidsai/shared-actions/get-pr-info@main + uses: nv-gha-runners/get-pr-info@main - name: Checkout code repo uses: actions/checkout@v4 with: From 2a3026dec9dca553c2be7d49f2d0e6c09a9f4589 Mon Sep 17 00:00:00 2001 From: Muhammad Haseeb <14217455+mhaseeb123@users.noreply.github.com> Date: Wed, 18 Sep 2024 10:04:31 -0700 Subject: [PATCH 13/15] Change the Parquet writer's `default_row_group_size_bytes` from 128MB to inf (#16750) Closes #16733. This PR changes the default value of Parquet writer's default max row group size from 128MB to 1Million rows. This allows avoiding thin row group strips when writing wide (> 512 cols) tables resulting in a significantly improved read throughput for wide tables (especially when low cardinality) with virtually no impact to narrow-tables read performance. Benchmarked using: #16751 ## Results ### Hardware ``` GPU: NVIDIA RTX 5880 Ada Generation SM Version: 890 (PTX Version: 860) Number of SMs: 110 SM Default Clock Rate: 18446744071874 MHz Global Memory: 23879 MiB Free / 48632 MiB Total Global Memory Bus Peak: 960 GB/sec (384-bit DDR @10001MHz) Max Shared Memory: 100 KiB/SM, 48 KiB/Block L2 Cache Size: 98304 KiB Maximum Active Blocks: 24/SM Maximum Active Threads: 1536/SM, 1024/Block Available Registers: 65536/SM, 65536/Block ECC Enabled: No ``` ### Read Throughput ``` ## parquet_read_wide_tables_mixed | T | num_rows | num_cols | GPU Time_old | GPU Time_new | bytes_per_second_old | bytes_per_second_new | peak_memory_usage_old | peak_memory_usage_new | encoded_file_size_old | encoded_file_size_new | |-----------|----------|----------|----------------|----------------|----------------------|----------------------|-----------------------|-----------------------|-----------------------|-----------------------| | INTEGRAL | 10000 | 64 | 940.690 us | 928.387 us | 570720378014 | 578283256754 | 3.405 MiB | 3.405 MiB | 748.248 KiB | 748.248 KiB | | INTEGRAL | 100000 | 64 | 2.053 ms | 2.037 ms | 261541794543 | 263500220325 | 28.308 MiB | 28.308 MiB | 5.164 MiB | 5.164 MiB | | INTEGRAL | 500000 | 64 | 5.783 ms | 5.693 ms | 92838553328 | 94296134644 | 139.928 MiB | 139.042 MiB | 24.698 MiB | 24.325 MiB | | INTEGRAL | 1000000 | 64 | 11.400 ms | 10.775 ms | 47092763803 | 49824643807 | 279.254 MiB | 277.470 MiB | 49.042 MiB | 48.284 MiB | | INTEGRAL | 10000 | 256 | 1.718 ms | 1.732 ms | 312407306091 | 309935794547 | 13.752 MiB | 13.752 MiB | 2.956 MiB | 2.956 MiB | | INTEGRAL | 100000 | 256 | 5.726 ms | 5.818 ms | 93765292338 | 92275580643 | 114.366 MiB | 114.366 MiB | 20.743 MiB | 20.743 MiB | | INTEGRAL | 500000 | 256 | 25.179 ms | 22.159 ms | 21322289603 | 24228371776 | 572.905 MiB | 561.786 MiB | 103.796 MiB | 97.677 MiB | | INTEGRAL | 1000000 | 256 | 48.259 ms | 42.428 ms | 11124725758 | 12653746472 | 1.117 GiB | 1.095 GiB | 206.155 MiB | 193.886 MiB | | INTEGRAL | 10000 | 512 | 2.741 ms | 2.758 ms | 195853280055 | 194632437549 | 27.508 MiB | 27.508 MiB | 5.918 MiB | 5.918 MiB | | INTEGRAL | 100000 | 512 | 11.197 ms | 10.600 ms | 47945685016 | 50646524148 | 235.910 MiB | 228.755 MiB | 44.559 MiB | 41.510 MiB | | INTEGRAL | 500000 | 512 | 54.929 ms | 43.554 ms | 9773962645 | 12326557981 | 1.146 GiB | 1.097 GiB | 221.266 MiB | 195.384 MiB | | INTEGRAL | 1000000 | 512 | 103.779 ms | 82.403 ms | 5173195193 | 6515218035 | 2.288 GiB | 2.190 GiB | 442.101 MiB | 387.861 MiB | | INTEGRAL | 10000 | 1024 | 5.210 ms | 5.405 ms | 103040438112 | 99319591295 | 54.937 MiB | 54.937 MiB | 11.829 MiB | 11.829 MiB | | INTEGRAL | 100000 | 1024 | 26.891 ms | 20.194 ms | 19964357393 | 26585391032 | 498.410 MiB | 456.756 MiB | 99.962 MiB | 82.939 MiB | | INTEGRAL | 500000 | 1024 | 135.404 ms | 84.676 ms | 3964957208 | 6340314329 | 2.434 GiB | 2.191 GiB | 500.554 MiB | 390.418 MiB | | INTEGRAL | 1000000 | 1024 | 256.033 ms | 162.217 ms | 2096879057 | 3309593393 | 4.869 GiB | 4.372 GiB | 1001.573 MiB | 775.040 MiB | | FLOAT | 10000 | 64 | 962.219 us | 951.565 us | 557950915640 | 564197923891 | 5.275 MiB | 5.275 MiB | 1012.101 KiB | 1012.101 KiB | | FLOAT | 100000 | 64 | 2.032 ms | 2.032 ms | 264218700681 | 264250413360 | 45.321 MiB | 45.321 MiB | 6.316 MiB | 6.316 MiB | | FLOAT | 500000 | 64 | 6.660 ms | 6.693 ms | 80611279094 | 80219014175 | 224.129 MiB | 222.946 MiB | 29.685 MiB | 29.044 MiB | | FLOAT | 1000000 | 64 | 13.560 ms | 13.758 ms | 39591771965 | 39023315442 | 447.103 MiB | 445.007 MiB | 58.762 MiB | 57.482 MiB | | FLOAT | 10000 | 256 | 1.808 ms | 1.825 ms | 297020886609 | 294226222306 | 21.109 MiB | 21.109 MiB | 3.968 MiB | 3.968 MiB | | FLOAT | 100000 | 256 | 6.921 ms | 6.307 ms | 77571490752 | 85116522574 | 185.578 MiB | 181.271 MiB | 27.393 MiB | 25.256 MiB | | FLOAT | 500000 | 256 | 30.064 ms | 25.955 ms | 17857874786 | 20684696586 | 914.366 MiB | 891.787 MiB | 128.981 MiB | 116.186 MiB | | FLOAT | 1000000 | 256 | 59.189 ms | 48.592 ms | 9070460126 | 11048464794 | 1.787 GiB | 1.738 GiB | 258.075 MiB | 229.920 MiB | | FLOAT | 10000 | 512 | 2.998 ms | 3.006 ms | 179078195058 | 178594968077 | 42.222 MiB | 42.222 MiB | 7.941 MiB | 7.941 MiB | | FLOAT | 100000 | 512 | 14.160 ms | 12.314 ms | 37915291403 | 43597041127 | 376.553 MiB | 362.567 MiB | 60.136 MiB | 50.537 MiB | | FLOAT | 500000 | 512 | 69.524 ms | 50.251 ms | 7722076774 | 10683715204 | 1.826 GiB | 1.742 GiB | 292.552 MiB | 232.393 MiB | | FLOAT | 1000000 | 512 | 130.729 ms | 95.458 ms | 4106742786 | 5624164002 | 3.647 GiB | 3.477 GiB | 581.180 MiB | 459.927 MiB | | FLOAT | 10000 | 1024 | 6.351 ms | 6.492 ms | 84532884515 | 82693769317 | 84.452 MiB | 84.452 MiB | 15.893 MiB | 15.893 MiB | | FLOAT | 100000 | 1024 | 36.898 ms | 26.302 ms | 14550146722 | 20411596018 | 778.441 MiB | 725.125 MiB | 136.809 MiB | 101.066 MiB | | FLOAT | 500000 | 1024 | 166.699 ms | 98.340 ms | 3220600409 | 5459311820 | 3.802 GiB | 3.484 GiB | 685.702 MiB | 464.775 MiB | | FLOAT | 1000000 | 1024 | 339.687 ms | 188.463 ms | 1580487011 | 2848673918 | 7.606 GiB | 6.953 GiB | 1.340 GiB | 919.840 MiB | | DECIMAL | 10000 | 64 | 1.076 ms | 1.092 ms | 498752693210 | 491676757508 | 7.485 MiB | 7.485 MiB | 1.216 MiB | 1.216 MiB | | DECIMAL | 100000 | 64 | 2.166 ms | 2.172 ms | 247840684988 | 247198078197 | 65.498 MiB | 65.498 MiB | 6.658 MiB | 6.658 MiB | | DECIMAL | 500000 | 64 | 7.421 ms | 7.058 ms | 72343289850 | 76066836305 | 325.515 MiB | 322.466 MiB | 31.349 MiB | 29.384 MiB | | DECIMAL | 1000000 | 64 | 15.239 ms | 14.020 ms | 35230516583 | 38291860266 | 649.547 MiB | 643.714 MiB | 61.759 MiB | 57.826 MiB | | DECIMAL | 10000 | 256 | 1.989 ms | 1.989 ms | 269930562597 | 269886680781 | 30.119 MiB | 30.119 MiB | 4.896 MiB | 4.896 MiB | | DECIMAL | 100000 | 256 | 7.839 ms | 6.966 ms | 68483613468 | 77073587059 | 269.638 MiB | 263.547 MiB | 30.588 MiB | 26.664 MiB | | DECIMAL | 500000 | 256 | 35.199 ms | 26.893 ms | 15252335676 | 19963411264 | 1.312 GiB | 1.267 GiB | 150.948 MiB | 117.601 MiB | | DECIMAL | 1000000 | 256 | 72.584 ms | 50.944 ms | 7396511691 | 10538553316 | 2.622 GiB | 2.529 GiB | 301.231 MiB | 231.353 MiB | | DECIMAL | 10000 | 512 | 3.612 ms | 3.595 ms | 148642296188 | 149335059500 | 60.283 MiB | 60.283 MiB | 9.801 MiB | 9.801 MiB | | DECIMAL | 100000 | 512 | 19.820 ms | 14.084 ms | 27087819156 | 38119174003 | 562.417 MiB | 527.494 MiB | 75.263 MiB | 53.349 MiB | | DECIMAL | 500000 | 512 | 94.913 ms | 51.910 ms | 5656452419 | 10342308581 | 2.747 GiB | 2.536 GiB | 377.112 MiB | 235.187 MiB | | DECIMAL | 1000000 | 512 | 180.513 ms | 98.562 ms | 2974131976 | 5447057883 | 5.494 GiB | 5.063 GiB | 754.738 MiB | 462.785 MiB | | DECIMAL | 10000 | 1024 | 7.667 ms | 6.777 ms | 70025338013 | 79218913933 | 120.656 MiB | 120.656 MiB | 19.616 MiB | 19.616 MiB | | DECIMAL | 100000 | 1024 | 61.182 ms | 26.946 ms | 8775038947 | 19923803470 | 1.184 GiB | 1.031 GiB | 201.928 MiB | 106.705 MiB | | DECIMAL | 500000 | 1024 | 261.218 ms | 102.314 ms | 2055261558 | 5247292283 | 5.921 GiB | 5.076 GiB | 1012.826 MiB | 470.402 MiB | | DECIMAL | 1000000 | 1024 | 513.386 ms | 196.347 ms | 1045744543 | 2734301880 | 11.843 GiB | 10.133 GiB | 1.980 GiB | 925.576 MiB | | TIMESTAMP | 10000 | 64 | 1.014 ms | 1.016 ms | 529606978079 | 528414399822 | 6.079 MiB | 6.079 MiB | 1.068 MiB | 1.068 MiB | | TIMESTAMP | 100000 | 64 | 2.057 ms | 2.053 ms | 261019684779 | 261455248599 | 52.688 MiB | 52.688 MiB | 6.436 MiB | 6.436 MiB | | TIMESTAMP | 500000 | 64 | 6.950 ms | 6.761 ms | 77245644716 | 79410211533 | 260.606 MiB | 259.304 MiB | 29.924 MiB | 29.164 MiB | | TIMESTAMP | 1000000 | 64 | 14.506 ms | 13.832 ms | 37010291008 | 38813599633 | 521.240 MiB | 517.604 MiB | 59.878 MiB | 57.601 MiB | | TIMESTAMP | 10000 | 256 | 1.878 ms | 1.889 ms | 285887176743 | 284275145551 | 24.328 MiB | 24.328 MiB | 4.290 MiB | 4.290 MiB | | TIMESTAMP | 100000 | 256 | 7.198 ms | 6.458 ms | 74586920018 | 83128450019 | 215.854 MiB | 210.739 MiB | 28.681 MiB | 25.734 MiB | | TIMESTAMP | 500000 | 256 | 34.185 ms | 26.654 ms | 15705060785 | 20142331826 | 1.044 GiB | 1.013 GiB | 137.016 MiB | 116.663 MiB | | TIMESTAMP | 1000000 | 256 | 66.420 ms | 49.599 ms | 8083007343 | 10824295857 | 2.085 GiB | 2.022 GiB | 272.580 MiB | 230.395 MiB | | TIMESTAMP | 10000 | 512 | 3.143 ms | 3.150 ms | 170821086658 | 170446277893 | 48.702 MiB | 48.702 MiB | 8.591 MiB | 8.591 MiB | | TIMESTAMP | 100000 | 512 | 17.652 ms | 12.615 ms | 30413872283 | 42557024194 | 440.115 MiB | 421.891 MiB | 63.197 MiB | 51.502 MiB | | TIMESTAMP | 500000 | 512 | 75.454 ms | 50.955 ms | 7115233856 | 10536117334 | 2.146 GiB | 2.028 GiB | 315.073 MiB | 233.355 MiB | | TIMESTAMP | 1000000 | 512 | 140.692 ms | 95.964 ms | 3815935506 | 5594485106 | 4.285 GiB | 4.048 GiB | 627.348 MiB | 460.885 MiB | | TIMESTAMP | 10000 | 1024 | 6.436 ms | 6.975 ms | 83411903593 | 76971777095 | 97.454 MiB | 97.454 MiB | 17.196 MiB | 17.196 MiB | | TIMESTAMP | 100000 | 1024 | 45.659 ms | 26.728 ms | 11758159876 | 20086145129 | 936.005 MiB | 844.159 MiB | 159.908 MiB | 103.000 MiB | | TIMESTAMP | 500000 | 1024 | 199.636 ms | 99.231 ms | 2689242353 | 5410303529 | 4.557 GiB | 4.057 GiB | 794.728 MiB | 466.703 MiB | | TIMESTAMP | 1000000 | 1024 | 372.691 ms | 192.598 ms | 1440523696 | 2787517681 | 9.104 GiB | 8.099 GiB | 1.551 GiB | 921.760 MiB | | DURATION | 10000 | 64 | 986.208 us | 989.153 us | 544379023579 | 542758221495 | 6.417 MiB | 6.417 MiB | 932.501 KiB | 932.501 KiB | | DURATION | 100000 | 64 | 2.222 ms | 2.018 ms | 241594183626 | 266034888500 | 57.291 MiB | 57.291 MiB | 6.079 MiB | 6.079 MiB | | DURATION | 500000 | 64 | 6.642 ms | 6.673 ms | 80830328889 | 80453377113 | 284.029 MiB | 283.224 MiB | 28.819 MiB | 28.288 MiB | | DURATION | 1000000 | 64 | 13.150 ms | 13.488 ms | 40828039129 | 39804805295 | 567.280 MiB | 565.669 MiB | 57.137 MiB | 56.075 MiB | | DURATION | 10000 | 256 | 1.805 ms | 1.815 ms | 297459887040 | 295856879191 | 25.686 MiB | 25.686 MiB | 3.665 MiB | 3.665 MiB | | DURATION | 100000 | 256 | 6.839 ms | 6.270 ms | 78502421937 | 85630914910 | 232.874 MiB | 229.165 MiB | 25.863 MiB | 24.323 MiB | | DURATION | 500000 | 256 | 29.886 ms | 26.234 ms | 17964080662 | 20464503730 | 1.125 GiB | 1.106 GiB | 123.885 MiB | 113.179 MiB | | DURATION | 1000000 | 256 | 58.290 ms | 48.418 ms | 9210348188 | 11088351436 | 2.250 GiB | 2.210 GiB | 247.272 MiB | 224.312 MiB | | DURATION | 10000 | 512 | 3.035 ms | 2.964 ms | 176885037888 | 181108374773 | 51.383 MiB | 51.383 MiB | 7.342 MiB | 7.342 MiB | | DURATION | 100000 | 512 | 14.492 ms | 12.136 ms | 37044853523 | 44237579412 | 474.355 MiB | 458.371 MiB | 55.996 MiB | 48.689 MiB | | DURATION | 500000 | 512 | 70.131 ms | 51.095 ms | 7655286246 | 10507294503 | 2.299 GiB | 2.213 GiB | 271.064 MiB | 226.438 MiB | | DURATION | 1000000 | 512 | 132.495 ms | 95.019 ms | 4051999205 | 5650150759 | 4.593 GiB | 4.419 GiB | 541.495 MiB | 448.815 MiB | | DURATION | 10000 | 1024 | 6.576 ms | 6.318 ms | 81638807422 | 84977253627 | 102.782 MiB | 102.782 MiB | 14.701 MiB | 14.701 MiB | | DURATION | 100000 | 1024 | 38.001 ms | 26.011 ms | 14127627316 | 20640219375 | 964.471 MiB | 916.755 MiB | 127.532 MiB | 97.394 MiB | | DURATION | 500000 | 1024 | 159.928 ms | 98.126 ms | 3356945213 | 5471258270 | 4.711 GiB | 4.426 GiB | 639.050 MiB | 452.925 MiB | | DURATION | 1000000 | 1024 | 305.818 ms | 188.647 ms | 1755524869 | 2845895428 | 9.422 GiB | 8.839 GiB | 1.249 GiB | 897.737 MiB | | STRING | 10000 | 64 | 2.241 ms | 2.244 ms | 239611491431 | 239240518530 | 15.926 MiB | 15.926 MiB | 2.075 MiB | 2.075 MiB | | STRING | 100000 | 64 | 4.862 ms | 4.822 ms | 110419679907 | 111346705245 | 132.646 MiB | 132.646 MiB | 8.087 MiB | 8.087 MiB | | STRING | 500000 | 64 | 20.498 ms | 17.812 ms | 26191957819 | 30140554720 | 664.294 MiB | 645.028 MiB | 40.456 MiB | 30.817 MiB | | STRING | 1000000 | 64 | 37.773 ms | 34.985 ms | 14213079575 | 15345709268 | 1.298 GiB | 1.255 GiB | 80.941 MiB | 59.259 MiB | | STRING | 10000 | 256 | 4.125 ms | 4.171 ms | 130163506067 | 128706550148 | 63.789 MiB | 63.789 MiB | 8.319 MiB | 8.319 MiB | | STRING | 100000 | 256 | 22.074 ms | 17.799 ms | 24321103825 | 30162947098 | 584.754 MiB | 530.912 MiB | 58.602 MiB | 32.330 MiB | | STRING | 500000 | 256 | 93.278 ms | 66.770 ms | 5755572906 | 8040584271 | 2.857 GiB | 2.521 GiB | 294.130 MiB | 123.271 MiB | | STRING | 1000000 | 256 | 190.999 ms | 122.359 ms | 2810851154 | 4387682165 | 5.715 GiB | 5.023 GiB | 588.586 MiB | 237.018 MiB | | STRING | 10000 | 512 | 7.520 ms | 8.010 ms | 71390390607 | 67021971176 | 127.538 MiB | 127.538 MiB | 16.634 MiB | 16.634 MiB | | STRING | 100000 | 512 | 51.666 ms | 32.251 ms | 10391219810 | 16646741143 | 1.259 GiB | 1.037 GiB | 173.940 MiB | 64.682 MiB | | STRING | 500000 | 512 | 251.723 ms | 125.963 ms | 2132782858 | 4262141577 | 6.300 GiB | 5.040 GiB | 873.437 MiB | 246.559 MiB | | STRING | 1000000 | 512 | 477.668 ms | 244.912 ms | 1123940871 | 2192101011 | 12.602 GiB | 10.044 GiB | 1.707 GiB | 474.121 MiB | | STRING | 10000 | 1024 | 17.184 ms | 16.128 ms | 31242201518 | 33288874029 | 276.395 MiB | 254.971 MiB | 40.126 MiB | 33.243 MiB | | STRING | 100000 | 1024 | 132.094 ms | 63.304 ms | 4064323158 | 8480799642 | 2.721 GiB | 2.073 GiB | 414.092 MiB | 129.316 MiB | | STRING | 500000 | 1024 | 608.283 ms | 251.026 ms | 882600977 | 2138709222 | 13.618 GiB | 10.076 GiB | 2.028 GiB | 493.067 MiB | | STRING | 1000000 | 1024 | 1.249 s | 485.734 ms | 429750505 | 1105276473 | 27.239 GiB | 20.079 GiB | 4.059 GiB | 948.185 MiB | ``` Authors: - Muhammad Haseeb (https://github.com/mhaseeb123) Approvers: - Nghia Truong (https://github.com/ttnghia) - Vukasin Milovanovic (https://github.com/vuule) - Bradley Dice (https://github.com/bdice) - Charles Blackmon-Luca (https://github.com/charlesbluca) URL: https://github.com/rapidsai/cudf/pull/16750 --- cpp/include/cudf/io/parquet.hpp | 5 +++-- cpp/src/io/parquet/writer_impl.cu | 10 ++++++++-- python/cudf/cudf/_lib/parquet.pyx | 16 ++++++++-------- python/cudf/cudf/core/dataframe.py | 2 +- python/cudf/cudf/io/parquet.py | 8 ++++---- python/cudf/cudf/utils/ioutils.py | 12 ++++-------- python/dask_cudf/dask_cudf/io/parquet.py | 7 ++----- 7 files changed, 30 insertions(+), 30 deletions(-) diff --git a/cpp/include/cudf/io/parquet.hpp b/cpp/include/cudf/io/parquet.hpp index ed7b2ac0850..ee03a382bec 100644 --- a/cpp/include/cudf/io/parquet.hpp +++ b/cpp/include/cudf/io/parquet.hpp @@ -39,8 +39,9 @@ namespace io { * @file */ -constexpr size_t default_row_group_size_bytes = 128 * 1024 * 1024; ///< 128MB per row group -constexpr size_type default_row_group_size_rows = 1000000; ///< 1 million rows per row group +constexpr size_t default_row_group_size_bytes = + std::numeric_limits::max(); ///< Infinite bytes per row group +constexpr size_type default_row_group_size_rows = 1'000'000; ///< 1 million rows per row group constexpr size_t default_max_page_size_bytes = 512 * 1024; ///< 512KB per page constexpr size_type default_max_page_size_rows = 20000; ///< 20k rows per page constexpr int32_t default_column_index_truncate_length = 64; ///< truncate to 64 bytes diff --git a/cpp/src/io/parquet/writer_impl.cu b/cpp/src/io/parquet/writer_impl.cu index 81fd4ab9f82..ec05f35d405 100644 --- a/cpp/src/io/parquet/writer_impl.cu +++ b/cpp/src/io/parquet/writer_impl.cu @@ -1819,8 +1819,14 @@ auto convert_table_to_parquet_data(table_input_metadata& table_meta, auto const table_size = std::reduce(column_sizes.begin(), column_sizes.end()); auto const avg_row_len = util::div_rounding_up_safe(table_size, input.num_rows()); if (avg_row_len > 0) { - auto const rg_frag_size = util::div_rounding_up_safe(max_row_group_size, avg_row_len); - max_page_fragment_size = std::min(rg_frag_size, max_page_fragment_size); + // Ensure `rg_frag_size` is not bigger than size_type::max for default max_row_group_size + // value (=uint64::max) to avoid a sign overflow when comparing + auto const rg_frag_size = + std::min(std::numeric_limits::max(), + util::div_rounding_up_safe(max_row_group_size, avg_row_len)); + // Safe comparison as rg_frag_size fits in size_type + max_page_fragment_size = + std::min(static_cast(rg_frag_size), max_page_fragment_size); } // dividing page size by average row length will tend to overshoot the desired diff --git a/python/cudf/cudf/_lib/parquet.pyx b/python/cudf/cudf/_lib/parquet.pyx index a0155671a26..e6c9d60b05b 100644 --- a/python/cudf/cudf/_lib/parquet.pyx +++ b/python/cudf/cudf/_lib/parquet.pyx @@ -438,7 +438,7 @@ def write_parquet( object statistics="ROWGROUP", object metadata_file_path=None, object int96_timestamps=False, - object row_group_size_bytes=_ROW_GROUP_SIZE_BYTES_DEFAULT, + object row_group_size_bytes=None, object row_group_size_rows=None, object max_page_size_bytes=None, object max_page_size_rows=None, @@ -616,9 +616,9 @@ cdef class ParquetWriter: Name of the compression to use. Use ``None`` for no compression. statistics : {'ROWGROUP', 'PAGE', 'COLUMN', 'NONE'}, default 'ROWGROUP' Level at which column statistics should be included in file. - row_group_size_bytes: int, default 134217728 + row_group_size_bytes: int, default ``uint64 max`` Maximum size of each stripe of the output. - By default, 134217728 (128MB) will be used. + By default, a virtually infinite size equal to ``uint64 max`` will be used. row_group_size_rows: int, default 1000000 Maximum number of rows of each stripe of the output. By default, 1000000 (10^6 rows) will be used. @@ -661,11 +661,11 @@ cdef class ParquetWriter: def __cinit__(self, object filepath_or_buffer, object index=None, object compression="snappy", str statistics="ROWGROUP", - int row_group_size_bytes=_ROW_GROUP_SIZE_BYTES_DEFAULT, - int row_group_size_rows=1000000, - int max_page_size_bytes=524288, - int max_page_size_rows=20000, - int max_dictionary_size=1048576, + size_t row_group_size_bytes=_ROW_GROUP_SIZE_BYTES_DEFAULT, + size_type row_group_size_rows=1000000, + size_t max_page_size_bytes=524288, + size_type max_page_size_rows=20000, + size_t max_dictionary_size=1048576, bool use_dictionary=True, bool store_schema=False): filepaths_or_buffers = ( diff --git a/python/cudf/cudf/core/dataframe.py b/python/cudf/cudf/core/dataframe.py index 58a16a6d504..d73ad8225ca 100644 --- a/python/cudf/cudf/core/dataframe.py +++ b/python/cudf/cudf/core/dataframe.py @@ -6840,7 +6840,7 @@ def to_parquet( statistics="ROWGROUP", metadata_file_path=None, int96_timestamps=False, - row_group_size_bytes=ioutils._ROW_GROUP_SIZE_BYTES_DEFAULT, + row_group_size_bytes=None, row_group_size_rows=None, max_page_size_bytes=None, max_page_size_rows=None, diff --git a/python/cudf/cudf/io/parquet.py b/python/cudf/cudf/io/parquet.py index 62be7378e9e..ce99f98b559 100644 --- a/python/cudf/cudf/io/parquet.py +++ b/python/cudf/cudf/io/parquet.py @@ -64,7 +64,7 @@ def _write_parquet( statistics="ROWGROUP", metadata_file_path=None, int96_timestamps=False, - row_group_size_bytes=ioutils._ROW_GROUP_SIZE_BYTES_DEFAULT, + row_group_size_bytes=None, row_group_size_rows=None, max_page_size_bytes=None, max_page_size_rows=None, @@ -149,7 +149,7 @@ def write_to_dataset( return_metadata=False, statistics="ROWGROUP", int96_timestamps=False, - row_group_size_bytes=ioutils._ROW_GROUP_SIZE_BYTES_DEFAULT, + row_group_size_bytes=None, row_group_size_rows=None, max_page_size_bytes=None, max_page_size_rows=None, @@ -205,7 +205,7 @@ def write_to_dataset( If ``False``, timestamps will not be altered. row_group_size_bytes: integer or None, default None Maximum size of each stripe of the output. - If None, 134217728 (128MB) will be used. + If None, no limit on row group stripe size will be used. row_group_size_rows: integer or None, default None Maximum number of rows of each stripe of the output. If None, 1000000 will be used. @@ -980,7 +980,7 @@ def to_parquet( statistics="ROWGROUP", metadata_file_path=None, int96_timestamps=False, - row_group_size_bytes=ioutils._ROW_GROUP_SIZE_BYTES_DEFAULT, + row_group_size_bytes=None, row_group_size_rows=None, max_page_size_bytes=None, max_page_size_rows=None, diff --git a/python/cudf/cudf/utils/ioutils.py b/python/cudf/cudf/utils/ioutils.py index 1627107b57d..1180da321e6 100644 --- a/python/cudf/cudf/utils/ioutils.py +++ b/python/cudf/cudf/utils/ioutils.py @@ -27,7 +27,7 @@ fsspec_parquet = None _BYTES_PER_THREAD_DEFAULT = 256 * 1024 * 1024 -_ROW_GROUP_SIZE_BYTES_DEFAULT = 128 * 1024 * 1024 +_ROW_GROUP_SIZE_BYTES_DEFAULT = np.iinfo(np.uint64).max _docstring_remote_sources = """ - cuDF supports local and remote data stores. See configuration details for @@ -275,10 +275,9 @@ timestamp[us] to the int96 format, which is the number of Julian days and the number of nanoseconds since midnight of 1970-01-01. If ``False``, timestamps will not be altered. -row_group_size_bytes: integer, default {row_group_size_bytes_val} +row_group_size_bytes: integer, default None Maximum size of each stripe of the output. - If None, {row_group_size_bytes_val} - ({row_group_size_bytes_val_in_mb} MB) will be used. + If None, no limit on row group stripe size will be used. row_group_size_rows: integer or None, default None Maximum number of rows of each stripe of the output. If None, 1000000 will be used. @@ -346,10 +345,7 @@ See Also -------- cudf.read_parquet -""".format( - row_group_size_bytes_val=_ROW_GROUP_SIZE_BYTES_DEFAULT, - row_group_size_bytes_val_in_mb=_ROW_GROUP_SIZE_BYTES_DEFAULT / 1024 / 1024, -) +""" doc_to_parquet = docfmt_partial(docstring=_docstring_to_parquet) _docstring_merge_parquet_filemetadata = """ diff --git a/python/dask_cudf/dask_cudf/io/parquet.py b/python/dask_cudf/dask_cudf/io/parquet.py index e793d4381d1..a781b8242fe 100644 --- a/python/dask_cudf/dask_cudf/io/parquet.py +++ b/python/dask_cudf/dask_cudf/io/parquet.py @@ -23,7 +23,6 @@ from cudf.io import write_to_dataset from cudf.io.parquet import _apply_post_filters, _normalize_filters from cudf.utils.dtypes import cudf_dtype_from_pa_type -from cudf.utils.ioutils import _ROW_GROUP_SIZE_BYTES_DEFAULT class CudfEngine(ArrowDatasetEngine): @@ -341,9 +340,7 @@ def write_partition( return_metadata=return_metadata, statistics=kwargs.get("statistics", "ROWGROUP"), int96_timestamps=kwargs.get("int96_timestamps", False), - row_group_size_bytes=kwargs.get( - "row_group_size_bytes", _ROW_GROUP_SIZE_BYTES_DEFAULT - ), + row_group_size_bytes=kwargs.get("row_group_size_bytes", None), row_group_size_rows=kwargs.get("row_group_size_rows", None), max_page_size_bytes=kwargs.get("max_page_size_bytes", None), max_page_size_rows=kwargs.get("max_page_size_rows", None), @@ -365,7 +362,7 @@ def write_partition( statistics=kwargs.get("statistics", "ROWGROUP"), int96_timestamps=kwargs.get("int96_timestamps", False), row_group_size_bytes=kwargs.get( - "row_group_size_bytes", _ROW_GROUP_SIZE_BYTES_DEFAULT + "row_group_size_bytes", None ), row_group_size_rows=kwargs.get( "row_group_size_rows", None From e68f55c98f257bdeedeb31e68c9737264bd0b393 Mon Sep 17 00:00:00 2001 From: Srinivas Yadav <43375352+srinivasyadav18@users.noreply.github.com> Date: Wed, 18 Sep 2024 12:12:23 -0500 Subject: [PATCH 14/15] Refactor mixed_semi_join using cuco::static_set (#16230) This PR refactors `mixed_semi_join` by replacing **cuco** legacy `static_map` with latest `static_set`. Contributes to #12261. Authors: - Srinivas Yadav (https://github.com/srinivasyadav18) - Muhammad Haseeb (https://github.com/mhaseeb123) Approvers: - Yunsong Wang (https://github.com/PointKernel) - Nghia Truong (https://github.com/ttnghia) URL: https://github.com/rapidsai/cudf/pull/16230 --- cpp/src/join/join_common_utils.hpp | 6 -- cpp/src/join/mixed_join_common_utils.cuh | 33 +++++++++ cpp/src/join/mixed_join_kernels_semi.cu | 35 ++++----- cpp/src/join/mixed_join_kernels_semi.cuh | 6 +- cpp/src/join/mixed_join_semi.cu | 90 +++++++----------------- cpp/tests/join/mixed_join_tests.cu | 30 ++++++++ 6 files changed, 109 insertions(+), 91 deletions(-) diff --git a/cpp/src/join/join_common_utils.hpp b/cpp/src/join/join_common_utils.hpp index 86402a0e7de..573101cefd9 100644 --- a/cpp/src/join/join_common_utils.hpp +++ b/cpp/src/join/join_common_utils.hpp @@ -22,7 +22,6 @@ #include #include -#include #include #include @@ -51,11 +50,6 @@ using mixed_multimap_type = cudf::detail::cuco_allocator, cuco::legacy::double_hashing<1, hash_type, hash_type>>; -using semi_map_type = cuco::legacy::static_map>; - using row_hash_legacy = cudf::row_hasher; diff --git a/cpp/src/join/mixed_join_common_utils.cuh b/cpp/src/join/mixed_join_common_utils.cuh index 19701816867..89c13285cfe 100644 --- a/cpp/src/join/mixed_join_common_utils.cuh +++ b/cpp/src/join/mixed_join_common_utils.cuh @@ -25,6 +25,7 @@ #include #include +#include namespace cudf { namespace detail { @@ -160,6 +161,38 @@ struct pair_expression_equality : public expression_equality { } }; +/** + * @brief Equality comparator that composes two row_equality comparators. + */ +struct double_row_equality_comparator { + row_equality const equality_comparator; + row_equality const conditional_comparator; + + __device__ bool operator()(size_type lhs_row_index, size_type rhs_row_index) const noexcept + { + using experimental::row::lhs_index_type; + using experimental::row::rhs_index_type; + + return equality_comparator(lhs_index_type{lhs_row_index}, rhs_index_type{rhs_row_index}) && + conditional_comparator(lhs_index_type{lhs_row_index}, rhs_index_type{rhs_row_index}); + } +}; + +// A CUDA Cooperative Group of 4 threads for the hash set. +auto constexpr DEFAULT_MIXED_JOIN_CG_SIZE = 4; + +// The hash set type used by mixed_semi_join with the build_table. +using hash_set_type = cuco::static_set, + cuda::thread_scope_device, + double_row_equality_comparator, + cuco::linear_probing, + cudf::detail::cuco_allocator, + cuco::storage<1>>; + +// The hash_set_ref_type used by mixed_semi_join kerenels for probing. +using hash_set_ref_type = hash_set_type::ref_type; + } // namespace detail } // namespace cudf diff --git a/cpp/src/join/mixed_join_kernels_semi.cu b/cpp/src/join/mixed_join_kernels_semi.cu index 7459ac3e99c..f2c5ff13638 100644 --- a/cpp/src/join/mixed_join_kernels_semi.cu +++ b/cpp/src/join/mixed_join_kernels_semi.cu @@ -38,12 +38,16 @@ CUDF_KERNEL void __launch_bounds__(block_size) table_device_view right_table, table_device_view probe, table_device_view build, - row_hash const hash_probe, row_equality const equality_probe, - cudf::detail::semi_map_type::device_view hash_table_view, + hash_set_ref_type set_ref, cudf::device_span left_table_keep_mask, cudf::ast::detail::expression_device_view device_expression_data) { + auto constexpr cg_size = hash_set_ref_type::cg_size; + + auto const tile = + cooperative_groups::tiled_partition(cooperative_groups::this_thread_block()); + // Normally the casting of a shared memory array is used to create multiple // arrays of different types from the shared memory buffer, but here it is // used to circumvent conflicts between arrays of different types between @@ -52,24 +56,24 @@ CUDF_KERNEL void __launch_bounds__(block_size) cudf::ast::detail::IntermediateDataType* intermediate_storage = reinterpret_cast*>(raw_intermediate_storage); auto thread_intermediate_storage = - &intermediate_storage[threadIdx.x * device_expression_data.num_intermediates]; - - cudf::size_type const left_num_rows = left_table.num_rows(); - cudf::size_type const right_num_rows = right_table.num_rows(); - auto const outer_num_rows = left_num_rows; + &intermediate_storage[tile.meta_group_rank() * device_expression_data.num_intermediates]; - cudf::size_type outer_row_index = threadIdx.x + blockIdx.x * block_size; + cudf::size_type const outer_num_rows = left_table.num_rows(); + auto const outer_row_index = cudf::detail::grid_1d::global_thread_id() / cg_size; auto evaluator = cudf::ast::detail::expression_evaluator( left_table, right_table, device_expression_data); if (outer_row_index < outer_num_rows) { + // Make sure to swap_tables here as hash_set will use probe table as the left one. + auto constexpr swap_tables = true; // Figure out the number of elements for this key. auto equality = single_expression_equality{ - evaluator, thread_intermediate_storage, false, equality_probe}; + evaluator, thread_intermediate_storage, swap_tables, equality_probe}; - left_table_keep_mask[outer_row_index] = - hash_table_view.contains(outer_row_index, hash_probe, equality); + auto const set_ref_equality = set_ref.with_key_eq(equality); + auto const result = set_ref_equality.contains(tile, outer_row_index); + if (tile.thread_rank() == 0) left_table_keep_mask[outer_row_index] = result; } } @@ -78,9 +82,8 @@ void launch_mixed_join_semi(bool has_nulls, table_device_view right_table, table_device_view probe, table_device_view build, - row_hash const hash_probe, row_equality const equality_probe, - cudf::detail::semi_map_type::device_view hash_table_view, + hash_set_ref_type set_ref, cudf::device_span left_table_keep_mask, cudf::ast::detail::expression_device_view device_expression_data, detail::grid_1d const config, @@ -94,9 +97,8 @@ void launch_mixed_join_semi(bool has_nulls, right_table, probe, build, - hash_probe, equality_probe, - hash_table_view, + set_ref, left_table_keep_mask, device_expression_data); } else { @@ -106,9 +108,8 @@ void launch_mixed_join_semi(bool has_nulls, right_table, probe, build, - hash_probe, equality_probe, - hash_table_view, + set_ref, left_table_keep_mask, device_expression_data); } diff --git a/cpp/src/join/mixed_join_kernels_semi.cuh b/cpp/src/join/mixed_join_kernels_semi.cuh index 43714ffb36a..b08298e64e4 100644 --- a/cpp/src/join/mixed_join_kernels_semi.cuh +++ b/cpp/src/join/mixed_join_kernels_semi.cuh @@ -45,9 +45,8 @@ namespace detail { * @param[in] right_table The right table * @param[in] probe The table with which to probe the hash table for matches. * @param[in] build The table with which the hash table was built. - * @param[in] hash_probe The hasher used for the probe table. * @param[in] equality_probe The equality comparator used when probing the hash table. - * @param[in] hash_table_view The hash table built from `build`. + * @param[in] set_ref The hash table device view built from `build`. * @param[out] left_table_keep_mask The result of the join operation with "true" element indicating * the corresponding index from left table is present in output * @param[in] device_expression_data Container of device data required to evaluate the desired @@ -58,9 +57,8 @@ void launch_mixed_join_semi(bool has_nulls, table_device_view right_table, table_device_view probe, table_device_view build, - row_hash const hash_probe, row_equality const equality_probe, - cudf::detail::semi_map_type::device_view hash_table_view, + hash_set_ref_type set_ref, cudf::device_span left_table_keep_mask, cudf::ast::detail::expression_device_view device_expression_data, detail::grid_1d const config, diff --git a/cpp/src/join/mixed_join_semi.cu b/cpp/src/join/mixed_join_semi.cu index cfb785e242c..719b1d47105 100644 --- a/cpp/src/join/mixed_join_semi.cu +++ b/cpp/src/join/mixed_join_semi.cu @@ -46,45 +46,6 @@ namespace cudf { namespace detail { -namespace { -/** - * @brief Device functor to create a pair of hash value and index for a given row. - */ -struct make_pair_function_semi { - __device__ __forceinline__ cudf::detail::pair_type operator()(size_type i) const noexcept - { - // The value is irrelevant since we only ever use the hash map to check for - // membership of a particular row index. - return cuco::make_pair(static_cast(i), 0); - } -}; - -/** - * @brief Equality comparator that composes two row_equality comparators. - */ -class double_row_equality { - public: - double_row_equality(row_equality equality_comparator, row_equality conditional_comparator) - : _equality_comparator{equality_comparator}, _conditional_comparator{conditional_comparator} - { - } - - __device__ bool operator()(size_type lhs_row_index, size_type rhs_row_index) const noexcept - { - using experimental::row::lhs_index_type; - using experimental::row::rhs_index_type; - - return _equality_comparator(lhs_index_type{lhs_row_index}, rhs_index_type{rhs_row_index}) && - _conditional_comparator(lhs_index_type{lhs_row_index}, rhs_index_type{rhs_row_index}); - } - - private: - row_equality _equality_comparator; - row_equality _conditional_comparator; -}; - -} // namespace - std::unique_ptr> mixed_join_semi( table_view const& left_equality, table_view const& right_equality, @@ -96,7 +57,7 @@ std::unique_ptr> mixed_join_semi( rmm::cuda_stream_view stream, rmm::device_async_resource_ref mr) { - CUDF_EXPECTS((join_type != join_kind::INNER_JOIN) && (join_type != join_kind::LEFT_JOIN) && + CUDF_EXPECTS((join_type != join_kind::INNER_JOIN) and (join_type != join_kind::LEFT_JOIN) and (join_type != join_kind::FULL_JOIN), "Inner, left, and full joins should use mixed_join."); @@ -137,7 +98,7 @@ std::unique_ptr> mixed_join_semi( // output column and follow the null-supporting expression evaluation code // path. auto const has_nulls = cudf::nullate::DYNAMIC{ - cudf::has_nulls(left_equality) || cudf::has_nulls(right_equality) || + cudf::has_nulls(left_equality) or cudf::has_nulls(right_equality) or binary_predicate.may_evaluate_null(left_conditional, right_conditional, stream)}; auto const parser = ast::detail::expression_parser{ @@ -156,27 +117,20 @@ std::unique_ptr> mixed_join_semi( auto right_conditional_view = table_device_view::create(right_conditional, stream); auto const preprocessed_build = - experimental::row::equality::preprocessed_table::create(build, stream); + cudf::experimental::row::equality::preprocessed_table::create(build, stream); auto const preprocessed_probe = - experimental::row::equality::preprocessed_table::create(probe, stream); + cudf::experimental::row::equality::preprocessed_table::create(probe, stream); auto const row_comparator = - cudf::experimental::row::equality::two_table_comparator{preprocessed_probe, preprocessed_build}; + cudf::experimental::row::equality::two_table_comparator{preprocessed_build, preprocessed_probe}; auto const equality_probe = row_comparator.equal_to(has_nulls, compare_nulls); - semi_map_type hash_table{ - compute_hash_table_size(build.num_rows()), - cuco::empty_key{std::numeric_limits::max()}, - cuco::empty_value{cudf::detail::JoinNoneValue}, - cudf::detail::cuco_allocator{rmm::mr::polymorphic_allocator{}, stream}, - stream.value()}; - // Create hash table containing all keys found in right table // TODO: To add support for nested columns we will need to flatten in many // places. However, this probably isn't worth adding any time soon since we // won't be able to support AST conditions for those types anyway. auto const build_nulls = cudf::nullate::DYNAMIC{cudf::has_nulls(build)}; auto const row_hash_build = cudf::experimental::row::hash::row_hasher{preprocessed_build}; - auto const hash_build = row_hash_build.device_hasher(build_nulls); + // Since we may see multiple rows that are identical in the equality tables // but differ in the conditional tables, the equality comparator used for // insertion must account for both sets of tables. An alternative solution @@ -191,20 +145,28 @@ std::unique_ptr> mixed_join_semi( auto const equality_build_equality = row_comparator_build.equal_to(build_nulls, compare_nulls); auto const preprocessed_build_condtional = - experimental::row::equality::preprocessed_table::create(right_conditional, stream); + cudf::experimental::row::equality::preprocessed_table::create(right_conditional, stream); auto const row_comparator_conditional_build = cudf::experimental::row::equality::two_table_comparator{preprocessed_build_condtional, preprocessed_build_condtional}; auto const equality_build_conditional = row_comparator_conditional_build.equal_to(build_nulls, compare_nulls); - double_row_equality equality_build{equality_build_equality, equality_build_conditional}; - make_pair_function_semi pair_func_build{}; - auto iter = cudf::detail::make_counting_transform_iterator(0, pair_func_build); + hash_set_type row_set{ + {compute_hash_table_size(build.num_rows())}, + cuco::empty_key{JoinNoneValue}, + {equality_build_equality, equality_build_conditional}, + {row_hash_build.device_hasher(build_nulls)}, + {}, + {}, + cudf::detail::cuco_allocator{rmm::mr::polymorphic_allocator{}, stream}, + {stream.value()}}; + + auto iter = thrust::make_counting_iterator(0); // skip rows that are null here. if ((compare_nulls == null_equality::EQUAL) or (not nullable(build))) { - hash_table.insert(iter, iter + right_num_rows, hash_build, equality_build, stream.value()); + row_set.insert(iter, iter + right_num_rows, stream.value()); } else { thrust::counting_iterator stencil(0); auto const [row_bitmask, _] = @@ -212,18 +174,19 @@ std::unique_ptr> mixed_join_semi( row_is_valid pred{static_cast(row_bitmask.data())}; // insert valid rows - hash_table.insert_if( - iter, iter + right_num_rows, stencil, pred, hash_build, equality_build, stream.value()); + row_set.insert_if(iter, iter + right_num_rows, stencil, pred, stream.value()); } - auto hash_table_view = hash_table.get_device_view(); - detail::grid_1d const config(outer_num_rows, DEFAULT_JOIN_BLOCK_SIZE); - auto const shmem_size_per_block = parser.shmem_per_thread * config.num_threads_per_block; + auto const shmem_size_per_block = + parser.shmem_per_thread * + cuco::detail::int_div_ceil(config.num_threads_per_block, hash_set_type::cg_size); auto const row_hash = cudf::experimental::row::hash::row_hasher{preprocessed_probe}; auto const hash_probe = row_hash.device_hasher(has_nulls); + hash_set_ref_type const row_set_ref = row_set.ref(cuco::contains).with_hash_function(hash_probe); + // Vector used to indicate indices from left/probe table which are present in output auto left_table_keep_mask = rmm::device_uvector(probe.num_rows(), stream); @@ -232,9 +195,8 @@ std::unique_ptr> mixed_join_semi( *right_conditional_view, *probe_view, *build_view, - hash_probe, equality_probe, - hash_table_view, + row_set_ref, cudf::device_span(left_table_keep_mask), parser.device_expression_data, config, diff --git a/cpp/tests/join/mixed_join_tests.cu b/cpp/tests/join/mixed_join_tests.cu index 6c147c8a128..08a0136700d 100644 --- a/cpp/tests/join/mixed_join_tests.cu +++ b/cpp/tests/join/mixed_join_tests.cu @@ -778,6 +778,21 @@ TYPED_TEST(MixedLeftSemiJoinTest, BasicEquality) {1}); } +TYPED_TEST(MixedLeftSemiJoinTest, MixedLeftSemiJoinGatherMap) +{ + auto const col_ref_left_1 = cudf::ast::column_reference(0, cudf::ast::table_reference::LEFT); + auto const col_ref_right_1 = cudf::ast::column_reference(0, cudf::ast::table_reference::RIGHT); + auto left_one_greater_right_one = + cudf::ast::operation(cudf::ast::ast_operator::GREATER, col_ref_left_1, col_ref_right_1); + + this->test({{2, 3, 9, 0, 1, 7, 4, 6, 5, 8}, {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}}, + {{6, 5, 9, 8, 10, 32}, {0, 1, 2, 3, 4, 5}, {7, 8, 9, 0, 1, 2}}, + {0}, + {1}, + left_one_greater_right_one, + {2, 7, 8}); +} + TYPED_TEST(MixedLeftSemiJoinTest, BasicEqualityDuplicates) { this->test({{0, 1, 2, 1}, {3, 4, 5, 6}, {10, 20, 30, 40}}, @@ -900,3 +915,18 @@ TYPED_TEST(MixedLeftAntiJoinTest, AsymmetricLeftLargerEquality) left_zero_eq_right_zero, {0, 1, 3}); } + +TYPED_TEST(MixedLeftAntiJoinTest, MixedLeftAntiJoinGatherMap) +{ + auto const col_ref_left_1 = cudf::ast::column_reference(0, cudf::ast::table_reference::LEFT); + auto const col_ref_right_1 = cudf::ast::column_reference(0, cudf::ast::table_reference::RIGHT); + auto left_one_greater_right_one = + cudf::ast::operation(cudf::ast::ast_operator::GREATER, col_ref_left_1, col_ref_right_1); + + this->test({{2, 3, 9, 0, 1, 7, 4, 6, 5, 8}, {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}}, + {{6, 5, 9, 8, 10, 32}, {0, 1, 2, 3, 4, 5}, {7, 8, 9, 0, 1, 2}}, + {0}, + {1}, + left_one_greater_right_one, + {0, 1, 3, 4, 5, 6, 9}); +} From 42c53247bd3933c83fde18d378902a76d1506c57 Mon Sep 17 00:00:00 2001 From: James Lamb Date: Wed, 18 Sep 2024 14:42:09 -0500 Subject: [PATCH 15/15] Use CI workflow branch 'branch-24.10' again (#16832) All RAPIDS libraries have been updated with Python 3.12 support, so Python 3.12 changes have been merged into `branch-24.10` of `shared-workflows`: https://github.com/rapidsai/shared-workflows/pull/213 This updates GitHub Actions configs here to that branch. --- .github/workflows/build.yaml | 28 +++++------ .github/workflows/pandas-tests.yaml | 2 +- .github/workflows/pr.yaml | 48 +++++++++---------- .../workflows/pr_issue_status_automation.yml | 6 +-- .github/workflows/test.yaml | 24 +++++----- 5 files changed, 54 insertions(+), 54 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index d6d3e3fdd33..b5d17022a3a 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -28,7 +28,7 @@ concurrency: jobs: cpp-build: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-build.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -37,7 +37,7 @@ jobs: python-build: needs: [cpp-build] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-python-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-python-build.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -46,7 +46,7 @@ jobs: upload-conda: needs: [cpp-build, python-build] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-upload-packages.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-upload-packages.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -57,7 +57,7 @@ jobs: if: github.ref_type == 'branch' needs: python-build secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: arch: "amd64" branch: ${{ inputs.branch }} @@ -69,7 +69,7 @@ jobs: sha: ${{ inputs.sha }} wheel-build-libcudf: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: # build for every combination of arch and CUDA version, but only for the latest Python matrix_filter: group_by([.ARCH, (.CUDA_VER|split(".")|map(tonumber)|.[0])]) | map(max_by(.PY_VER|split(".")|map(tonumber))) @@ -81,7 +81,7 @@ jobs: wheel-publish-libcudf: needs: wheel-build-libcudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -92,7 +92,7 @@ jobs: wheel-build-pylibcudf: needs: [wheel-publish-libcudf] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -102,7 +102,7 @@ jobs: wheel-publish-pylibcudf: needs: wheel-build-pylibcudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -113,7 +113,7 @@ jobs: wheel-build-cudf: needs: wheel-publish-pylibcudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -123,7 +123,7 @@ jobs: wheel-publish-cudf: needs: wheel-build-cudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -134,7 +134,7 @@ jobs: wheel-build-dask-cudf: needs: wheel-publish-cudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". matrix_filter: map(select(.ARCH == "amd64")) | group_by(.CUDA_VER|split(".")|map(tonumber)|.[0]) | map(max_by([(.PY_VER|split(".")|map(tonumber)), (.CUDA_VER|split(".")|map(tonumber))])) @@ -146,7 +146,7 @@ jobs: wheel-publish-dask-cudf: needs: wheel-build-dask-cudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} @@ -157,7 +157,7 @@ jobs: wheel-build-cudf-polars: needs: wheel-publish-pylibcudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". matrix_filter: map(select(.ARCH == "amd64")) | group_by(.CUDA_VER|split(".")|map(tonumber)|.[0]) | map(max_by([(.PY_VER|split(".")|map(tonumber)), (.CUDA_VER|split(".")|map(tonumber))])) @@ -169,7 +169,7 @@ jobs: wheel-publish-cudf-polars: needs: wheel-build-cudf-polars secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-publish.yaml@branch-24.10 with: build_type: ${{ inputs.build_type || 'branch' }} branch: ${{ inputs.branch }} diff --git a/.github/workflows/pandas-tests.yaml b/.github/workflows/pandas-tests.yaml index d670132cca9..10c803f7921 100644 --- a/.github/workflows/pandas-tests.yaml +++ b/.github/workflows/pandas-tests.yaml @@ -17,7 +17,7 @@ jobs: pandas-tests: # run the Pandas unit tests secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". matrix_filter: map(select(.ARCH == "amd64")) | group_by(.CUDA_VER|split(".")|map(tonumber)|.[0]) | map(max_by([(.PY_VER|split(".")|map(tonumber)), (.CUDA_VER|split(".")|map(tonumber))])) diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml index d7d14ea12ff..b515dbff9f3 100644 --- a/.github/workflows/pr.yaml +++ b/.github/workflows/pr.yaml @@ -37,7 +37,7 @@ jobs: - pandas-tests - pandas-tests-diff secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/pr-builder.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/pr-builder.yaml@branch-24.10 if: always() with: needs: ${{ toJSON(needs) }} @@ -104,39 +104,39 @@ jobs: - '!notebooks/**' checks: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/checks.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/checks.yaml@branch-24.10 with: enable_check_generated_files: false conda-cpp-build: needs: checks secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-build.yaml@branch-24.10 with: build_type: pull-request conda-cpp-checks: needs: conda-cpp-build secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-post-build-checks.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-post-build-checks.yaml@branch-24.10 with: build_type: pull-request enable_check_symbols: true conda-cpp-tests: needs: [conda-cpp-build, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-tests.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-tests.yaml@branch-24.10 if: needs.changed-files.outputs.test_cpp == 'true' with: build_type: pull-request conda-python-build: needs: conda-cpp-build secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-python-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-python-build.yaml@branch-24.10 with: build_type: pull-request conda-python-cudf-tests: needs: [conda-python-build, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: build_type: pull-request @@ -145,7 +145,7 @@ jobs: # Tests for dask_cudf, custreamz, cudf_kafka are separated for CI parallelism needs: [conda-python-build, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: build_type: pull-request @@ -153,7 +153,7 @@ jobs: conda-java-tests: needs: [conda-cpp-build, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 if: needs.changed-files.outputs.test_java == 'true' with: build_type: pull-request @@ -164,7 +164,7 @@ jobs: static-configure: needs: checks secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: pull-request # Use the wheel container so we can skip conda solves and since our @@ -174,7 +174,7 @@ jobs: conda-notebook-tests: needs: [conda-python-build, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 if: needs.changed-files.outputs.test_notebooks == 'true' with: build_type: pull-request @@ -185,7 +185,7 @@ jobs: docs-build: needs: conda-python-build secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: pull-request node_type: "gpu-v100-latest-1" @@ -195,7 +195,7 @@ jobs: wheel-build-libcudf: needs: checks secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: # build for every combination of arch and CUDA version, but only for the latest Python matrix_filter: group_by([.ARCH, (.CUDA_VER|split(".")|map(tonumber)|.[0])]) | map(max_by(.PY_VER|split(".")|map(tonumber))) @@ -204,21 +204,21 @@ jobs: wheel-build-pylibcudf: needs: [checks, wheel-build-libcudf] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: build_type: pull-request script: "ci/build_wheel_pylibcudf.sh" wheel-build-cudf: needs: wheel-build-pylibcudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: build_type: pull-request script: "ci/build_wheel_cudf.sh" wheel-tests-cudf: needs: [wheel-build-cudf, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: build_type: pull-request @@ -226,7 +226,7 @@ jobs: wheel-build-cudf-polars: needs: wheel-build-pylibcudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". matrix_filter: map(select(.ARCH == "amd64")) | group_by(.CUDA_VER|split(".")|map(tonumber)|.[0]) | map(max_by([(.PY_VER|split(".")|map(tonumber)), (.CUDA_VER|split(".")|map(tonumber))])) @@ -235,7 +235,7 @@ jobs: wheel-tests-cudf-polars: needs: [wheel-build-cudf-polars, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". @@ -247,7 +247,7 @@ jobs: wheel-build-dask-cudf: needs: wheel-build-cudf secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-build.yaml@branch-24.10 with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". matrix_filter: map(select(.ARCH == "amd64")) | group_by(.CUDA_VER|split(".")|map(tonumber)|.[0]) | map(max_by([(.PY_VER|split(".")|map(tonumber)), (.CUDA_VER|split(".")|map(tonumber))])) @@ -256,7 +256,7 @@ jobs: wheel-tests-dask-cudf: needs: [wheel-build-dask-cudf, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". @@ -265,7 +265,7 @@ jobs: script: ci/test_wheel_dask_cudf.sh devcontainer: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/build-in-devcontainer.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/build-in-devcontainer.yaml@branch-24.10 with: arch: '["amd64"]' cuda: '["12.5"]' @@ -276,7 +276,7 @@ jobs: unit-tests-cudf-pandas: needs: [wheel-build-cudf, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". @@ -287,7 +287,7 @@ jobs: # run the Pandas unit tests using PR branch needs: [wheel-build-cudf, changed-files] secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 if: needs.changed-files.outputs.test_python == 'true' with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". @@ -299,7 +299,7 @@ jobs: pandas-tests-diff: # diff the results of running the Pandas unit tests and publish a job summary needs: pandas-tests - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: node_type: cpu4 build_type: pull-request diff --git a/.github/workflows/pr_issue_status_automation.yml b/.github/workflows/pr_issue_status_automation.yml index fe77ad4b6b2..45e5191eb54 100644 --- a/.github/workflows/pr_issue_status_automation.yml +++ b/.github/workflows/pr_issue_status_automation.yml @@ -23,7 +23,7 @@ on: jobs: get-project-id: - uses: rapidsai/shared-workflows/.github/workflows/project-get-item-id.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/project-get-item-id.yaml@branch-24.10 if: github.event.pull_request.state == 'open' secrets: inherit permissions: @@ -34,7 +34,7 @@ jobs: update-status: # This job sets the PR and its linked issues to "In Progress" status - uses: rapidsai/shared-workflows/.github/workflows/project-get-set-single-select-field.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/project-get-set-single-select-field.yaml@branch-24.10 if: ${{ github.event.pull_request.state == 'open' && needs.get-project-id.outputs.ITEM_PROJECT_ID != '' }} needs: get-project-id with: @@ -50,7 +50,7 @@ jobs: update-sprint: # This job sets the PR and its linked issues to the current "Weekly Sprint" - uses: rapidsai/shared-workflows/.github/workflows/project-get-set-iteration-field.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/project-get-set-iteration-field.yaml@branch-24.10 if: ${{ github.event.pull_request.state == 'open' && needs.get-project-id.outputs.ITEM_PROJECT_ID != '' }} needs: get-project-id with: diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 4af6a0d690d..8605fa46f68 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -16,7 +16,7 @@ on: jobs: conda-cpp-checks: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-post-build-checks.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-post-build-checks.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -25,7 +25,7 @@ jobs: enable_check_symbols: true conda-cpp-tests: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-tests.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-cpp-tests.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -33,7 +33,7 @@ jobs: sha: ${{ inputs.sha }} conda-cpp-memcheck-tests: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -45,7 +45,7 @@ jobs: run_script: "ci/test_cpp_memcheck.sh" static-configure: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: pull-request # Use the wheel container so we can skip conda solves and since our @@ -54,7 +54,7 @@ jobs: run_script: "ci/configure_cpp_static.sh" conda-python-cudf-tests: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -64,7 +64,7 @@ jobs: conda-python-other-tests: # Tests for dask_cudf, custreamz, cudf_kafka are separated for CI parallelism secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/conda-python-tests.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -73,7 +73,7 @@ jobs: script: "ci/test_python_other.sh" conda-java-tests: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -85,7 +85,7 @@ jobs: run_script: "ci/test_java.sh" conda-notebook-tests: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -97,7 +97,7 @@ jobs: run_script: "ci/test_notebooks.sh" wheel-tests-cudf: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -106,7 +106,7 @@ jobs: script: ci/test_wheel_cudf.sh wheel-tests-dask-cudf: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 with: # This selects "ARCH=amd64 + the latest supported Python + CUDA". matrix_filter: map(select(.ARCH == "amd64")) | group_by(.CUDA_VER|split(".")|map(tonumber)|.[0]) | map(max_by([(.PY_VER|split(".")|map(tonumber)), (.CUDA_VER|split(".")|map(tonumber))])) @@ -117,7 +117,7 @@ jobs: script: ci/test_wheel_dask_cudf.sh unit-tests-cudf-pandas: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/wheels-test.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }} @@ -126,7 +126,7 @@ jobs: script: ci/cudf_pandas_scripts/run_tests.sh third-party-integration-tests-cudf-pandas: secrets: inherit - uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@python-3.12 + uses: rapidsai/shared-workflows/.github/workflows/custom-job.yaml@branch-24.10 with: build_type: nightly branch: ${{ inputs.branch }}