-
Notifications
You must be signed in to change notification settings - Fork 70
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
Increase speed and parallelism of the limit algorithm and implement descending sorting #75
Merged
Merged
Changes from 3 commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
8f76f49
Increase speed and parallelism of the limit algorithm
nils-braun 6d5ac2c
Fixed docs
nils-braun 49d969c
Implement descending sorting. Fixes #10
nils-braun 8fd9220
Replace two delayed usages. Thanks @mrocklin
nils-braun 75eea3d
Merge branch 'main' into feature/faster-limiting
nils-braun b98baf2
Remoe the reference to the function - to make it usable without dask-…
nils-braun 2bc8920
Merge branch 'main' into feature/faster-limiting
nils-braun File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,8 +1,7 @@ | ||
from typing import Dict, List | ||
from typing import List | ||
|
||
import dask | ||
import dask.dataframe as dd | ||
from dask.highlevelgraph import HighLevelGraph | ||
from dask.dataframe.core import new_dd_object | ||
import pandas as pd | ||
import dask.array as da | ||
|
||
|
@@ -62,11 +61,6 @@ def _apply_sort( | |
first_sort_column = sort_columns[0] | ||
first_sort_ascending = sort_ascending[0] | ||
|
||
# Sort the first column with set_index. Currently, we can only handle ascending sort | ||
if not first_sort_ascending: | ||
raise NotImplementedError( | ||
"The first column needs to be sorted ascending (yet)" | ||
) | ||
# We can only sort if there are no NaNs or infs. | ||
# Therefore we need to do a single pass over the dataframe | ||
# to warn the user | ||
|
@@ -79,6 +73,12 @@ def _apply_sort( | |
raise ValueError("Can not sort a column with NaNs") | ||
|
||
df = df.set_index(first_sort_column, drop=False).reset_index(drop=True) | ||
if not first_sort_ascending: | ||
# As set_index().reset_index() always sorts ascending, we need to reverse | ||
# the order inside all partitions and the order of the partitions itself | ||
df = dd.from_delayed( | ||
reversed([self.reverse_partition(p) for p in df.to_delayed()]) | ||
) | ||
|
||
# sort the remaining columns if given | ||
if len(sort_columns) > 1: | ||
|
@@ -94,8 +94,7 @@ def _apply_offset(self, df: dd.DataFrame, offset: int, end: int) -> dd.DataFrame | |
Limit the dataframe to the window [offset, end]. | ||
That is unfortunately, not so simple as we do not know how many | ||
items we have in each partition. We have therefore no other way than to | ||
calculate (!!!) the sizes of each partition | ||
(this means we need to compute the dataframe already here). | ||
calculate (!!!) the sizes of each partition. | ||
|
||
After that, we can create a new dataframe from the old | ||
dataframe by calculating for each partition if and how much | ||
|
@@ -104,24 +103,32 @@ def _apply_offset(self, df: dd.DataFrame, offset: int, end: int) -> dd.DataFrame | |
we need to pass the partition number to the selection | ||
function, which is not possible with normal "map_partitions". | ||
""" | ||
# As we need to calculate the partition size, we better persist | ||
# the df. I think... | ||
# TODO: check if this is the best thing to do | ||
df = df.persist() | ||
if not offset: | ||
# We do a (hopefully) very quick check: if the first partition | ||
# is already enough, we will just ust this | ||
first_partition_length = df.map_partitions(lambda x: len(x)).to_delayed()[0] | ||
first_partition_length = first_partition_length.compute() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. len(df.partitions[0]) |
||
if first_partition_length >= end: | ||
return df.head(end, compute=False) | ||
|
||
# First, we need to find out which partitions we want to use. | ||
# Therefore we count the total number of entries | ||
partition_borders = df.map_partitions(lambda x: len(x)).compute() | ||
partition_borders = partition_borders.cumsum().to_dict() | ||
partition_borders = df.map_partitions(lambda x: len(x)) | ||
|
||
# Now we let each of the partitions figure out, how much it needs to return | ||
# using these partition borders | ||
# For this, we generate out own dask computation graph (as it does not really) | ||
# fit well with one of the already present methods | ||
# For this, we generate out own dask computation graph (as it does not really | ||
# fit well with one of the already present methods). | ||
|
||
# (a) we define a method to be calculated on each partition | ||
# This method returns the part of the partition, which falls between [offset, fetch] | ||
def select_from_to(df, partition_index): | ||
# Please note that the dask object "partition_borders", will be turned into | ||
# its pandas representation at this point and we can calculate the cumsum | ||
# (which is not possible on the dask object). Recalculating it should not cost | ||
# us much, as we assume the number of partitions is rather small. | ||
@dask.delayed | ||
def select_from_to(df, partition_index, partition_borders): | ||
partition_borders = partition_borders.cumsum().to_dict() | ||
this_partition_border_left = ( | ||
partition_borders[partition_index - 1] if partition_index > 0 else 0 | ||
) | ||
|
@@ -141,20 +148,17 @@ def select_from_to(df, partition_index): | |
|
||
return df.iloc[from_index:to_index] | ||
|
||
# Then we (b) define a task graph. It should calculate the function above on each of the partitions of | ||
# df (specified by (df._name, i) for each partition i). As an argument, we pass the partition_index. | ||
dask_graph_name = df._name + "-limit" | ||
dask_graph_dict = {} | ||
|
||
for partition_index in range(df.npartitions): | ||
dask_graph_dict[(dask_graph_name, partition_index)] = ( | ||
select_from_to, | ||
(df._name, partition_index), | ||
partition_index, | ||
) | ||
|
||
# We replace df with our new graph | ||
graph = HighLevelGraph.from_collections( | ||
dask_graph_name, dask_graph_dict, dependencies=[df] | ||
# (b) Now we just need to apply the function on every partition | ||
# We do this via the delayed interface, which seems the easiest one. | ||
return dd.from_delayed( | ||
[ | ||
select_from_to(partition, partition_number, partition_borders) | ||
for partition_number, partition in enumerate(df.to_delayed()) | ||
] | ||
) | ||
return new_dd_object(graph, dask_graph_name, df._meta, df.divisions) | ||
|
||
@staticmethod | ||
@dask.delayed | ||
def reverse_partition(partition): | ||
"""Small helper function to revert a partition in dask""" | ||
return partition[::-1] |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You may want to become familiar with the
df.partitions
iterable.Going between delayed and dataframes introduces some inconvenience in graph-handling.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Uh nice! That looks awesome. Yes, this is definitely much better.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
At some point I think that we'll want to go through and remove all of the
to/from_delayed
calls. This isn't critical, but it's nice to do from a performance standpoint (happy to go into this in more depth if you like).There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your advice @mrocklin! Really appreciated. I would actually be really interested in your thoughts about this part.
Probably you see it right away, but what I would like to achieve is basically two things
partition_borders
only when executing the DAG - not already when creating it (this is what I have done before this PR, and it slows down the process).So far, the
delayed
function was all I could come up with, but I definitely think there is a better way.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have created an issue to fix this, so we can go on with this PR.