From 21d433bd88a173d7c2333ebc6f20b6b017244006 Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Wed, 1 Dec 2021 11:47:29 -0500 Subject: [PATCH 1/2] Use custom function in sort_values for complex sort ops [test-upstream] --- dask_sql/physical/utils/sort.py | 26 ++++++++++---------------- 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/dask_sql/physical/utils/sort.py b/dask_sql/physical/utils/sort.py index 46b72b6c8..fd4c5d734 100644 --- a/dask_sql/physical/utils/sort.py +++ b/dask_sql/physical/utils/sort.py @@ -41,33 +41,27 @@ def apply_sort( except ValueError: pass - # Dask doesn't natively support multi-column sorting; - # we work around this by initially sorting by the first - # column then handling the rest with `map_partitions` - df = df.sort_values( + # if standard `sort_values` can't handle ascending / null position params, + # we extend it using our custom sort function + return df.sort_values( by=sort_columns[0], ascending=sort_ascending[0], na_position="first" if sort_null_first[0] else "last", + sort_function=make_pickable_without_dask_sql(sort_partition_func), + sort_function_kwargs={ + "sort_columns": sort_columns, + "sort_ascending": sort_ascending, + "sort_null_first": sort_null_first, + }, ).persist() - # sort the remaining columns if given - if len(sort_columns) > 1: - df = df.map_partitions( - make_pickable_without_dask_sql(sort_partition_func), - meta=df, - sort_columns=sort_columns, - sort_ascending=sort_ascending, - sort_null_first=sort_null_first, - ).persist() - - return df - def sort_partition_func( partition: pd.DataFrame, sort_columns: List[str], sort_ascending: List[bool], sort_null_first: List[bool], + **kwargs, ): if partition.empty: return partition From df5994e490ce149506ec260afe84bd712ea2c66b Mon Sep 17 00:00:00 2001 From: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> Date: Fri, 14 Jan 2022 16:07:14 -0500 Subject: [PATCH 2/2] [test-upstream] just to be safe