Skip to content

Commit 0058b52

Browse files
authored
Fix dask_cudf.read_csv (#17612)
Recent changes in dask and dask-expr have broken `dask_cudf.read_csv` (dask/dask-expr#1178, dask/dask#11603). Fortunately, the breaking changes help us avoid legacy CSV code in the long run. Authors: - Richard (Rick) Zamora (https://github.com/rjzamora) Approvers: - Peter Andreas Entschev (https://github.com/pentschev) - Lawrence Mitchell (https://github.com/wence-) URL: #17612
1 parent becfacc commit 0058b52

File tree

3 files changed

+215
-25
lines changed

3 files changed

+215
-25
lines changed

python/dask_cudf/dask_cudf/backends.py

Lines changed: 25 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -714,21 +714,35 @@ def read_csv(
714714
storage_options=None,
715715
**kwargs,
716716
):
717-
import dask_expr as dx
718-
from fsspec.utils import stringify_path
717+
try:
718+
# TODO: Remove when cudf is pinned to dask>2024.12.0
719+
import dask_expr as dx
720+
from dask_expr.io.csv import ReadCSV
721+
from fsspec.utils import stringify_path
722+
723+
if not isinstance(path, str):
724+
path = stringify_path(path)
725+
return dx.new_collection(
726+
ReadCSV(
727+
path,
728+
dtype_backend=dtype_backend,
729+
storage_options=storage_options,
730+
kwargs=kwargs,
731+
header=header,
732+
dataframe_backend="cudf",
733+
)
734+
)
735+
except ImportError:
736+
# Requires dask>2024.12.0
737+
from dask_cudf.io.csv import read_csv
719738

720-
if not isinstance(path, str):
721-
path = stringify_path(path)
722-
return dx.new_collection(
723-
dx.io.csv.ReadCSV(
739+
return read_csv(
724740
path,
725-
dtype_backend=dtype_backend,
726-
storage_options=storage_options,
727-
kwargs=kwargs,
741+
*args,
728742
header=header,
729-
dataframe_backend="cudf",
743+
storage_options=storage_options,
744+
**kwargs,
730745
)
731-
)
732746

733747
@staticmethod
734748
def read_json(*args, **kwargs):
Lines changed: 190 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1,8 +1,193 @@
11
# Copyright (c) 2024, NVIDIA CORPORATION.
22

3-
from dask_cudf import _deprecated_api
3+
import os
4+
from glob import glob
5+
from warnings import warn
46

5-
read_csv = _deprecated_api(
6-
"dask_cudf.io.csv.read_csv",
7-
new_api="dask_cudf.read_csv",
8-
)
7+
from fsspec.utils import infer_compression
8+
9+
from dask import dataframe as dd
10+
from dask.dataframe.io.csv import make_reader
11+
from dask.utils import parse_bytes
12+
13+
import cudf
14+
15+
16+
def read_csv(path, blocksize="default", **kwargs):
17+
"""
18+
Read CSV files into a :class:`.DataFrame`.
19+
20+
This API parallelizes the :func:`cudf:cudf.read_csv` function in
21+
the following ways:
22+
23+
It supports loading many files at once using globstrings:
24+
25+
>>> import dask_cudf
26+
>>> df = dask_cudf.read_csv("myfiles.*.csv")
27+
28+
In some cases it can break up large files:
29+
30+
>>> df = dask_cudf.read_csv("largefile.csv", blocksize="256 MiB")
31+
32+
It can read CSV files from external resources (e.g. S3, HTTP, FTP)
33+
34+
>>> df = dask_cudf.read_csv("s3://bucket/myfiles.*.csv")
35+
>>> df = dask_cudf.read_csv("https://www.mycloud.com/sample.csv")
36+
37+
Internally ``read_csv`` uses :func:`cudf:cudf.read_csv` and
38+
supports many of the same keyword arguments with the same
39+
performance guarantees. See the docstring for
40+
:func:`cudf:cudf.read_csv` for more information on available
41+
keyword arguments.
42+
43+
Parameters
44+
----------
45+
path : str, path object, or file-like object
46+
Either a path to a file (a str, :py:class:`pathlib.Path`, or
47+
py._path.local.LocalPath), URL (including http, ftp, and S3
48+
locations), or any object with a read() method (such as
49+
builtin :py:func:`open` file handler function or
50+
:py:class:`~io.StringIO`).
51+
blocksize : int or str, default "256 MiB"
52+
The target task partition size. If ``None``, a single block
53+
is used for each file.
54+
**kwargs : dict
55+
Passthrough key-word arguments that are sent to
56+
:func:`cudf:cudf.read_csv`.
57+
58+
Notes
59+
-----
60+
If any of `skipfooter`/`skiprows`/`nrows` are passed,
61+
`blocksize` will default to None.
62+
63+
Examples
64+
--------
65+
>>> import dask_cudf
66+
>>> ddf = dask_cudf.read_csv("sample.csv", usecols=["a", "b"])
67+
>>> ddf.compute()
68+
a b
69+
0 1 hi
70+
1 2 hello
71+
2 3 ai
72+
73+
"""
74+
# Set default `blocksize`
75+
if blocksize == "default":
76+
if (
77+
kwargs.get("skipfooter", 0) != 0
78+
or kwargs.get("skiprows", 0) != 0
79+
or kwargs.get("nrows", None) is not None
80+
):
81+
# Cannot read in blocks if skipfooter,
82+
# skiprows or nrows is passed.
83+
blocksize = None
84+
else:
85+
blocksize = "256 MiB"
86+
87+
if "://" in str(path):
88+
func = make_reader(cudf.read_csv, "read_csv", "CSV")
89+
return func(path, blocksize=blocksize, **kwargs)
90+
else:
91+
return _internal_read_csv(path=path, blocksize=blocksize, **kwargs)
92+
93+
94+
def _internal_read_csv(path, blocksize="256 MiB", **kwargs):
95+
if isinstance(blocksize, str):
96+
blocksize = parse_bytes(blocksize)
97+
98+
if isinstance(path, list):
99+
filenames = path
100+
elif isinstance(path, str):
101+
filenames = sorted(glob(path))
102+
elif hasattr(path, "__fspath__"):
103+
filenames = sorted(glob(path.__fspath__()))
104+
else:
105+
raise TypeError(f"Path type not understood:{type(path)}")
106+
107+
if not filenames:
108+
msg = f"A file in: {filenames} does not exist."
109+
raise FileNotFoundError(msg)
110+
111+
compression = kwargs.get("compression", "infer")
112+
113+
if compression == "infer":
114+
# Infer compression from first path by default
115+
compression = infer_compression(filenames[0])
116+
117+
if compression and blocksize:
118+
# compressed CSVs reading must read the entire file
119+
kwargs.pop("byte_range", None)
120+
warn(
121+
"Warning %s compression does not support breaking apart files\n"
122+
"Please ensure that each individual file can fit in memory and\n"
123+
"use the keyword ``blocksize=None to remove this message``\n"
124+
"Setting ``blocksize=(size of file)``" % compression
125+
)
126+
blocksize = None
127+
128+
if blocksize is None:
129+
return read_csv_without_blocksize(path, **kwargs)
130+
131+
# Let dask.dataframe generate meta
132+
dask_reader = make_reader(cudf.read_csv, "read_csv", "CSV")
133+
kwargs1 = kwargs.copy()
134+
usecols = kwargs1.pop("usecols", None)
135+
dtype = kwargs1.pop("dtype", None)
136+
meta = dask_reader(filenames[0], **kwargs1)._meta
137+
names = meta.columns
138+
if usecols or dtype:
139+
# Regenerate meta with original kwargs if
140+
# `usecols` or `dtype` was specified
141+
meta = dask_reader(filenames[0], **kwargs)._meta
142+
143+
i = 0
144+
path_list = []
145+
kwargs_list = []
146+
for fn in filenames:
147+
size = os.path.getsize(fn)
148+
for start in range(0, size, blocksize):
149+
kwargs2 = kwargs.copy()
150+
kwargs2["byte_range"] = (
151+
start,
152+
blocksize,
153+
) # specify which chunk of the file we care about
154+
if start != 0:
155+
kwargs2["names"] = names # no header in the middle of the file
156+
kwargs2["header"] = None
157+
path_list.append(fn)
158+
kwargs_list.append(kwargs2)
159+
i += 1
160+
161+
return dd.from_map(_read_csv, path_list, kwargs_list, meta=meta)
162+
163+
164+
def _read_csv(fn, kwargs):
165+
return cudf.read_csv(fn, **kwargs)
166+
167+
168+
def read_csv_without_blocksize(path, **kwargs):
169+
"""Read entire CSV with optional compression (gzip/zip)
170+
171+
Parameters
172+
----------
173+
path : str
174+
path to files (support for glob)
175+
"""
176+
if isinstance(path, list):
177+
filenames = path
178+
elif isinstance(path, str):
179+
filenames = sorted(glob(path))
180+
elif hasattr(path, "__fspath__"):
181+
filenames = sorted(glob(path.__fspath__()))
182+
else:
183+
raise TypeError(f"Path type not understood:{type(path)}")
184+
185+
meta_kwargs = kwargs.copy()
186+
if "skipfooter" in meta_kwargs:
187+
meta_kwargs.pop("skipfooter")
188+
if "nrows" in meta_kwargs:
189+
meta_kwargs.pop("nrows")
190+
# Read "head" of first file (first 5 rows).
191+
# Convert to empty df for metadata.
192+
meta = cudf.read_csv(filenames[0], nrows=5, **meta_kwargs).iloc[:0]
193+
return dd.from_map(cudf.read_csv, filenames, meta=meta, **kwargs)

python/dask_cudf/dask_cudf/io/tests/test_csv.py

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -185,11 +185,6 @@ def test_read_csv_blocksize_none(tmp_path, compression, size):
185185
df2 = dask_cudf.read_csv(path, blocksize=None, dtype=typ)
186186
dd.assert_eq(df, df2)
187187

188-
# Test chunksize deprecation
189-
with pytest.warns(FutureWarning, match="deprecated"):
190-
df3 = dask_cudf.read_csv(path, chunksize=None, dtype=typ)
191-
dd.assert_eq(df, df3)
192-
193188

194189
@pytest.mark.parametrize("dtype", [{"b": str, "c": int}, None])
195190
def test_csv_reader_usecols(tmp_path, dtype):
@@ -275,7 +270,3 @@ def test_deprecated_api_paths(tmp_path):
275270
with pytest.warns(match="dask_cudf.io.read_csv is now deprecated"):
276271
df2 = dask_cudf.io.read_csv(csv_path)
277272
dd.assert_eq(df, df2, check_divisions=False)
278-
279-
with pytest.warns(match="dask_cudf.io.csv.read_csv is now deprecated"):
280-
df2 = dask_cudf.io.csv.read_csv(csv_path)
281-
dd.assert_eq(df, df2, check_divisions=False)

0 commit comments

Comments
 (0)