Skip to content

dispatch.registers to their own file#7503

Merged
jrbourbeau merged 10 commits intodask:mainfrom
jsignell:move-pandas-dispatch-register
May 24, 2021
Merged

dispatch.registers to their own file#7503
jrbourbeau merged 10 commits intodask:mainfrom
jsignell:move-pandas-dispatch-register

Conversation

@jsignell
Copy link
Member

@jsignell jsignell commented Apr 1, 2021

These could also go into backends.py if people prefer. I have no strong opinion.

@jsignell jsignell changed the title Move pandas dispatch register dispatch.registers to their own file Apr 1, 2021
@kkraus14
Copy link
Member

kkraus14 commented Apr 1, 2021

Thanks @jsignell this looks great generally. This is a hugely breaking change that will likely break anyone using DataFrame dispatches outside of Dask (not sure if there's anyone other than dask-cudf).

I'm going to kickoff some conversations internally on our side, but if possible it would be ideal to hold off on merging this for a bit to give us a chance to get our ducks in a row.

@jsignell
Copy link
Member Author

jsignell commented Apr 1, 2021

Thanks @jsignell this looks great generally. This is a hugely breaking change that will likely break anyone using DataFrame dispatches outside of Dask (not sure if there's anyone other than dask-cudf).

We should discuss if this is something that is worth doing. To my mind, it's a good clean-up, but if we are committed to breaking people, we should make sure that it's change that we are really happy with.

I'm going to kickoff some conversations internally on our side, but if possible it would be ideal to hold off on merging this for a bit to give us a chance to get our ducks in a row.

Absolutely

@kkraus14
Copy link
Member

kkraus14 commented Apr 1, 2021

We should discuss if this is something that is worth doing. To my mind, it's a good clean-up, but if we are committed to breaking people, we should make sure that it's change that we are really happy with.

I'm a big +1 to this change and it's much appreciated.

Regardless, after the meeting this morning and this RAPIDS release, we've decided as a team that we're going to start constraining Dask to its latest release version at the time of our release instead of letting it float. Breaking changes are going in too frequently and Dask releases go out in between our releases too frequently, where repeatedly fixing our previous release is too expensive for our team to do at this time.

@jsignell
Copy link
Member Author

jsignell commented Apr 2, 2021

This PR would break a lot less if the functions that are being registered were private. Is that already the implication? If it is not then I would be happy to open a smaller PR that prefixes them all with an underscore.

@jsignell jsignell force-pushed the move-pandas-dispatch-register branch from 607a407 to a7b8e8f Compare April 2, 2021 20:55
@jsignell
Copy link
Member Author

jsignell commented Apr 2, 2021

Ok I looked at dask_cudf and tried to preserve the imports that are used there. Would someone be able to run the cudf tests on this PR and let me know if there are any failures?

@jsignell
Copy link
Member Author

jsignell commented Apr 7, 2021

@kkraus14 and @quasiben can you test this with cudf and let me know if there are failures.

@jsignell
Copy link
Member Author

Maybe now that GTC is over :)

@quasiben
Copy link
Member

quasiben commented Apr 19, 2021

@charlesbluca would you have time to run these tests and report back ?

@charlesbluca
Copy link
Member

Ran through the dask-cudf tests, here are the results - lots of failures, but not sure if that's because of my env:

$ conda list | grep -e "dask\|cudf"
cudf                      0.20.0a210419   cuda_11.2_py38_g867d6eecca_166    rapidsai-nightly
dask                      2.9.0+733.g881a7474           dev_0    <develop>
dask-cudf                 0.20.0a210419   py38_g867d6eecca_166    rapidsai-nightly
libcudf                   0.20.0a210419   cuda11.2_g808262f49c_167    rapidsai-nightly

Let me know if I should change anything and rerun the tests.

test_results.txt

@quasiben
Copy link
Member

here is an example of an error:

________________________ test_categorical_basic[data0] _________________________

data = ['a', 'a', 'b', 'c', 'a']
Categories (3, object): ['a', 'b', 'c']

    @pytest.mark.parametrize("data", [data_cat_1()])
    def test_categorical_basic(data):
        cat = data.copy()
        pdsr = pd.Series(cat)
        sr = Series(cat)
        dsr = dgd.from_cudf(sr, npartitions=2)
        result = dsr.compute()
        np.testing.assert_array_equal(cat.codes, result.to_array())

        assert dsr.dtype.to_pandas() == pdsr.dtype
        # Test attributes
        assert pdsr.cat.ordered == dsr.cat.ordered

        assert_eq(pdsr.cat.categories, dsr.cat.categories)

        np.testing.assert_array_equal(pdsr.cat.codes.values, result.to_array())

        string = str(result)
        expect_str = """
    0 a
    1 a
    2 b
    3 c
    4 a
    """
        assert all(x == y for x, y in zip(string.split(), expect_str.split()))

        df = DataFrame()
        df["a"] = ["xyz", "abc", "def"] * 10

        pdf = df.to_pandas()
        cddf = dgd.from_cudf(df, 1)
>       cddf["b"] = cddf["a"].astype("category")

test_accessor.py:142:
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
../../../../../dask-dispatch-test/dask/dataframe/core.py:3885: in __setitem__
    df = self.assign(**{key: value})
../../../../../dask-dispatch-test/dask/dataframe/core.py:4196: in assign
    df2 = self._meta_nonempty.assign(**_extract_meta(kwargs, nonempty=True))
../../../../../dask-dispatch-test/dask/dataframe/core.py:5496: in _extract_meta
    res[k] = _extract_meta(x[k], nonempty)
../../../../../dask-dispatch-test/dask/dataframe/core.py:5488: in _extract_meta
    return x._meta_nonempty if nonempty else x._meta
../../../../../dask-dispatch-test/dask/dataframe/core.py:375: in _meta_nonempty
    return meta_nonempty(self._meta)
../../../../../dask-dispatch-test/dask/utils.py:512: in __call__
    return meth(arg, *args, **kwargs)
../backends.py:96: in _nonempty_series
    data = _get_non_empty_data(s)
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _

s = Series([], Name: a, dtype: category
Categories (0, dtype: category)

    def _get_non_empty_data(s):
        if isinstance(s._column, cudf.core.column.CategoricalColumn):
            categories = (
                s._column.categories
                if len(s._column.categories)
                else [UNKNOWN_CATEGORIES]
            )
>           codes = column.full(size=2, fill_value=0, dtype="int32")
E           NameError: name 'column' is not defined

@galipremsagar @kkraus14 do you have an idea of what's going on here?

@galipremsagar
Copy link
Contributor

galipremsagar commented Apr 19, 2021

      codes = column.full(size=2, fill_value=0, dtype="int32")

E NameError: name 'column' is not defined


@galipremsagar @kkraus14 do you have an idea of what's going on here?

This is a bug that has to be fixed in dask-cudf, column -> cudf.core.column

@galipremsagar
Copy link
Contributor

galipremsagar commented Apr 19, 2021

This is a bug that has to be fixed in dask-cudf, column -> cudf.core.column

@charlesbluca can you confirm if making just this change unblocks you for validating against changes in this PR?

@charlesbluca
Copy link
Member

Yes, that resolves all failures - thanks @galipremsagar! Should I open a PR with cuDF for this?

@galipremsagar
Copy link
Contributor

Yes, that resolves all failures - thanks @galipremsagar! Should I open a PR with cuDF for this?

Thanks for confirming, I'm on it 👍

@jsignell
Copy link
Member Author

Thanks @charlesbluca, @quasiben, and @galipremsagar! So seems like this is safe from the dask-cudf perspective?

@jsignell jsignell mentioned this pull request Apr 20, 2021
3 tasks
@kkraus14
Copy link
Member

Thanks @charlesbluca, @quasiben, and @galipremsagar! So seems like this is safe from the dask-cudf perspective?

Yes. Generally we are now constraining dask / distributed versions at release time to the latest released patch version so we should be more protected against breaking changes being introduced.

@jsignell
Copy link
Member Author

Ok @jrbourbeau should we merge this and #7505?

@jakirkham jakirkham mentioned this pull request Apr 27, 2021
3 tasks
@galipremsagar
Copy link
Contributor

Can we get this PR to be merged? This will help unblock other PRs around having major changes in dispatch: #7586

cc: @jakirkham @quasiben @jrbourbeau

@jakirkham
Copy link
Member

+1 from me

@jrbourbeau do you have any thoughts? Should we merge this and the array PR ( #7505 )?

@jsignell
Copy link
Member Author

James is fixing up #7505 to match the pattern in this PR and then we'll merge both.

"""
Dispatch in dask.dataframe.

Also see extension.py
Copy link
Member

Choose a reason for hiding this comment

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

Nitpick: Should this be backends.py?

Copy link
Member Author

Choose a reason for hiding this comment

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

I think it this is right. There are some Dispatch objects initialized in there as well. I thought they were different in some way and should stay in extension.py, but I can't remember why I thought that. Do you think they should move into this file?

Copy link
Member

Choose a reason for hiding this comment

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

Ah, I see. Since those are specifically for registering extension array, let's keep those where they are

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 @jsignell!

@jrbourbeau jrbourbeau merged commit 123cd5f into dask:main May 24, 2021
rapids-bot bot pushed a commit to rapidsai/dask-cuda that referenced this pull request May 26, 2021
Requires PR ( rapidsai/cudf#8342 )

As these functions recently got relocated, handle `import`ing from the new location with a fallback to the old location.

xref: dask/dask#7503
xref: dask/dask#7505

Authors:
  - https://github.com/jakirkham

Approvers:
  - GALI PREM SAGAR (https://github.com/galipremsagar)
  - Benjamin Zaitlen (https://github.com/quasiben)

URL: #623
@chrisroat
Copy link
Contributor

I think this may have broken the use of make_meta. (I also see @GenevieveBuckley noted this).

>>> dd.utils.make_meta([('a', 'i8'), ('b', 'O')])
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/home/croat/.local/share/virtualenvs/starmap-T47byR32/lib/python3.8/site-packages/dask/utils.py", line 511, in __call__
    meth = self.dispatch(type(arg))
  File "/home/croat/.local/share/virtualenvs/starmap-T47byR32/lib/python3.8/site-packages/dask/utils.py", line 505, in dispatch
    raise TypeError("No dispatch for {0}".format(cls))
TypeError: No dispatch for <class 'list'>

@galipremsagar
Copy link
Contributor

I think this may have broken the use of make_meta. (I also see @GenevieveBuckley noted this).

>>> dd.utils.make_meta([('a', 'i8'), ('b', 'O')])
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/home/croat/.local/share/virtualenvs/starmap-T47byR32/lib/python3.8/site-packages/dask/utils.py", line 511, in __call__
    meth = self.dispatch(type(arg))
  File "/home/croat/.local/share/virtualenvs/starmap-T47byR32/lib/python3.8/site-packages/dask/utils.py", line 505, in dispatch
    raise TypeError("No dispatch for {0}".format(cls))
TypeError: No dispatch for <class 'list'>

In #7586, there was a new function introduced that has to be invoked to make meta i.e.,
make_meta_util: dd.utils.make_meta_util and not make_meta directly.

@chrisroat
Copy link
Contributor

@galipremsagar make_meta_util calls make_meta for things like a list, so it fails in the same way:

$ python
Python 3.8.5 (default, Jan 27 2021, 15:41:15) 
[GCC 9.3.0] on linux
Type "help", "copyright", "credits" or "license" for more information.
>>> import dask
>>> dask.__version__
'2021.05.1'
>>> import dask.dataframe as dd
>>> dd.utils.make_meta_util([('a', 'i8'), ('b', 'O')])
Traceback (most recent call last):
  File "/home/croat/make_meta/venv/lib/python3.8/site-packages/dask/dataframe/dispatch.py", line 100, in make_meta_util
    return make_meta(x, index=index)
  File "/home/croat/make_meta/venv/lib/python3.8/site-packages/dask/utils.py", line 511, in __call__
    meth = self.dispatch(type(arg))
  File "/home/croat/make_meta/venv/lib/python3.8/site-packages/dask/utils.py", line 505, in dispatch
    raise TypeError("No dispatch for {0}".format(cls))
TypeError: No dispatch for <class 'list'>

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/home/croat/make_meta/venv/lib/python3.8/site-packages/dask/dataframe/dispatch.py", line 106, in make_meta_util
    func = make_meta_obj.dispatch(type(x))
  File "/home/croat/make_meta/venv/lib/python3.8/site-packages/dask/utils.py", line 505, in dispatch
    raise TypeError("No dispatch for {0}".format(cls))
TypeError: No dispatch for <class 'list'>
>>> 

@jakirkham
Copy link
Member

@chrisroat could you please file this in an issue? This discussion is a bit hard to track in a merged PR, which seems unlikely to be related to the issue at hand

@chrisroat
Copy link
Contributor

#7731

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.

Move pandas dispatch.registers to their own file

8 participants