Open luxcem opened 6 months ago
Yes, the npartition=1 is a shortcut to avoid computing the quantiles, which is a huge performance pain in most cases. Is there a scenario where you need the divisions there?
@luxcem can you explain why you are interested in divisions in this example? Dask itself won't use them internally as soon as we're on a single partitioned dataframe since the algorithms for single partitions don't require divisions. Therefore, with query planning we are not calculating them. The legacy dataframe performs a possibly expensive computation to get them.
If you are interested in the min/max values, instead, I recommend doing dask.compute(ddf.index.min(), ddf.index.max())
instead of relying on the divisions.
Typically, I employ this approach with a variable npartitions
, which varies based on factors like data size or cluster availability. This value can be set to 1. Certain functions require the divisions
parameter to be defined. Failure to set divisions properly can lead to subsequent computational errors. Could I be approaching this incorrectly?
For instance, the compute
function invokes optimize
within _expr
, and this may potentially trigger an AssertionError
in : https://github.com/dask/dask-expr/blob/main/dask_expr/_expr.py#L497.
For instance, the compute function invokes optimize within _expr, and this may potentially trigger an
This is a bug and we would appreciate it if you could share a reproducer. We certainly don't want to trigger any exceptions just because divisions are not set. The optimizer must deal with this automatically.
Regarding the availability of divisions themselves, I would rather consider this a best effort attribute. We will not always guarantee this to be set with meaningful values and in the single partition case this is one of the cases that we choose to not set them. I recommend to not rely on this being set yourself.
Ok I'll work on a reproducer.
Another example is with repartition
after divisions
are lost, it will trigger an Exception in https://github.com/dask/dask-expr/blob/main/dask_expr/_repartition.py#L253
Thanks for working on a reproducer. I am curious to see where things are wrong. @fjetter is correct that we normally don't need divisions for one partition dfs, since we can work with them independently of divisions
DataFrame.divisions are lost when using
repartition
orset_index
withnpartitions == 1
Environment:
2024.3.0
3.12.2