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 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225
|
Extending DataFrames
====================
Subclass DataFrames
-------------------
There are a few projects that subclass or replicate the functionality of Pandas
objects:
- GeoPandas: for Geospatial analytics
- cuDF: for data analysis on GPUs
- ...
These projects may also want to produce parallel variants of themselves with
Dask, and may want to reuse some of the code in Dask DataFrame. Subclassing
Dask DataFrames is intended for maintainers of these libraries and not for
general users.
Implement dask, name, meta, and divisions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You will need to implement ``._meta``, ``.dask``, ``.divisions``, and
``._name`` as defined in the :doc:`DataFrame design docs <dataframe-design>`.
Extend Dispatched Methods
^^^^^^^^^^^^^^^^^^^^^^^^^
If you are going to pass around Pandas-like objects that are not normal Pandas
objects, then we ask you to extend a few dispatched methods: ``make_meta``,
``get_collection_type``, and ``concat``.
make_meta
"""""""""
This function returns an empty version of one of your non-Dask objects, given a
non-empty non-Dask object:
.. code-block:: python
from dask.dataframe.dispatch import make_meta_dispatch
@make_meta_dispatch.register(MyDataFrame)
def make_meta_dataframe(df, index=None):
return df.head(0)
@make_meta_dispatch.register(MySeries)
def make_meta_series(s, index=None):
return s.head(0)
@make_meta_dispatch.register(MyIndex)
def make_meta_index(ind, index=None):
return ind[:0]
For dispatching any arbitrary ``object`` types to a respective back-end, we
recommend registering a dispatch for ``make_meta_obj``:
.. code-block:: python
from dask.dataframe.dispatch import make_meta_obj
@make_meta_obj.register(MyDataFrame)
def make_meta_object(x, index=None):
if isinstance(x, dict):
return MyDataFrame()
elif isinstance(x, int):
return MySeries
.
.
.
Additionally, you should create a similar function that returns a non-empty
version of your non-Dask DataFrame objects filled with a few rows of
representative or random data. This is used to guess types when they are not
provided. It should expect an empty version of your object with columns,
dtypes, index name, and it should return a non-empty version:
.. code-block:: python
from dask.dataframe.utils import meta_nonempty
@meta_nonempty.register(MyDataFrame)
def meta_nonempty_dataframe(df):
...
return MyDataFrame(..., columns=df.columns,
index=MyIndex(..., name=df.index.name)
@meta_nonempty.register(MySeries)
def meta_nonempty_series(s):
...
@meta_nonempty.register(MyIndex)
def meta_nonempty_index(ind):
...
get_collection_type
"""""""""""""""""""
Given a non-Dask DataFrame object, return the Dask equivalent:
.. code-block:: python
from dask.dataframe import get_collection_type
@get_collection_type.register(MyDataFrame)
def get_collection_type_dataframe(df):
return MyDaskDataFrame
@get_collection_type.register(MySeries)
def get_collection_type_series(s):
return MyDaskSeries
@get_collection_type.register(MyIndex)
def get_collection_type_index(ind):
return MyDaskIndex
concat
""""""
Concatenate many of your non-Dask DataFrame objects together. It should expect
a list of your objects (homogeneously typed):
.. code-block:: python
from dask.dataframe.methods import concat_dispatch
@concat_dispatch.register((MyDataFrame, MySeries, MyIndex))
def concat_pandas(dfs, axis=0, join='outer', uniform=False, filter_warning=True):
...
.. _extensionarrays:
Extension Arrays
----------------
Rather than subclassing Pandas DataFrames, you may be interested in extending
Pandas with `Extension Arrays
<https://pandas.pydata.org/pandas-docs/stable/extending.html>`_.
All of the first-party extension arrays (those implemented in pandas itself)
are supported directly by dask.
Developers implementing third-party extension arrays (outside of pandas) will
need to do register their ``ExtensionDtype`` with Dask so that it works
correctly in ``dask.dataframe``.
For example, we'll register the *test-only* ``DecimalDtype`` from pandas
test suite.
.. code-block:: python
from decimal import Decimal
from dask.dataframe.extensions import make_array_nonempty, make_scalar
from pandas.tests.extension.decimal import DecimalArray, DecimalDtype
@make_array_nonempty.register(DecimalDtype)
def _(dtype):
return DecimalArray._from_sequence([Decimal('0'), Decimal('NaN')],
dtype=dtype)
@make_scalar.register(Decimal)
def _(x):
return Decimal('1')
Internally, Dask will use this to create a small dummy Series for tracking
metadata through operations.
.. code-block:: python
>>> make_array_nonempty(DecimalDtype())
<DecimalArray>
[Decimal('0'), Decimal('NaN')]
Length: 2, dtype: decimal
So you (or your users) can now create and store a dask ``DataFrame`` or
``Series`` with your extension array contained within.
.. code-block:: python
>>> from decimal import Decimal
>>> import dask.dataframe as dd
>>> import pandas as pd
>>> from pandas.tests.extension.decimal import DecimalArray
>>> s = pd.Series(DecimalArray([Decimal('0.0')] * 10))
>>> ds = dd.from_pandas(s, 3)
>>> ds
Dask Series Structure:
npartitions=3
0 decimal
4 ...
8 ...
9 ...
dtype: decimal
Dask Name: from_pandas, 3 tasks
Notice the ``decimal`` dtype.
.. _dataframe.accessors:
Accessors
---------
Many extension arrays expose their functionality on Series or DataFrame objects
using accessors. Dask provides decorators to register accessors similar to pandas. See
`the pandas documentation on accessors <http://pandas.pydata.org/pandas-docs/stable/development/extending.html#registering-custom-accessors>`_
for more.
.. currentmodule:: dask.dataframe
.. autofunction:: dask.dataframe.extensions.register_dataframe_accessor
.. autofunction:: dask.dataframe.extensions.register_series_accessor
.. autofunction:: dask.dataframe.extensions.register_index_accessor
|