Skip to content

Commit d037c82

Browse files
committed
Grab meta from arrow rather than dask.dataframe
1 parent 3fc1904 commit d037c82

File tree

1 file changed

+10
-8
lines changed

1 file changed

+10
-8
lines changed

dask_expr/io/parquet.py

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -448,12 +448,6 @@ def _filtered_task(self, i):
448448

449449
@staticmethod
450450
def to_pandas(t: pa.Table) -> pd.DataFrame:
451-
def types_mapper(pyarrow_dtype):
452-
if pyarrow_dtype == pa.string():
453-
return pd.StringDtype("pyarrow")
454-
if "decimal" in str(pyarrow_dtype) or "date32" in str(pyarrow_dtype):
455-
return pd.ArrowDtype(pyarrow_dtype)
456-
457451
df = t.to_pandas(
458452
use_threads=False,
459453
ignore_metadata=False,
@@ -504,6 +498,13 @@ def _simplify_up(self, parent):
504498
# return Literal(sum(_lengths))
505499

506500

501+
def types_mapper(pyarrow_dtype):
502+
if pyarrow_dtype == pa.string():
503+
return pd.StringDtype("pyarrow")
504+
if "decimal" in str(pyarrow_dtype) or "date32" in str(pyarrow_dtype):
505+
return pd.ArrowDtype(pyarrow_dtype)
506+
507+
507508
@functools.lru_cache
508509
def meta_and_filenames(path):
509510
if str(path).startswith("s3://"):
@@ -520,9 +521,10 @@ def meta_and_filenames(path):
520521
else:
521522
filenames = [path] # TODO: split by row group
522523

523-
import dask.dataframe as dd
524+
ds = pq.ParquetDataset(path)
525+
t = pa.Table.from_pylist([], schema=ds.schema)
526+
meta = t.to_pandas(types_mapper=types_mapper)
524527

525-
meta = dd.read_parquet(path)._meta
526528
return meta, filenames
527529

528530

0 commit comments

Comments
 (0)