diff --git a/.gitignore b/.gitignore index 1a0f3faf1..f9d8eef88 100644 --- a/.gitignore +++ b/.gitignore @@ -31,6 +31,7 @@ htmlcov/ coverage.xml *.cover .pytest_cache/ +.hypothesis/ # Jupyter Notebook .ipynb_checkpoints diff --git a/dask_sql/physical/utils/sort.py b/dask_sql/physical/utils/sort.py index 46b72b6c8..5022b5cd4 100644 --- a/dask_sql/physical/utils/sort.py +++ b/dask_sql/physical/utils/sort.py @@ -18,8 +18,14 @@ def apply_sort( sort_ascending: List[bool], sort_null_first: List[bool], ) -> dd.DataFrame: - # if we have a single partition, we can sometimes sort with map_partitions - if df.npartitions == 1 and (all(sort_null_first) or not any(sort_null_first)): + # when sort_values doesn't support lists of ascending / null + # position booleans, we can still do the sort provided that + # the list(s) are homogeneous: + single_ascending = len(set(sort_ascending)) == 1 + single_null_first = len(set(sort_null_first)) == 1 + + # pandas / cudf don't support lists of null positions + if df.npartitions == 1 and single_null_first: return df.map_partitions( M.sort_values, by=sort_columns, @@ -27,17 +33,20 @@ def apply_sort( na_position="first" if sort_null_first[0] else "last", ).persist() - # dask-cudf only supports ascending sort / nulls last: - # https://github.com/rapidsai/cudf/pull/9250 - # https://github.com/rapidsai/cudf/pull/9264 - if ( + # dask / dask-cudf don't support lists of ascending / null positions + if len(sort_columns) == 1 or ( dask_cudf is not None and isinstance(df, dask_cudf.DataFrame) - and all(sort_ascending) - and not any(sort_null_first) + and single_ascending + and single_null_first ): try: - return df.sort_values(sort_columns, ignore_index=True).persist() + return df.sort_values( + by=sort_columns, + ascending=sort_ascending[0], + na_position="first" if sort_null_first[0] else "last", + ignore_index=True, + ).persist() except ValueError: pass