Skip to content

DataFrame.set_index fails when column has pandas extension dtype #5720

@jrbourbeau

Description

@jrbourbeau

I ran across an issue where set_index fails when the column that's being set to the index has a dtype of Int64 (an extension dtype from pandas). Here's a reproducer:

import numpy as np
import pandas as pd
import dask.dataframe as dd

df = pd.DataFrame({"a": [1, np.nan, 3], "b": [4, 5, 6]})
df = df.astype({"a": "Int64", "b": "int"})

ddf = dd.from_pandas(df, npartitions=2)
# Column "a" is Int64, while "b" is normal int64
print(f"ddf.dtypes = {ddf.dtypes}")
# set_index on int64 column works
print(f"ddf.set_index('b') = {ddf.set_index('b')}")
# set_index on Int64 column fails
print(f"ddf.set_index('a') = {ddf.set_index('a')}")

fails with TypeError: data type not understood

Full traceback:
ddf.dtypes = a    Int64
b    int64
dtype: object
ddf.set_index('b') = Dask DataFrame Structure:
                   a
npartitions=1
4              Int64
6                ...
Dask Name: sort_index, 3 tasks
Traceback (most recent call last):
  File "set_index_extension_type.py", line 14, in <module>
    print(f"ddf.set_index('a') = {ddf.set_index('a')}")
  File "/Users/jbourbeau/github/dask/dask/dask/dataframe/core.py", line 3532, in set_index
    **kwargs
  File "/Users/jbourbeau/github/dask/dask/dask/dataframe/shuffle.py", line 71, in set_index
    divisions, sizes, mins, maxes, optimize_graph=False
  File "/Users/jbourbeau/github/dask/dask/dask/base.py", line 436, in compute
    results = schedule(dsk, keys, **kwargs)
  File "/Users/jbourbeau/github/dask/dask/dask/threaded.py", line 81, in get
    **kwargs
  File "/Users/jbourbeau/github/dask/dask/dask/local.py", line 486, in get_async
    raise_exception(exc, tb)
  File "/Users/jbourbeau/github/dask/dask/dask/local.py", line 316, in reraise
    raise exc
  File "/Users/jbourbeau/github/dask/dask/dask/local.py", line 222, in execute_task
    result = _execute_task(task, data)
  File "/Users/jbourbeau/github/dask/dask/dask/core.py", line 119, in _execute_task
    return func(*args2)
  File "/Users/jbourbeau/github/dask/dask/dask/dataframe/partitionquantiles.py", line 415, in percentiles_summary
    vals, n = _percentile(data, qs, interpolation=interpolation)
  File "/Users/jbourbeau/github/dask/dask/dask/array/percentile.py", line 25, in _percentile
    if np.issubdtype(a.dtype, np.datetime64):
  File "/Users/jbourbeau/miniconda/envs/dask-dev/lib/python3.7/site-packages/numpy/core/numerictypes.py", line 393, in issubdtype
    arg1 = dtype(arg1).type
TypeError: data type not understood

It looks like when we compute the divisions here

divisions = index2._repartition_quantiles(npartitions, upsample=upsample)

we pass the column dtype to NumPy here

if np.issubdtype(a.dtype, np.datetime64):

which is where the breakage occurs

Metadata

Metadata

Assignees

Labels

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions