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

Random values/wrong type when grouping a filtered DataFrame by an unfiltered column #1876

Closed
p-himik opened this issue Jan 1, 2017 · 4 comments
Milestone

Comments

@p-himik
Copy link
Contributor

p-himik commented Jan 1, 2017

The issue has been discovered while working through 03a-DataFrame.ipynb notebook of the tutorial.
There we have a Dask DataFrame df.

real_df = df.compute()
# otherwise groupby issues "ValueError: cannot reindex from a duplicate axis"
real_df.index = range(len(real_df))

### using pandas
print(real_df[real_df.amount < 0].groupby(real_df.id).amount.mean())
# id
# 0     -1532.021665
# 2       -46.787321
# 4        -5.000000
# 8       -83.500000
# 11     -116.254859

### grouping by unfiltered column
# this doesn't issue any error for duplicate index items
print(df[df.amount < 0].groupby(df.id).amount.mean().compute())  
# id
# 0.0     -148.489168
# 2.0       -4.333333
# 4.0             NaN
# 8.0      -13.166667
# 11.0      -9.701362

### grouping by filtered column
print(df[df.amount < 0].groupby(df[df.amount < 0].id).amount.mean().compute())
# id
# 0     -1532.021665
# 2       -46.787321
# 4        -5.000000
# 8       -83.500000
# 11     -116.254859

As you can see, for some reason in Dask if you use unfiltered column for groupby, you don't receive any ValueError, you end up with float64 column type instead of int64 for the groupby column, and the values are very different from the ones obtained with pandas. In fact, they are different each time you run the code.

@mrocklin
Copy link
Member

mrocklin commented Jan 2, 2017

Thanks for the error report @p-himik !

Can I ask you to do a bit more and provide a reproducible example? Ideally this would be simple code that someone could copy-paste to easily get this same error.

@p-himik
Copy link
Contributor Author

p-himik commented Jan 2, 2017

import numpy as np
import pandas as pd

import dask.dataframe as df

NAMES = ['Alice', 'Bob', 'Charlie', 'Dan', 'Edith', 'Frank', 'George',
         'Hannah', 'Ingrid', 'Jerry', 'Kevin', 'Laura', 'Michael', 'Norbert', 'Oliver',
         'Patricia', 'Quinn', 'Ray', 'Sarah', 'Tim', 'Ursula', 'Victor', 'Wendy',
         'Xavier', 'Yvonne', 'Zelda']


def account_params(k):
    ids = np.arange(k, dtype=int)
    names2 = np.random.choice(NAMES, size=k, replace=True)
    wealth_mag = np.random.exponential(100, size=k)
    wealth_trend = np.random.normal(10, 10, size=k)
    freq = np.random.exponential(size=k)
    freq /= freq.sum()

    return ids, names2, wealth_mag, wealth_trend, freq


def account_entries(n, ids, names, wealth_mag, wealth_trend, freq):
    indices = np.random.choice(ids, size=n, replace=True, p=freq)
    amounts = (np.random.normal(size=n) + wealth_trend[indices]) * wealth_mag[indices]

    return pd.DataFrame({'id': indices,
                         'names': names[indices],
                         'amount': amounts.astype('i4')},
                        columns=['id', 'names', 'amount'])


def accounts_dataframes(num_dfs, n, k):
    args = account_params(k)

    return [account_entries(n, *args) for _ in range(num_dfs)]


# interesting - n=1000, k=50 does produce the correct result for each invocation
p_dfs = accounts_dataframes(3, 1000000, 500)

d_df = df.concat([df.from_pandas(d, npartitions=1, sort=False) for d in p_dfs])

p_df = d_df.compute()
p_df.index = range(len(p_df))

print(p_df[p_df.amount < 0].groupby(p_df.id).amount.mean().head())
print(d_df[d_df.amount < 0].groupby(d_df.id).amount.mean().head())
print(d_df[d_df.amount < 0].groupby(d_df[d_df.amount < 0].id).amount.mean().head())

@jcrist
Copy link
Member

jcrist commented Jan 27, 2017

Apologies for letting this sit so long. This appears to be a bug in pandas, see pandas-dev/pandas#15244.

The issue seems isolated to only grouping by unaligned indices. (p_df[p_df.amount < 0] and p_df.id have different indices). As such, you can avoid this bug by doing the filter beforehand (as you've noticed above).

In the long run, it'd be best to get this fixed in pandas. For now, we may be able to avoid this ourselves by aligning before the call to groupby. I'll look into this.

jcrist added a commit to jcrist/dask that referenced this issue Jan 27, 2017
Pandas allows grouping by a key that isn't aligned with the grouped
DataFrame/Series. If this happens, then groupby first aligns the indices
before performing the grouping. Unfortunately this operation isn't
threadsafe, and can lead to incorrect results. Since grouping by an
unaligned key isn't recommended, we just error loudly in these cases.

Fixes dask#1876.
@jcrist
Copy link
Member

jcrist commented Jan 27, 2017

Actually, using align doesn't work either, as it exhibits the same behavior. Since passing in unaligned indices to groupby should be considered an antipattern, and can always be avoided (in your case by doing df[df.amount < 0].groupby('id').amount.mean()), we just error in these cases. See fix in #1941.

jcrist added a commit that referenced this issue Jan 31, 2017
Pandas allows grouping by a key that isn't aligned with the grouped
DataFrame/Series. If this happens, then groupby first aligns the indices
before performing the grouping. Unfortunately this operation isn't
threadsafe, and can lead to incorrect results. Since grouping by an
unaligned key isn't recommended, we just error loudly in these cases.

Fixes #1876.
@sinhrks sinhrks added this to the 0.14.0 milestone Mar 4, 2017
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

4 participants