diff --git a/dask_expr/_collection.py b/dask_expr/_collection.py index 88681dd1..7d9bce24 100644 --- a/dask_expr/_collection.py +++ b/dask_expr/_collection.py @@ -14,7 +14,7 @@ import numpy as np import pandas as pd import pyarrow as pa -from dask import compute, config, get_annotations +from dask import compute, get_annotations from dask.array import Array from dask.base import DaskMethodsMixin, is_dask_collection, named_schedulers from dask.core import flatten @@ -5099,6 +5099,7 @@ def from_dask_array(x, columns=None, index=None, meta=None): return from_legacy_dataframe(df, optimize=True) +@dataframe_creation_dispatch.register_inplace("pandas") def read_csv( path, *args, @@ -5109,7 +5110,6 @@ def read_csv( ): from dask_expr.io.csv import ReadCSV - dataframe_backend = config.get("dataframe.backend", "pandas") if not isinstance(path, str): path = stringify_path(path) return new_collection( @@ -5119,7 +5119,7 @@ def read_csv( storage_options=storage_options, kwargs=kwargs, header=header, - dataframe_backend=dataframe_backend, + dataframe_backend="pandas", ) ) @@ -5174,6 +5174,7 @@ def read_fwf( ) +@dataframe_creation_dispatch.register_inplace("pandas") def read_parquet( path=None, columns=None,