File: dataframe-categoricals.rst

package info (click to toggle)
dask 2024.12.1%2Bdfsg-2
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 20,024 kB
  • sloc: python: 105,182; javascript: 1,917; makefile: 159; sh: 88
file content (105 lines) | stat: -rw-r--r-- 3,951 bytes parent folder | download | duplicates (2)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
Categoricals
============

Dask DataFrame divides `categorical data`_ into two types:

- Known categoricals have the ``categories`` known statically (on the ``_meta``
  attribute).  Each partition **must** have the same categories as found on the
  ``_meta`` attribute
- Unknown categoricals don't know the categories statically, and may have
  different categories in each partition.  Internally, unknown categoricals are
  indicated by the presence of ``dd.utils.UNKNOWN_CATEGORIES`` in the
  categories on the ``_meta`` attribute.  Since most DataFrame operations
  propagate the categories, the known/unknown status should propagate through
  operations (similar to how ``NaN`` propagates)

For metadata specified as a description (option 2 above), unknown categoricals
are created.

Certain operations are only available for known categoricals.  For example,
``df.col.cat.categories`` would only work if ``df.col`` has known categories,
since the categorical mapping is only known statically on the metadata of known
categoricals.

The known/unknown status for a categorical column can be found using the
``known`` property on the categorical accessor:

.. code-block:: python

    >>> ddf.col.cat.known
    False

Additionally, an unknown categorical can be converted to known using
``.cat.as_known()``.  If you have multiple categorical columns in a DataFrame,
you may instead want to use ``df.categorize(columns=...)``, which will convert
all specified columns to known categoricals.  Since getting the categories
requires a full scan of the data, using ``df.categorize()`` is more efficient
than calling ``.cat.as_known()`` for each column (which would result in
multiple scans):

.. code-block:: python

    >>> col_known = ddf.col.cat.as_known()  # use for single column
    >>> col_known.cat.known
    True
    >>> ddf_known = ddf.categorize()        # use for multiple columns
    >>> ddf_known.col.cat.known
    True

To convert a known categorical to an unknown categorical, there is also the
``.cat.as_unknown()`` method. This requires no computation as it's just a
change in the metadata.

Non-categorical columns can be converted to categoricals in a few different
ways:

.. code-block:: python

    # astype operates lazily, and results in unknown categoricals
    ddf = ddf.astype({'mycol': 'category', ...})
    # or
    ddf['mycol'] = ddf.mycol.astype('category')

    # categorize requires computation, and results in known categoricals
    ddf = ddf.categorize(columns=['mycol', ...])

Additionally, with Pandas 0.19.2 and up, ``dd.read_csv`` and ``dd.read_table``
can read data directly into unknown categorical columns by specifying a column
dtype as ``'category'``:

.. code-block:: python

    >>> ddf = dd.read_csv(..., dtype={col_name: 'category'})

.. _`categorical data`: https://pandas.pydata.org/pandas-docs/stable/categorical.html

Moreover, with Pandas 0.21.0 and up, ``dd.read_csv`` and ``dd.read_table`` can read
data directly into *known* categoricals by specifying instances of
``pd.api.types.CategoricalDtype``:

.. code-block:: python

    >>> dtype = {'col': pd.api.types.CategoricalDtype(['a', 'b', 'c'])}
    >>> ddf = dd.read_csv(..., dtype=dtype)

If you write and read to parquet, Dask will forget known categories.
This happens because, due to performance concerns, all the categories are
saved in every partition rather than in the parquet metadata.
It is possible to manually load the categories:

.. code-block:: python

    >>> import dask.dataframe as dd
    >>> import pandas as pd
    >>> df = pd.DataFrame(data=list('abcaabbcc'), columns=['col'])
    >>> df.col = df.col.astype('category')
    >>> ddf = dd.from_pandas(df, npartitions=1)
    >>> ddf.col.cat.known
    True
    >>> ddf.to_parquet('tmp')
    >>> ddf2 = dd.read_parquet('tmp')
    >>> ddf2.col.cat.known
    False
    >>> ddf2.col = ddf2.col.cat.set_categories(ddf2.col.head(1).cat.categories)
    >>> ddf2.col.cat.known
    True