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
|
Publish Datasets
================
A *published dataset* is a named reference to a Dask collection or list of
futures that has been published to the cluster. It is available for any client
to see and persists beyond the scope of an individual session.
Publishing datasets is useful in the following cases:
* You want to share computations with colleagues
* You want to persist results on the cluster between interactive sessions
Motivating Example
------------------
In this example we load a dask.dataframe from S3, manipulate it, and then
publish the result.
**Connect and Load**
.. code-block:: python
from dask.distributed import Client
client = Client('scheduler-address:8786')
import dask.dataframe as dd
df = dd.read_csv('s3://my-bucket/*.csv')
df2 = df[df.balance < 0]
df2 = client.persist(df2)
>>> df2.head()
name balance
0 Alice -100
1 Bob -200
2 Charlie -300
3 Dennis -400
4 Edith -500
**Publish**
To share this collection with a colleague we publish it under the name
``'negative_accounts'``
.. code-block:: python
client.publish_dataset(negative_accounts=df2)
**Load published dataset from different client**
Now any other client can connect to the scheduler and retrieve this published
dataset.
.. code-block:: python
>>> from dask.distributed import Client
>>> client = Client('scheduler-address:8786')
>>> client.list_datasets()
['negative_accounts']
>>> df = client.get_dataset('negative_accounts')
>>> df.head()
name balance
0 Alice -100
1 Bob -200
2 Charlie -300
3 Dennis -400
4 Edith -500
This allows users to easily share results. It also allows for the persistence
of important and commonly used datasets beyond a single session. Published
datasets continue to reside in distributed memory even after all clients
requesting them have disconnected.
Dictionary interface
--------------------
Alternatively you can use the ``.datasets`` mapping on the client to publish,
list, get, and delete global datasets.
.. code-block:: python
>>> client.datasets['negative_accounts'] = df
>>> list(client.datasets)
['negative_accounts']
>>> df = client.datasets['negative_accounts']
This mapping is globally shared among all clients connected to the same
scheduler.
Notes
-----
Published collections are not automatically persisted. If you publish an
un-persisted collection then others will still be able to get the collection
from the scheduler, but operations on that collection will start from scratch.
This allows you to publish views on data that do not permanently take up
cluster memory but can be surprising if you expect "publishing" to
automatically make a computed dataset rapidly available.
Any client can publish or unpublish a dataset.
Publishing too many large datasets can quickly consume a cluster's RAM.
API
---
.. currentmodule:: distributed.client
.. autosummary::
Client.publish_dataset
Client.list_datasets
Client.get_dataset
Client.unpublish_dataset
|