You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
As part of #335 we did some investigation to understand how we can improve our performance, and we came up with a simple broadcast merge to perform the left-anti join.
However that approach doesn't scale when right is greater than system memory (works fine when right is greater than device memory).
Followups
Fuzzy Duplication
We could explore reading dataset with add_filename=True even though we know that results in slower I/O.
However the slower IO will only be paid once at minhash time (which is ~20% of the whole pipeline runtime)
In subsequent LSH / Jaccard / Connected Components stages we use filename as a join key along with id_col
And in the end once we've identified duplicates, we'll know which file in the original set they belong to, and we can perform a file by file removal.
If this approach indeed works, we'll increase our identification runtime by a ~few % points, however removal should be faster and be able to complete given any dataset size, as we're not shuffling anything, just performing a file by file merge
Exact Deduplication
When we perform the hashing we anyway perform a shuffle followed by a map_partitions(lambda x : x[x["_hashes"].duplicated(keep=False)) to return all the set of duplicates.
Instead of keep=False, we can do keep=first which will allow us to return the first document in each "group" which is the same as identify and removal.
Broadcast merge in Dask, has different implementations for left vs inner, where one is much more parallelizable than the other.
A left join shuffles right (partition A,B become P,Q) and shuffles each partition of left (1,2,3 -> 1_1, 1_2 and 2_1, 2_2 and 3_2, 3_3) and then performs a concatenation of merge(1_1, P) and merge(1_2, Q) and ends up OOMing
We suspect an inner join could be faster, as it doesn't need to shuffle left or right, and can just merge(1, A) and merge(1, B) in parallel, and then in the end have a task to concat all 1's together. (in fact even this could be optimized if you don't need to concat and just write out merge(1, A)'s output.
However this still means, a 64tb dataset (~20bn rows) read with 1gb partitions will have 64000 partitions and it's set of duplicates (30% of dataset) will be 20bn * 30% * 30 bytes (for the join key) ~180gb, and when read with ~1gb partitions, will also be 180 partitions resulting in a very large task graph of 11.5million tasks
More so in practice, it is observe that for both left and inner join we're still spilling a lot during the cudf.merge even though from my understanding each merge should be only between 1 and P (which in example above are assumed to be 1gb each), but it might be that P and Q live on the same worker / graph isn't evaluated until the concat is happening in which case they're evaluated on the same worker
In left it just spills so much so soon during the graph that we reach ~80% of system memory result in dask to pause the worker and get stuck in loop.
In inner, while in theory it shouldn't have spilled as each time we're just merging 1, A and 1, B we still observe a lot of time during the profile being spent in the spilling.
This problem seems to be only at "scale" and not for smaller datasets
Conclusion
Our analysis above showed that there is huge memory requirement for large merges, and to timebox this effort we didn't dive any deeper into merge performance at dask / cudf level. And pushed #509 which works for the ~most dataset sizes and is an improvement over our current implementation of compute. For datasets larger where duplicates are > system memory, we should try an approach indicated above, or users can perform removal at a partition level (e.g your dataset is partitioned by year, and each id_col has a year prefix so you know which id needs to be removed from which partition after the identify step)
Broadcast Inner Join
After 1 hour on 8 H100s
~96% of time being spent in merge, where almost all of it is spilling
As part of #335 we did some investigation to understand how we can improve our performance, and we came up with a simple broadcast merge to perform the left-anti join.
However that approach doesn't scale when right is greater than system memory (works fine when right is greater than device memory).
Followups
add_filename=True
even though we know that results in slower I/O.filename
as a join key along with id_colhashing
we anyway perform a shuffle followed by amap_partitions(lambda x : x[x["_hashes"].duplicated(keep=False))
to return all the set of duplicates.keep=False
, we can dokeep=first
which will allow us to return the first document in each "group" which is the same as identify and removal.Analysis from #335
A,B
becomeP,Q
) and shuffles each partition of left (1,2,3
->1_1, 1_2
and2_1, 2_2
and3_2, 3_3
) and then performs aconcat
enation ofmerge(1_1, P) and merge(1_2, Q)
and ends up OOMingmerge(1, A)
andmerge(1, B)
in parallel, and then in the end have a task toconcat
all 1's together. (in fact even this could be optimized if you don't need to concat and just write out merge(1, A)'s output.left
andinner
join we're still spilling a lot during thecudf.merge
even though from my understanding each merge should be only between 1 and P (which in example above are assumed to be 1gb each), but it might be that P and Q live on the same worker / graph isn't evaluated until theconcat
is happening in which case they're evaluated on the same worker1, A
and1, B
we still observe a lot of time during the profile being spent in the spilling.Conclusion
Our analysis above showed that there is huge memory requirement for large merges, and to timebox this effort we didn't dive any deeper into merge performance at dask / cudf level. And pushed #509 which works for the ~most dataset sizes and is an improvement over our current implementation of
compute
. For datasets larger where duplicates are > system memory, we should try an approach indicated above, or users can perform removal at a partition level (e.g your dataset is partitioned by year, and eachid_col
has ayear
prefix so you know whichid
needs to be removed from which partition after the identify step)Broadcast Inner Join
After 1 hour on 8 H100s
~96% of time being spent in merge, where almost all of it is spilling
Task Graphs
Code Logic
Left Join
Inner Join
The text was updated successfully, but these errors were encountered: