-
-
Notifications
You must be signed in to change notification settings - Fork 27
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
DataFrame from read_parquet is slow after repartition (each new partition reads from many old partitions) #1181
Comments
Hi, thanks for your report. Dask-Expr combines multiple parquet files into a single partition to ensure reasonably sized partitions. Small partitions is generally bad for performance in Dask. You can disable this behavior if needed through setting
I would advise against that for real workload that are more than exploratory though, tiny partitions are pretty bad for performance and scalability |
Ah ok thanks ! Is there some documentation about this ? I'm trying to understand this better to share it with the community. In particular does this mean that running |
API docs is where this is documented: https://docs.dask.org/en/stable/generated/dask_expr.read_parquet.html#dask_expr.read_parquet Assuming your parquet files are tiny, df will be a single partition since we combine all parquet files into one and then
|
I observe something different. I don't get one single partition but one partition per parquet file: fs = fsspec.filesystem("memory")
pq.write_table(pa.table({"i": [0] * 10}), f"0.parquet", filesystem=fs)
pq.write_table(pa.table({"i": [1] * 10}), f"1.parquet", filesystem=fs)
import dask.dataframe as dd
df = dd.read_parquet("memory://*.parquet")
df.npartitions # 2 each partition corresponds to one parquet file: df.partitions[0].compute().to_dict(orient="records")
# [{'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}]
df.partitions[1].compute().to_dict(orient="records")
# [{'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}, {'i': 1}] but for some reason if I df.repartition(npartitions=5).head().to_dict(orient="records")
# [{'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}, {'i': 0}]
fs.delete("1.parquet")
df.repartition(npartitions=5).head().to_dict(orient="records")
# FileNotFoundError And I suspect this is linked to bad effects I'm getting when I run dask on HF Datasets where read_parquet + repartition returns a dataframe that is super slow because each partitions reads from many files instead of one. |
The partition squashing kicks in during optimization. Try
that will return 1 npartitions is generally an implementation detail with the query optimizer and you shouldn't rely on us not changing the value. This was exposed with the legacy implementation because it was too dumb to handle things automatically, but it's generally an abstraction leak |
hmmm df.optimize().npartitions
# 2
df.repartition(npartitions=5).optimize().npartitions
# 5 |
Yikes, sorry about the misinformation, that only happens with the arrow filesystem. There is some issue that's going on with partition pruning that's not working properly with a threaded scheduler. That is expected and something we are unlikely to fix. You should always create a local cluster to get good performance anyway. Could you try instantiating a client when reading from the hugging faces dataset with:
That will properly drop tasks that you don't need and only read from one file. |
Yesssss that works ! It solves the issue with the I'll make sure to mention this in HF docs in the future. May also I suggest to show users a warning when they are in such a case maybe ? |
This is something we've discussed here and there. The threaded scheduler is generally not a good idea for actual workloads (testing is fine), but using a proper client requires the distributed package. So there is a little bit of a tradeoff involved, pushing users more aggressively towards a proper cluster is something we want to do though |
I would argue that the threaded scheduler is not fine for testing because of this behavior 😬 |
Talking about unit tests for correctness, not performance |
I see ! Thanks for all the answers :) |
Describe the issue:
When I run
read_parquet
on several files, I get a DataFrame withoutdivisions
. The issue is: if I increasenpartitions
with.repartition()
, the resulting DataFrame has very slow partitions.For example, calling
head()
starts reading from many files instead of one. This is not the case when usingdask
withoutdask-expr
Minimal Complete Verifiable Example:
minimal reproducible example:
real world issue
Anything else we need to know?:
I need this to work correctly to showcase Dask to the Hugging Face community :)
There are >200k datasets on HF in Parquet format, and making partitions fast again can be a big improvement. E.g. a fast
.head()
can be a big plus for DX.Environment:
main
with 2024.12.1+8.g4fb8993c and 1.1.21+1.gedb6fd5)The text was updated successfully, but these errors were encountered: