-
Notifications
You must be signed in to change notification settings - Fork 915
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
[BUG] std incorrectly calculated in dask describe
calls for specific dataframe values
#7402
Comments
Just discovered that |
describe
calls for specific dataframe valuesdescribe
calls for specific dataframe values
The way how cudf/python/dask_cudf/dask_cudf/core.py Lines 390 to 425 in dcf949c
cudf/python/dask_cudf/dask_cudf/core.py Lines 353 to 362 in dcf949c
In [1]: import cudf, dask_cudf, dask.dataframe as dd, dask.array as da, pandas as pd, numpy as np
In [2]: gdf = dask_cudf.from_cudf(cudf.Series([9.39590082e+08, 9.39590082e+08, 9.39590082e+08]), npartitions=1)
In [3]: gdf.var().compute()
Out[3]: -192.0
In [4]: a = np.array([9.39590082e+08, 9.39590082e+08, 9.39590082e+08])
In [5]: a.var()
Out[5]: 0.0
In [6]: x = a.sum()
In [7]: x2 = (a ** 2).sum()
In [8]: n = len(a)
In [9]: (x2 / n)
Out[9]: 8.828295221927666e+17
In [10]: (x / n) ** 2
Out[10]: 8.828295221927667e+17
In [11]: (x2 / n) - (x / n) ** 2
Out[11]: -128.0 The final result of I believe both code pieces in |
cc @rjzamora (in case you have thoughts here 🙂) |
Thank you for investigating this @pentschev - dask_cudf is clearly using a naive algorithm for var/std that is very easy to parallelize. This approach is known to have numerical-stability issues (as you clearly demonstrated). Perhaps it makes sense to use the parallel version of Welford's online algorithm (Chan 79')? I seem to get good results when I do something like the following in def _local_var(x, skipna):
n = len(x)
avg = x.mean(skipna=skipna)
m2 = ((x - avg)**2).sum(skipna=skipna)
return n, avg, m2
def _aggregate_var(parts):
n, avg, m2 = parts[0]
for i in range(1, len(parts)):
n_a, avg_a, m2_a = n, avg, m2
n_b, avg_b, m2_b = parts[i]
n = n_a + n_b
avg = (n_a * avg_a + n_b * avg_b) / n
delta = avg_b - avg_a
m2 = m2_a + m2_b + delta ** 2 * n_a * n_b / n
return m2 / (n - 1)
dsk = {}
name = "var-" + tokenize(axis, skipna, ddof, split_every, dtype, out)
local_name = "local-" + name
num = self._get_numeric_data()
parts = []
for n in range(num.npartitions):
parts.append((local_name, n))
dsk[parts[-1]] = (_local_var, (num._name, n), skipna)
dsk[(name, 0)] = (_aggregate_var, parts)
graph = HighLevelGraph.from_collections(name, dsk, dependencies=[num])
return dd.core.new_dd_object(graph, name, meta, (None, None)) |
…std (#7453) Closes #7402 This PR improves the numerical stability of the `var` (and indirectly `std`) methods in `DataFrame` and `Series`. As discussed in #7402, the existing (naive) approach is problematic for large numbers with relatively small var/std. Note that follow-up work may be needed to improve the algorithm(s) in groupby. Authors: - Richard (Rick) Zamora (@rjzamora) Approvers: - William Hicks (@wphicks) - Peter Andreas Entschev (@pentschev) - Keith Kraus (@kkraus14) - @jakirkham URL: #7453
Describe the bug
When a dataframe is filled with specific values, the reported
std
output bydescribe
is NaNSteps/Code to reproduce bug
Expected behavior
The reported
std
should be 0.0, as we see from the Pandas output instead of NaNEnvironment overview (please complete the following information)
source build of branch-0.18
Environment details
Click here to see environment details
Additional context
I found this while working on tracking down another bug. I haven't done much to triage it, except confirming that this does not occur without Dask (e.g. on a direct call of
df.describe()
in the above.The text was updated successfully, but these errors were encountered: