Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BUG] Calling to_pandas() on a dask dataframe containing a 'struct' column can result in a raised PyArrow exception #13305

Closed
drobison00 opened this issue May 5, 2023 · 4 comments · Fixed by #13315
Assignees
Labels
bug Something isn't working

Comments

@drobison00
Copy link

Describe the bug
Calling to_pandas() on a dask dataframe created from a cuDF dataframe containing a 'struct' column can result in a raised PyArrow exception:

/opt/anaconda/envs/morpheus/lib/python3.10/site-packages/cudf/core/column/struct.py:56: in to_arrow
    return pa.StructArray.from_buffers(
pyarrow/array.pxi:1110: in pyarrow.lib.Array.from_buffers
    ???
pyarrow/array.pxi:1504: in pyarrow.lib.Array.validate
    ???
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 

>   ???
E   pyarrow.lib.ArrowInvalid: Buffer #0 too small in array of type struct<> and length 10: expected at least 2 byte(s), got 0

pyarrow/error.pxi:100: ArrowInvalid

Steps/Code to reproduce bug

Using this example file azure_ad_logs.json: https://github.com/nv-morpheus/Morpheus/blob/branch-23.07/tests/tests_data/azure_ad_logs.json

def test_cudf_struct_type_conversion_dask():
    import cudf
    import dask_cudf

    cdf = cudf.read_json("azure_ad_logs.json")
    ddf = dask_cudf.from_cudf(cdf, npartitions=1)

    result = ddf['properties'].compute().to_pandas()

    print(result)

Expected behavior
A pandas dataframe should be returned.

Environment overview (please complete the following information)
Conda environment

grep 'cudf|arrow|dask'
arrow                     1.2.3                    pypi_0    pypi
arrow-cpp                 10.0.1          ha770c72_14_cuda    conda-forge
cudf                      23.02.00        cuda_11_py310_g5ad4a85b9d_0    rapidsai
dask                      2023.1.1           pyhd8ed1ab_0    conda-forge
dask-core                 2023.1.1           pyhd8ed1ab_0    conda-forge
dask-cuda                 23.2.1             pyhd8ed1ab_1    conda-forge
dask-cudf                 23.02.00        cuda_11_py310_g5ad4a85b9d_0    rapidsai
libarrow                  10.0.1          h255618e_14_cuda    conda-forge
libcudf                   23.02.00        cuda11_g5ad4a85b9d_0    rapidsai
pyarrow                   10.0.1          py310hc81d9b2_14_cuda    conda-forge

Environment details
Please run and paste the output of the cudf/print_env.sh script here, to gather any other relevant environment details

Additional context
Add any other context about the problem here.

@drobison00 drobison00 added Needs Triage Need team to review and classify bug Something isn't working labels May 5, 2023
@shwina
Copy link
Contributor

shwina commented May 8, 2023

Thanks for reporting, @drobison00 -- is there any chance you're able to share the file azure_ad_logs.json? Even better would be if it's possible to reproduce this error from a much smaller dataset or by hand-constructing the inputs but that's not necessary. Thanks!

I apologize! I missed that you dropped a link to it :)

@wence-
Copy link
Contributor

wence- commented May 9, 2023

Here is a more minimal reproducer, it appears to be some interaction between a column of empty structs with nulls and the way dask_cudf initialises the data.

import cudf
import dask_cudf
s = cudf.Series([None, {}])
ds = dask_cudf.from_cudf(s, npartitions=1)
ds.compute()

@wence-
Copy link
Contributor

wence- commented May 9, 2023

OK, this is a bug slicing struct columns:

import cudf
s = cudf.Series([None, {}])
s.iloc[0:2] # => `ArrowInvalid` error

@wence-
Copy link
Contributor

wence- commented May 9, 2023

The null mask returned from libcudf's slice for a struct column with zero children is incorrectly sized as 0 bytes...

The null mask wrapped up by the cython layer's from_column_view is incorrectly sized as zero bytes.

@wence- wence- self-assigned this May 9, 2023
@wence- wence- removed the Needs Triage Need team to review and classify label May 9, 2023
rapids-bot bot pushed a commit that referenced this issue May 9, 2023
An empty struct column (dtype of StructDtype({})) has no children, and
hence a base_size of zero. However, it may still have a non-zero size
and non-empty null mask. When slicing such a column, the mask size
must be transferred over correctly by inspecting the size and offset
of the owning column. Previously, we incorrectly determined the sliced
column to have a mask buffer of zero bytes in this case.

Closes #13305.
rapids-bot bot pushed a commit that referenced this issue May 17, 2023
An empty struct column (dtype of StructDtype({})) has no children, and
hence a base_size of zero. However, it may still have a non-zero size
and non-empty null mask. When slicing such a column, the mask size
must be transferred over correctly by inspecting the size and offset
of the owning column. Previously, we incorrectly determined the sliced
column to have a mask buffer of zero bytes in this case.

Closes #13305.

Authors:
  - Lawrence Mitchell (https://github.com/wence-)
  - Ashwin Srinath (https://github.com/shwina)

Approvers:
  - Bradley Dice (https://github.com/bdice)

URL: #13315
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants