Skip to content

Commit

Permalink
add docstring and simplify code a bit. Improve test
Browse files Browse the repository at this point in the history
  • Loading branch information
rjzamora committed Nov 9, 2022
1 parent 5031bad commit 7bf96b5
Show file tree
Hide file tree
Showing 3 changed files with 68 additions and 28 deletions.
16 changes: 4 additions & 12 deletions python/dask_cudf/dask_cudf/backends.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@

import warnings
from collections.abc import Iterator
from functools import partial

import cupy as cp
import numpy as np
Expand Down Expand Up @@ -494,17 +493,10 @@ def read_parquet(*args, engine=None, **kwargs):
)

@staticmethod
def read_json(*args, engine="cudf", **kwargs):
return _default_backend(
dd.read_json,
*args,
engine=(
partial(cudf.read_json, engine=engine)
if isinstance(engine, str)
else engine
),
**kwargs,
)
def read_json(*args, **kwargs):
from dask_cudf.io.json import read_json

return read_json(*args, **kwargs)

@staticmethod
def read_orc(*args, **kwargs):
Expand Down
67 changes: 58 additions & 9 deletions python/dask_cudf/dask_cudf/io/json.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,17 +6,66 @@

import cudf

from dask_cudf.backends import _default_backend

def read_json(path, engine="auto", **kwargs):
# Wrap `dd.read_json` with special engine handling

def read_json(url_path, engine="auto", **kwargs):
"""Create a dask_cudf DataFrame collection from JSON data
This function wraps ``dask.dataframe.read_json``, and passes
``engine=partial(cudf.read_json, engine="auto")`` by default.
Parameters
----------
url_path: str, list of str
Location to read from. If a string, can include a glob character to
find a set of file names.
Supports protocol specifications such as ``"s3://"``.
engine : str or Callable, default "auto"
If str, this value will be used as the ``engine`` argument when
``cudf.read_json`` is used to create each partition. If Callable,
this value will be used as the underlying function used to create
each partition from JSON data. The default value is "auto", so
that ``engine=partial(cudf.read_json, engine="auto")`` will be
pased to ``dask.dataframe.read_json`` by default.
**kwargs :
Key-word arguments to pass through to ``dask.dataframe.read_json``.
Returns
-------
dask_cudf.DataFrame
Examples
--------
Load single file
>>> from dask_cudf import read_json
>>> read_json('myfile.json') # doctest: +SKIP
Load large line-delimited JSON files using partitions of approx
256MB size
>>> read_json('data/file*.csv', blocksize=2**28) # doctest: +SKIP
Load nested JSON data
>>> read_json('myfile.json', engine='cudf_experimental') # doctest: +SKIP
See Also
--------
dask.dataframe.io.json.read_json
"""

# TODO: Add optimized code path to leverage the
# `byte_range` argument in `cudf.read_json` for
# local storage (see `dask_cudf.read_csv`)
if isinstance(engine, str):
# Pass `str` engine argument to `cudf.read_json``
engine = partial(cudf.read_json, engine=engine)
elif not callable(engine):
raise ValueError(f"Unsupported engine option: {engine}")
# Pass `callable` engine argument to `dd.read_json`
return dask.dataframe.read_json(path, engine=engine, **kwargs)
return _default_backend(
dask.dataframe.read_json,
url_path,
engine=(
partial(cudf.read_json, engine=engine)
if isinstance(engine, str)
else engine
),
**kwargs,
)
13 changes: 6 additions & 7 deletions python/dask_cudf/dask_cudf/io/tests/test_json.py
Original file line number Diff line number Diff line change
Expand Up @@ -73,8 +73,7 @@ def test_read_json_lines(lines):
dd.assert_eq(actual, actual_pd)


@pytest.mark.filterwarnings("ignore:Using CPU")
def test_read_json_nested_experimental():
def test_read_json_nested_experimental(tmp_path):
# Check that `engine="cudf_experimental"` can
# be used to support nested data
df = pd.DataFrame(
Expand All @@ -84,9 +83,9 @@ def test_read_json_nested_experimental():
"c": [1, 3, 5, 7],
}
)
lines = dict(orient="records", lines=True)
with tmpfile("json") as f:
df.to_json(f, **lines)
actual = dask_cudf.read_json(f, engine="cudf_experimental", **lines)
actual_pd = pd.read_json(f, **lines)
kwargs = dict(orient="records", lines=True)
with tmp_path / "data.json" as f:
df.to_json(f, **kwargs)
actual = dask_cudf.read_json(f, engine="cudf_experimental", **kwargs)
actual_pd = pd.read_json(f, **kwargs)
dd.assert_eq(actual, actual_pd)

0 comments on commit 7bf96b5

Please sign in to comment.