diff --git a/dask_sql/input_utils/dask.py b/dask_sql/input_utils/dask.py index 2da11e701..4dbeeef63 100644 --- a/dask_sql/input_utils/dask.py +++ b/dask_sql/input_utils/dask.py @@ -4,11 +4,6 @@ from dask_sql.input_utils.base import BaseInputPlugin -try: - import dask_cudf -except ImportError: - dask_cudf = None - class DaskInputPlugin(BaseInputPlugin): """Input Plugin for Dask DataFrames, just keeping them""" @@ -27,7 +22,9 @@ def to_dc( **kwargs ): if gpu: # pragma: no cover - if not dask_cudf: + try: + import dask_cudf + except ImportError: raise ModuleNotFoundError( "Setting `gpu=True` for table creation requires dask_cudf" ) diff --git a/dask_sql/input_utils/location.py b/dask_sql/input_utils/location.py index 8cb23a444..e1db51e6b 100644 --- a/dask_sql/input_utils/location.py +++ b/dask_sql/input_utils/location.py @@ -7,11 +7,6 @@ from dask_sql.input_utils.base import BaseInputPlugin from dask_sql.input_utils.convert import InputUtil -try: - import dask_cudf -except ImportError: - dask_cudf = None - class LocationInputPlugin(BaseInputPlugin): """Input Plugin for everything, which can be read in from a file (on disk, remote etc.)""" @@ -44,7 +39,9 @@ def to_dc( format = extension.lstrip(".") try: if gpu: # pragma: no cover - if not dask_cudf: + try: + import dask_cudf + except ImportError: raise ModuleNotFoundError( "Setting `gpu=True` for table creation requires dask-cudf" ) diff --git a/dask_sql/input_utils/pandaslike.py b/dask_sql/input_utils/pandaslike.py index 32d7ff5ea..7c3b66de0 100644 --- a/dask_sql/input_utils/pandaslike.py +++ b/dask_sql/input_utils/pandaslike.py @@ -3,11 +3,6 @@ from dask_sql.input_utils.base import BaseInputPlugin -try: - import cudf -except ImportError: - cudf = None - class PandasLikeInputPlugin(BaseInputPlugin): """Input Plugin for Pandas Like DataFrames, which get converted to dask DataFrames""" @@ -30,7 +25,9 @@ def to_dc( ): npartitions = kwargs.pop("npartitions", 1) if gpu: # pragma: no cover - if not cudf: + try: + import cudf + except ImportError: raise ModuleNotFoundError( "Setting `gpu=True` for table creation requires cudf" ) diff --git a/dask_sql/physical/rel/logical/aggregate.py b/dask_sql/physical/rel/logical/aggregate.py index cae4e42ed..0dc3bc926 100644 --- a/dask_sql/physical/rel/logical/aggregate.py +++ b/dask_sql/physical/rel/logical/aggregate.py @@ -8,11 +8,6 @@ import pandas as pd from dask import config as dask_config -try: - import dask_cudf -except ImportError: - dask_cudf = None - from dask_sql.datacontainer import ColumnContainer, DataContainer from dask_sql.physical.rel.base import BaseRelPlugin from dask_sql.physical.rex.core.call import IsNullOperation @@ -83,7 +78,7 @@ def get_supported_aggregation(self, series): if pd.api.types.is_string_dtype(series.dtype): # If dask_cudf strings dtype, return built-in aggregation - if dask_cudf is not None and isinstance(series, dask_cudf.Series): + if "cudf" in str(series._partition_type): return built_in_aggregation # With pandas StringDtype built-in aggregations work diff --git a/dask_sql/physical/utils/sort.py b/dask_sql/physical/utils/sort.py index 0e4cc9d85..c2ccce3c1 100644 --- a/dask_sql/physical/utils/sort.py +++ b/dask_sql/physical/utils/sort.py @@ -6,11 +6,6 @@ from dask_sql.utils import make_pickable_without_dask_sql -try: - import dask_cudf -except ImportError: - dask_cudf = None - def apply_sort( df: dd.DataFrame, @@ -35,10 +30,7 @@ def apply_sort( # 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 single_ascending - and single_null_first + "cudf" in str(df._partition_type) and single_ascending and single_null_first ): try: return df.sort_values( diff --git a/tests/integration/test_create.py b/tests/integration/test_create.py index 456435b7e..ef8f82fb4 100644 --- a/tests/integration/test_create.py +++ b/tests/integration/test_create.py @@ -363,3 +363,34 @@ def test_drop(c): with pytest.raises(dask_sql.utils.ParsingException): c.sql("SELECT a FROM new_table") + + +def test_create_gpu_error(c, df, temporary_data_file): + try: + import cudf + except ImportError: + cudf = None + + if cudf is not None: + pytest.skip("GPU-related import errors only need to be checked on CPU") + + with pytest.raises(ModuleNotFoundError): + c.create_table("new_table", df, gpu=True) + + with pytest.raises(ModuleNotFoundError): + c.create_table("new_table", dd.from_pandas(df, npartitions=2), gpu=True) + + df.to_csv(temporary_data_file, index=False) + + with pytest.raises(ModuleNotFoundError): + c.sql( + f""" + CREATE TABLE + new_table + WITH ( + location = '{temporary_data_file}', + format = 'csv', + gpu = True + ) + """ + )