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

Fix issue with header passed to read_csv #10355

Merged
merged 2 commits into from Jun 20, 2023
Merged

Conversation

galipremsagar
Copy link
Contributor

We currently have an issue in read_csv where the following types of inputs fail:

In [1]: import dask.dataframe as dd

In [2]: !cat ../s.csv
x
x
x
A, B, C, D
1, 2, 3, 4
2, 3, 5, 1
4, 5, 2, 5


In [3]: dd.read_csv("../s.csv", header=3).compute()
---------------------------------------------------------------------------
IndexError                                Traceback (most recent call last)
File /nvme/0/pgali/cudf/dask/dask/backends.py:136, in CreationDispatch.register_inplace.<locals>.decorator.<locals>.wrapper(*args, **kwargs)
    135 try:
--> 136     return func(*args, **kwargs)
    137 except Exception as e:

File /nvme/0/pgali/cudf/dask/dask/dataframe/io/csv.py:760, in make_reader.<locals>.read(urlpath, blocksize, lineterminator, compression, sample, sample_rows, enforce, assume_missing, storage_options, include_path_column, **kwargs)
    747 def read(
    748     urlpath,
    749     blocksize="default",
   (...)
    758     **kwargs,
    759 ):
--> 760     return read_pandas(
    761         reader,
    762         urlpath,
    763         blocksize=blocksize,
    764         lineterminator=lineterminator,
    765         compression=compression,
    766         sample=sample,
    767         sample_rows=sample_rows,
    768         enforce=enforce,
    769         assume_missing=assume_missing,
    770         storage_options=storage_options,
    771         include_path_column=include_path_column,
    772         **kwargs,
    773     )

File /nvme/0/pgali/cudf/dask/dask/dataframe/io/csv.py:620, in read_pandas(reader, urlpath, blocksize, lineterminator, compression, sample, sample_rows, enforce, assume_missing, storage_options, include_path_column, **kwargs)
    619     firstrow += header
--> 620 header = b"" if header is None else parts[firstrow] + b_lineterminator
    622 # Use sample to infer dtypes and check for presence of include_path_column

IndexError: list index out of range

The above exception was the direct cause of the following exception:

IndexError                                Traceback (most recent call last)
Cell In[3], line 1
----> 1 dd.read_csv("../s.csv", header=3).compute()

File /nvme/0/pgali/cudf/dask/dask/backends.py:138, in CreationDispatch.register_inplace.<locals>.decorator.<locals>.wrapper(*args, **kwargs)
    136     return func(*args, **kwargs)
    137 except Exception as e:
--> 138     raise type(e)(
    139         f"An error occurred while calling the {funcname(func)} "
    140         f"method registered to the {self.backend} backend.\n"
    141         f"Original Message: {e}"
    142     ) from e

IndexError: An error occurred while calling the read_csv method registered to the pandas backend.
Original Message: list index out of range

In [4]: import pandas as pd

In [5]: pd.read_csv("../s.csv", header=3)
Out[5]: 
   A   B   C   D
0  1   2   3   4
1  2   3   5   1
2  4   5   2   5

This PR fixes the issue by factoring in the header & firstrow parameters while making the parts split.

  • Closes #xxxx
  • Tests added / passed
  • Passes pre-commit run --all-files

Copy link
Collaborator

@phofl phofl left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm

@galipremsagar
Copy link
Contributor Author

Bump for review/merge @jakirkham | @jrbourbeau

@jrbourbeau jrbourbeau changed the title Fix an issue with read_csv when header is passed Fix issue with header passed to read_csv Jun 20, 2023
Copy link
Member

@jrbourbeau jrbourbeau left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @galipremsagar -- this is in

@jrbourbeau jrbourbeau merged commit a6ee367 into dask:main Jun 20, 2023
25 checks passed
@galipremsagar
Copy link
Contributor Author

Thanks @jrbourbeau !

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants