[REVIEW] Use shallow copy for assign operations#5740
[REVIEW] Use shallow copy for assign operations#5740TomAugspurger merged 4 commits intodask:masterfrom
Conversation
|
Note: It appears that the current implementation is problematic for import pandas as pd
import dask.dataframe as dd
df = pd.DataFrame({"x": [1, 2, 3, 4, 5], "y": [1.0, 2.0, 3.0, 4.0, 5.0]})
ddf = dd.from_pandas(df, npartitions=2)
print("Before ddf Modified...\n", df.y)
ddf.y = ddf.x + ddf.y
ddf.persist()
print("\nAfter ddf Modified...\n", df.y)Output: |
|
Does that issue only show up with attribute assignment or is it present with setitem too? Pandas strongly recommends against assigning to an attribute. |
I don't think that we're using setattr internally. I think that it's more likely that this is what you were referring to earlier, that a shallow copy might not be enough to sufficiently protect the input dataframe. |
|
@rjzamora here's a potential workaround: only do a shallow copy when we're adding a new column: diff --git a/dask/dataframe/methods.py b/dask/dataframe/methods.py
index e3e393093..e09d8297d 100644
--- a/dask/dataframe/methods.py
+++ b/dask/dataframe/methods.py
@@ -263,8 +263,13 @@ def set_column(df, name, value):
def assign(df, *pairs):
- df = df.copy(deep=False)
- for name, val in dict(partition(2, pairs)).items():
+ pairs = dict(partition(2, pairs))
+ # deep copy when updating an element, to avoid issues
+ # with modifying the original
+ deep = bool(set(pairs) & set(df.columns))
+
+ df = df.copy(deep=bool(deep))
+ for name, val in pairs.items():
df[name] = val
return dfI think that avoids the behavior failing the test. Can you check if it retains the performance boost? Do people think this is likely to cause issues elsewhere? I'm having trouble reasoning through things just in my head. |
|
Thanks @TomAugspurger! I will give it a try. You have good timing - I was just about to revisit this :) |
|
@TomAugspurger Thanks for the suggestion! I do think your clever work-around is exactly what we need :) |
|
Great to hear. Should we go ahead and merge this in? I think I've convinced myself that this is safe to do. |
|
I'm pretty confident this is safe to merge - I appreciate your help here @TomAugspurger ! |
|
Woo!
…On Wed, Feb 19, 2020 at 9:27 AM Tom Augspurger ***@***.***> wrote:
Merged #5740 <#5740> into master.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#5740?email_source=notifications&email_token=AACKZTH3462RAKUEINHL6ADRDVTYFA5CNFSM4J6DOVJ2YY3PNVWWK3TUL52HS4DFWZEXG43VMVCXMZLOORHG65DJMZUWGYLUNFXW5KTDN5WW2ZLOORPWSZGOWXZ2TCI#event-3052644745>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTGU4GEITLOQV25KEEDRDVTYFANCNFSM4J6DOVJQ>
.
|
Closes #5739
This PR simply follows the suggestion made by @mrocklin in #5739 . Note that this simple change has a significant effect on the results of the cudf merge benchmark, which spends a significant amount of time performing a deep copy for a temporary column assignment:
AFTER Changes:
Worker-Worker Transfer Rates
BEFORE Changes:
Worker-Worker Transfer Rates
TODO (Reason for WIP status):
Address test failures
Tests added / passed
Passes
black dask/flake8 dask