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

TypeError: '<=' not supported between instances of 'int' and 'str' if dask.dataframe use map_partitions for twice #8085

Closed
df19900725 opened this issue Aug 25, 2021 · 6 comments

Comments

@df19900725
Copy link

What happened: dask.DataFrame use map_partitions to add a new column. But the first time it will be successfully transformed, but it will throw TypeError: '<=' not supported between instances of 'int' and 'str' for the second time

What you expected to happen: it should be get correct new column for the second time

Minimal Complete Verifiable Example:

import pandas as pd
import dask.dataframe as dd

df = pd.DataFrame([[1, 2, 3], [2, 2, 3]])
print(df)

col_name = 0
df = dd.from_pandas(df, npartitions=1)
df["new_0"] = df.map_partitions(
    lambda x: [val for val in x[col_name]]
)
print(df.head())

df["new_1"] = df.map_partitions(
    lambda x: [val for val in x[col_name]]
)
print(df.head())

Anything else we need to know?:
The first and the second 'print' will get correct result, however, the last one throws error:

TypeError: '<=' not supported between instances of 'int' and 'str'

Environment:

  • Dask version: 2021.3.1
  • Python version: 3.7
  • Operating System: windows 10
  • Install method (conda, pip, source): pip
@df19900725
Copy link
Author

The full traceback is as follows:

Traceback (most recent call last):
  File "D:/工作目录/00 programs/LODAPSER/resource/data/DMPAsset/training/Model_mcs_dataprocess/projectfiles/test/user_data_process/dask_map_partition.py", line 17, in <module>
    lambda x: [val for val in x[col_name]]
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\core.py", line 3881, in __setitem__
    df = self.assign(**{key: value})
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\core.py", line 4193, in assign
    return elemwise(methods.assign, self, *pairs, meta=df2)
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\core.py", line 5102, in elemwise
    args = _maybe_align_partitions(args)
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\multi.py", line 165, in _maybe_align_partitions
    dfs = [df for df in args if isinstance(df, _Frame) and not _is_broadcastable(df)]
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\multi.py", line 165, in <listcomp>
    dfs = [df for df in args if isinstance(df, _Frame) and not _is_broadcastable(df)]
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\core.py", line 5063, in is_broadcastable
    for df in dfs
  File "D:\工作目录\00 programs\LODAPSER\resource\data\DMPAsset\training\Model_mcs_dataprocess\venv\lib\site-packages\dask\dataframe\core.py", line 5064, in <genexpr>
    if isinstance(df, DataFrame)
  File "D:\Program Files\Python37\lib\site-packages\pandas\core\base.py", line 764, in min
    return nanops.nanmin(self._values, skipna=skipna)
  File "D:\Program Files\Python37\lib\site-packages\pandas\core\nanops.py", line 135, in f
    result = alt(values, axis=axis, skipna=skipna, **kwds)
  File "D:\Program Files\Python37\lib\site-packages\pandas\core\nanops.py", line 394, in new_func
    result = func(values, axis=axis, skipna=skipna, mask=mask, **kwargs)
  File "D:\Program Files\Python37\lib\site-packages\pandas\core\nanops.py", line 977, in reduction
    result = getattr(values, meth)(axis)
  File "D:\Program Files\Python37\lib\site-packages\numpy\core\_methods.py", line 34, in _amin
    return umr_minimum(a, axis, None, out, keepdims, initial, where)
TypeError: '<=' not supported between instances of 'str' and 'int'

Process finished with exit code 1

If I add str col names on the original dataframe, then the program will run successfully. I found that it seems try to get min value from new column names [0,1,2,"new_0"] by use pandas.DataFrame.index.min.

I think this is a bug of dask...does anyone who have some commments of this?

@jrbourbeau
Copy link
Member

Thanks for reporting this issue and providing a nice reproducer @df19900725. Indeed it looks like our logic here

dask/dask/dataframe/core.py

Lines 5220 to 5233 in fb4e90a

def is_broadcastable(dfs, s):
"""
This Series is broadcastable against another dataframe in the sequence
"""
return (
isinstance(s, Series)
and s.npartitions == 1
and s.known_divisions
and any(
s.divisions == (df.columns.min(), df.columns.max())
for df in dfs
if isinstance(df, DataFrame)
)
)

isn't robust to mixed str and int column names. Is this something you're interested in working on? (no obligation though)

@df19900725
Copy link
Author

Sure. I will try to fix this.

@ncclementi
Copy link
Member

@df19900725 checking in here, did you have the chance to take a look at this, if so, would you like to open a PR with the fix?

@df19900725
Copy link
Author

@ncclementi I find it was not easy to fix because if that dataframe.index contains both str and int, it can not be sorted. I can not find a good solution to solve this. Maybe we can throw an exception here?

@jrbourbeau
Copy link
Member

Closed via #9485

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

No branches or pull requests

3 participants