Skip to content

[REVIEW] Use shallow copy for assign operations#5740

Merged
TomAugspurger merged 4 commits intodask:masterfrom
rjzamora:shallow-assign
Feb 19, 2020
Merged

[REVIEW] Use shallow copy for assign operations#5740
TomAugspurger merged 4 commits intodask:masterfrom
rjzamora:shallow-assign

Conversation

@rjzamora
Copy link
Member

@rjzamora rjzamora commented Dec 20, 2019

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:

Merge benchmark
--------------------------
Chunk-size  | 100000000
Frac-match  | 0.3
Ignore-size | 1.05 MB
Protocol    | ucx
Device(s)   | 0,1,2,3
rmm-pool    | True
tcp         | True
ib          | False
nvlink      | True
==========================
Total time  | 967.77 ms
Total time  | 826.82 ms
Total time  | 844.98 ms
==========================

Worker-Worker Transfer Rates
(w1,w2)     | 25% 50% 75% (total nbytes)
--------------------------
(00,01)     | 4.10 GB/s 4.26 GB/s 4.76 GB/s (7.20 GB)
(00,02)     | 4.69 GB/s 5.00 GB/s 8.64 GB/s (14.40 GB)
(00,03)     | 4.72 GB/s 5.06 GB/s 5.35 GB/s (7.20 GB)
(01,00)     | 4.09 GB/s 4.64 GB/s 4.84 GB/s (7.20 GB)
(01,02)     | 4.37 GB/s 5.16 GB/s 5.52 GB/s (9.60 GB)
(01,03)     | 4.29 GB/s 4.88 GB/s 5.57 GB/s (12.00 GB)
(02,00)     | 4.46 GB/s 4.87 GB/s 7.50 GB/s (12.00 GB)
(02,01)     | 4.12 GB/s 4.86 GB/s 5.48 GB/s (7.20 GB)
(02,03)     | 4.53 GB/s 4.60 GB/s 5.56 GB/s (7.20 GB)
(03,00)     | 4.31 GB/s 5.36 GB/s 6.17 GB/s (9.60 GB)
(03,01)     | 4.11 GB/s 5.65 GB/s 7.19 GB/s (14.40 GB)
(03,02)     | 4.37 GB/s 5.12 GB/s 5.18 GB/s (7.20 GB)

BEFORE Changes:

Merge benchmark
--------------------------
Chunk-size  | 100000000
Frac-match  | 0.3
Ignore-size | 1.05 MB
Protocol    | ucx
Device(s)   | 0,1,2,3
rmm-pool    | True
tcp         | True
ib          | False
nvlink      | True
==========================
Total time  | 1.59 s
Total time  | 1.85 s
Total time  | 1.79 s
==========================

Worker-Worker Transfer Rates
(w1,w2)     | 25% 50% 75% (total nbytes)
--------------------------
(00,01)     | 3.36 GB/s 4.20 GB/s 5.54 GB/s (6.50 GB)
(00,02)     | 4.16 GB/s 4.55 GB/s 4.84 GB/s (8.90 GB)
(00,03)     | 3.97 GB/s 5.10 GB/s 6.93 GB/s (11.30 GB)
(01,00)     | 3.74 GB/s 4.15 GB/s 5.54 GB/s (7.20 GB)
(01,02)     | 3.75 GB/s 4.73 GB/s 6.42 GB/s (10.30 GB)
(01,03)     | 4.30 GB/s 5.22 GB/s 6.56 GB/s (11.00 GB)
(02,00)     | 4.17 GB/s 4.76 GB/s 6.55 GB/s (9.60 GB)
(02,01)     | 4.78 GB/s 5.86 GB/s 6.68 GB/s (12.00 GB)
(02,03)     | 4.92 GB/s 5.76 GB/s 6.14 GB/s (7.20 GB)
(03,00)     | 3.60 GB/s 5.56 GB/s 7.38 GB/s (10.30 GB)
(03,01)     | 3.50 GB/s 4.68 GB/s 5.75 GB/s (9.60 GB)
(03,02)     | 4.39 GB/s 4.86 GB/s 6.59 GB/s (10.30 GB)

TODO (Reason for WIP status):

  • Address test failures

  • Tests added / passed

  • Passes black dask / flake8 dask

@rjzamora
Copy link
Member Author

Note: It appears that the current implementation is problematic for test_attribute_assignment:

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:

Before ddf Modified...
0    1.0
1    2.0
2    3.0
3    4.0
4    5.0
Name: y, dtype: float64

After ddf Modified...
0     2.0
1     4.0
2     6.0
3     8.0
4    10.0
Name: y, dtype: float64

@TomAugspurger
Copy link
Member

Does that issue only show up with attribute assignment or is it present with setitem too? Pandas strongly recommends against assigning to an attribute.

@mrocklin
Copy link
Member

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.

@TomAugspurger
Copy link
Member

@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 df

I 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.

@rjzamora
Copy link
Member Author

Thanks @TomAugspurger! I will give it a try. You have good timing - I was just about to revisit this :)

@rjzamora
Copy link
Member Author

rjzamora commented Feb 18, 2020

@TomAugspurger Thanks for the suggestion! I do think your clever work-around is exactly what we need :)

@rjzamora rjzamora changed the title [WIP] Use shallow copy for assign operations [REVIEW] Use shallow copy for assign operations Feb 18, 2020
@rjzamora rjzamora requested a review from mrocklin February 18, 2020 18:53
@TomAugspurger
Copy link
Member

Great to hear. Should we go ahead and merge this in? I think I've convinced myself that this is safe to do.

@rjzamora
Copy link
Member Author

I'm pretty confident this is safe to merge - I appreciate your help here @TomAugspurger !

@TomAugspurger TomAugspurger merged commit 474e274 into dask:master Feb 19, 2020
@mrocklin
Copy link
Member

mrocklin commented Feb 21, 2020 via email

@rjzamora rjzamora deleted the shallow-assign branch May 21, 2024 00:04
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Use shallow copies when assigning columns

3 participants