Skip to content

Add std() support for datetime64 dtype for Pandas-like objects (#8214)#8523

Merged
jsignell merged 6 commits intodask:mainfrom
bglossner:datetime_only_v5
Feb 24, 2022
Merged

Add std() support for datetime64 dtype for Pandas-like objects (#8214)#8523
jsignell merged 6 commits intodask:mainfrom
bglossner:datetime_only_v5

Conversation

@bglossner
Copy link
Contributor

Adds support for calling std() on Dask Series and DataFrames when they include datetime64 dtypes.

@GPUtester
Copy link
Collaborator

Can one of the admins verify this patch?

@ian-r-rose ian-r-rose self-requested a review January 5, 2022 17:25
@bglossner
Copy link
Contributor Author

Just checking in on the review here. Did you have a chance to look at it @ian-r-rose?

Copy link
Collaborator

@ian-r-rose ian-r-rose left a comment

Choose a reason for hiding this comment

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

Thanks for working on this @bglossner, this is a nice piece of work!

v = numeric_dd.var(skipna=skipna, ddof=ddof, split_every=split_every)
name = self._token_prefix + "std"

def sqrt_and_convert(p):
Copy link
Collaborator

Choose a reason for hiding this comment

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

Here and a couple of places above: it's my understanding that core Dask generally avoids using closures and lambdas for performance reasons. Could we move this into a module-private function like _sqrt_and_convert_to_timedelta? Variables in the closure like is_df_like would probably need to be args in that case.

from .io import from_pandas
from .numeric import to_numeric

def convert_to_numeric(s):
Copy link
Collaborator

Choose a reason for hiding this comment

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

I see that you at some point experimented with using view("i8"), but couldn't because it was not implemented yet. There is currently an open PR in #8533 adding it. Would that help to simplify things? It might also improve performance, since it would locally be a copy-free operation.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Unfortunately, it seems that view has the same numeric value for NaT, and skipping nans is a feature of TimeDeltaArray.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The performance improvement would at least be nice but I agree in that it doesn't solve the issue that would have been helpful here. I am fine with waiting until that PR is merged and refactoring after that to use view, since it won't be much of a change!

_raise_if_object_series(self, "std")

meta = self._meta_nonempty.std(axis=axis, skipna=skipna)
is_df_like = is_dataframe_like(self._meta_nonempty)
Copy link
Collaborator

Choose a reason for hiding this comment

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

Any reason to use _meta_nonempty instead of _meta here?

Copy link
Contributor Author

@bglossner bglossner Jan 14, 2022

Choose a reason for hiding this comment

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

Nope, I didn't know it made a difference. Is there a reason to use one over the other?

return to_numeric(s).mask(s.isnull(), np.nan)

if is_df_like:
time_cols = self._meta_nonempty.select_dtypes(
Copy link
Collaborator

Choose a reason for hiding this comment

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

As above, can you do this with self._meta?

Copy link
Contributor Author

@bglossner bglossner Jan 14, 2022

Choose a reason for hiding this comment

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

For this one, I just assumed _meta_nonempty to be a safer choice, since the columns in it are guaranteed to show up in the result. However, it should never make a difference since datetime is supported in general now so _meta and _meta_nonempty should both include datetime columns. So I guess my question is the same as above for why to use each.

# If there's different types, just convert everything to NaNs for the time columns
if axis == 1 and len(time_cols) != len(self.columns):
# This is faster that converting each columns to numeric when it's not necessary
numeric_dd[time_cols.tolist()] = from_pandas(
Copy link
Collaborator

Choose a reason for hiding this comment

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

I don't think you need to tolist(), I think you can index with time_cols directly

# Single column always results in NaT
assert_eq(ddf[["dt1"]].std(axis=1), pdf[["dt1"]].std(axis=1))

# Mix of datetimes with other numeric types produces NaNs
Copy link
Collaborator

Choose a reason for hiding this comment

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

This would be a useful comment in the implementation, I was confused at first why you were creating a manual dataframe

Comment on lines +1617 to +1618
assert_eq(ddf2.std(axis=1, skipna=True), pdf2.std(axis=1, skipna=True))
assert_eq(ddf2.std(axis=1, skipna=False), pdf2.std(axis=1, skipna=False))
Copy link
Collaborator

Choose a reason for hiding this comment

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

I might suggest using pytest.parametrize over axis and skipna in these tests rather than repeating variations on assert_eq

# Convert timedelta and datetime columns to integer types so we can use var
for col in time_cols:
numeric_dd[col] = convert_to_numeric(numeric_dd[col])
else:
Copy link
Collaborator

Choose a reason for hiding this comment

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

An aside, and others might disagree with me here: I find this deeply nested branching kind of tough to read. Specifically: what "if" block is this attached to? Obviously I can scroll up and down, and some editor support can help here, but to me it might help with legibility to add some small comments like

      else:   # not is_df_like

Copy link
Member

Choose a reason for hiding this comment

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

I'm wondering if this might actually be more legible if the conditionals were collapsed, so it would be:

if PANDAS_GT_120 and is_df_like:
    ....
elif PANDAS_GT_120 and not is_df_like:
    ...

@ian-r-rose
Copy link
Collaborator

Also, sorry for the delay on the review!

@bglossner
Copy link
Contributor Author

bglossner commented Jan 28, 2022

Is there a good way to rerun the tests without another commit (although I don't think any of the tests are failing from my change)? Also, before it gets merged, do I need to rebase everything into a single commit in a different branch?

Let me know what you think of the new changes, thanks!

Copy link
Member

@jsignell jsignell left a comment

Choose a reason for hiding this comment

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

I left a few small comments, but to answer your question, no you don't have to merge main or tidy up your commits, we will squash-merge when this is done so it all ends up as on commit anyways. If you want to trigger CI you can just push an empty commit, but our tests have been having some chronic issues (particularly windows 3.7) so as long as the PR doesn't introduce new failures, we can still merge it.

# Convert timedelta and datetime columns to integer types so we can use var
for col in time_cols:
numeric_dd[col] = convert_to_numeric(numeric_dd[col])
else:
Copy link
Member

Choose a reason for hiding this comment

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

I'm wondering if this might actually be more legible if the conditionals were collapsed, so it would be:

if PANDAS_GT_120 and is_df_like:
    ....
elif PANDAS_GT_120 and not is_df_like:
    ...

)

if is_df_like:
result = result.astype(meta.dtype)
Copy link
Member

Choose a reason for hiding this comment

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

This feels a little strange to me. Is meta guaranteed to have a dtype?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not sure :/. Since this function is @derived_from(pd.DataFrame), is the dtype from it not included? meta is created from _meta_nonempty._std() which I think outputs a Series? I am honestly still not sure how to get the resulting dtypes to match other than to do this.

One thing I can do is check if meta has the dtype attribute at least before doing this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This has been included in the new commit!

@jsignell
Copy link
Member

jsignell commented Feb 1, 2022

Hmm I just tried this out locally and I ran into an issue. I suspect it is because _convert_to_numeric returns an Array, and maybe this is coming up because I am trying to use an Index rather than a Series

import dask

ddf = dask.datasets.timeseries(freq="1H")
ddf.index.std()
---------------------------------------------------------------------------
TypeError                                 Traceback (most recent call last)
Input In [5], in <module>
----> 1 ddf.index.std()

File ~/dask/dask/dataframe/core.py:96, in _numeric_only.<locals>.wrapper(self, *args, **kwargs)
     94 elif kwargs.get("numeric_only") is True:
     95     self = self._get_numeric_data()
---> 96 return func(self, *args, **kwargs)

File ~/dask/dask/dataframe/core.py:2214, in _Frame.std(self, axis, skipna, ddof, split_every, dtype, out, numeric_only)
   2211     return handle_out(out, result)
   2213 # Case where axis=0 or axis=None
-> 2214 v = numeric_dd.var(skipna=skipna, ddof=ddof, split_every=split_every)
   2215 name = self._token_prefix + "std"
   2217 if needs_time_conversion:

TypeError: var() got an unexpected keyword argument 'skipna'

@bglossner
Copy link
Contributor Author

bglossner commented Feb 1, 2022

Hmm I just tried this out locally and I ran into an issue. I suspect it is because _convert_to_numeric returns an Array, and maybe this is coming up because I am trying to use an Index rather than a Series

import dask

ddf = dask.datasets.timeseries(freq="1H")
ddf.index.std()

Oof. Testing this out and it's getting messy. An Index is not series like or dataframe like (at least returns False for is_series_like). Because we get a Dask Array back, skipna can't be used. This is fine and we can just have an if/else to cover this case. The next problem occurs in map_partitions, where since Dask Arrays are not replicas of Pandas objects we get a big complaint:

File "c:\users\beng\desktop\college school\csc_369\dask\dask\dataframe\core.py", line 6138, in map_partitions
raise ValueError(
ValueError: Meta is not valid, map_partitions expects output to be a pandas object. Try passing a pandas object as meta or a
dict or tuple representing the (name, dtype) of the columns.

My thinking here is that we can either convert things that are arraylike() to a series inside the conversion to a numeric and run through the computation, since it will output the same thing OR we can skip the map_partitions call with another if/else since index.std() will be a scalar once computed.

One other thing to consider here is that perhaps we should just treat index.std() as calls to dask array std() instead. Because dask array std() still doesn't support datetimes, another task would be made to add support there where this would be done. I am not sure if this would involve changing the derived_from or adding another decorator or just simply converting it to a Dask Array and calling std() on it.

Thoughts?

@jsignell
Copy link
Member

jsignell commented Feb 2, 2022

I actually think it might be a bug in pandas that makes view return an array rather than an index object. But I tried changing from view to astype to see how far that would get us for the index case. It doesn't get us very far because there is no Index.var in pandas. So I think you should just raise a NotImplementedError when trying to do the dd.Index.std.

@jsignell jsignell added feature Something is missing needs review Needs review from a contributor. labels Feb 4, 2022
@bglossner
Copy link
Contributor Author

Sorry, not sure if this is waiting on changes from me or another reviewer based on the new label. I pushed some changes for the NotImplemented issue before those labels were added.

@jsignell
Copy link
Member

Thanks for the ping @bglossner! This is just waiting on me or @ian-r-rose to give it a last look over. I am expecting it to get in this week.

@jsignell
Copy link
Member

ok to test

Copy link
Member

@jsignell jsignell left a comment

Choose a reason for hiding this comment

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

This looks good! I just have one idea about how to tidy up the pre-existing test. Let me know if you want me to just push to your branch @bglossner

@jsignell jsignell removed the needs review Needs review from a contributor. label Feb 24, 2022
@jsignell jsignell merged commit 54e2164 into dask:main Feb 24, 2022
@jsignell
Copy link
Member

jsignell commented Feb 24, 2022

Thanks @bglossner - this is in!!

phobson pushed a commit to phobson/dask that referenced this pull request Feb 28, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

dataframe feature Something is missing

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Pandas 1.2.0 compatibility - std() for the datetime64[ns] dtype

4 participants