Skip to content

Try to make divisions behavior clearer#8379

Merged
jsignell merged 5 commits intodask:mainfrom
jsignell:divisions
Nov 17, 2021
Merged

Try to make divisions behavior clearer#8379
jsignell merged 5 commits intodask:mainfrom
jsignell:divisions

Conversation

@jsignell
Copy link
Member

I think this makes things clearer and it makes the example runnable without being huge.

Copy link
Collaborator

@gjoseph92 gjoseph92 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @jsignell! This does help clarify things quite a lot.

I have one other suggestion I couldn't comment on the lines for (noting that npartitions is ignored when divisions is given):

        npartitions: int, None, or 'auto'
            The ideal number of output partitions. If None, use the same as
            the input. If 'auto' then decide by memory use.
            Only used when ``divisions`` is not given. If ``divisions`` is given,
            the number of output partitions will be ``len(divisions) - 1``.

Comment on lines +4342 to +4351
>>> divisions = pd.date_range(start="2021-01-01", end="2021-01-07", freq='1D')
... divisions
DatetimeIndex(['2021-01-01', '2021-01-02', '2021-01-03', '2021-01-04',
'2021-01-05', '2021-01-06', '2021-01-07'],
dtype='datetime64[ns]', freq='D')

Note that ``len(divisons)`` is equal to ``npartitions + 1``. This is because ``divisions``
represents the upper and lower bounds of each partition.

>>> ddf2 = ddf.set_index("timestamp", sorted=True, divisions=divisions)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if the example would be easier to read if we used the name column and wrote the divisions by hand:

        >>> divisions = ["Alice", "Frank", "Laura", "Quinn", "Ursula", "Zelda"]
        >>> ddf2 = ddf.set_index("name", divisions=divisions)

This doesn't illustrate sorted=True, but it does show a probably-more-common case of using a different unsorted column, and that writing divisions yourself is not that scary.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah I like that idea.

@@ -4310,7 +4310,7 @@ def set_index(
See https://docs.dask.org/en/latest/dataframe-design.html#partitions
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

        divisions: list, optional
            The "dividing lines" used to split the new index into partitions.
            For ``divisions=[0, 10, 50, 100]``, there would be three output partitions,
            where the new index contained [0, 10), [10, 50), and [50, 100), respectively.
            See https://docs.dask.org/en/latest/dataframe-design.html#partitions.
            If not given (default), good divisions are calculated by immediately computing
            the data and looking at the distribution of its values. For large datasets,
            this can be expensive.
            Note that if ``sorted=True``, specified divisions are assumed to match
            the existing partitions in the data; if this is untrue you should
            leave divisions empty and call ``repartition`` after ``set_index``.

Something like this might make the divisions parameter easier to understand for me, don't know if it would help anyone else?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Improve documentation of DataFrame.divisions

2 participants