Source code distributed/queues.py

  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
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
import asyncio
from collections import defaultdict
import logging
import uuid

from .client import Future, Client
from .utils import tokey, sync, thread_state
from .worker import get_client
from .utils import parse_timedelta

logger = logging.getLogger(__name__)


class QueueExtension:
    """An extension for the scheduler to manage queues

    This adds the following routes to the scheduler

    *  queue_create
    *  queue_release
    *  queue_put
    *  queue_get
    *  queue_size
    """

    def __init__(self, scheduler):
        self.scheduler = scheduler
        self.queues = dict()
        self.client_refcount = dict()
        self.future_refcount = defaultdict(lambda: 0)

        self.scheduler.handlers.update(
            {
                "queue_create": self.create,
                "queue_put": self.put,
                "queue_get": self.get,
                "queue_qsize": self.qsize,
            }
        )

        self.scheduler.stream_handlers.update(
            {"queue-future-release": self.future_release, "queue_release": self.release}
        )

        self.scheduler.extensions["queues"] = self

    def create(self, comm=None, name=None, client=None, maxsize=0):
        logger.debug("Queue name: {}".format(name))
        if name not in self.queues:
            self.queues[name] = asyncio.Queue(maxsize=maxsize)
            self.client_refcount[name] = 1
        else:
            self.client_refcount[name] += 1

    def release(self, comm=None, name=None, client=None):
        if name not in self.queues:
            return

        self.client_refcount[name] -= 1
        if self.client_refcount[name] == 0:
            del self.client_refcount[name]
            futures = self.queues[name]._queue
            del self.queues[name]
            keys = [d["value"] for d in futures if d["type"] == "Future"]
            if keys:
                self.scheduler.client_releases_keys(keys=keys, client="queue-%s" % name)

    async def put(
        self, comm=None, name=None, key=None, data=None, client=None, timeout=None
    ):
        if key is not None:
            record = {"type": "Future", "value": key}
            self.future_refcount[name, key] += 1
            self.scheduler.client_desires_keys(keys=[key], client="queue-%s" % name)
        else:
            record = {"type": "msgpack", "value": data}
        await asyncio.wait_for(self.queues[name].put(record), timeout=timeout)

    def future_release(self, name=None, key=None, client=None):
        self.future_refcount[name, key] -= 1
        if self.future_refcount[name, key] == 0:
            self.scheduler.client_releases_keys(keys=[key], client="queue-%s" % name)
            del self.future_refcount[name, key]

    async def get(self, comm=None, name=None, client=None, timeout=None, batch=False):
        def process(record):
            """ Add task status if known """
            if record["type"] == "Future":
                record = record.copy()
                key = record["value"]
                ts = self.scheduler.tasks.get(key)
                state = ts.state if ts is not None else "lost"

                record["state"] = state
                if state == "erred":
                    record["exception"] = ts.exception_blame.exception
                    record["traceback"] = ts.exception_blame.traceback

            return record

        if batch:
            q = self.queues[name]
            out = []
            if batch is True:
                while not q.empty():
                    record = await q.get()
                    out.append(record)
            else:
                if timeout is not None:
                    msg = (
                        "Dask queues don't support simultaneous use of "
                        "integer batch sizes and timeouts"
                    )
                    raise NotImplementedError(msg)
                for i in range(batch):
                    record = await q.get()
                    out.append(record)
            out = [process(o) for o in out]
            return out
        else:
            record = await asyncio.wait_for(self.queues[name].get(), timeout=timeout)
            record = process(record)
            return record

    def qsize(self, comm=None, name=None, client=None):
        return self.queues[name].qsize()


class Queue:
    """Distributed Queue

    This allows multiple clients to share futures or small bits of data between
    each other with a multi-producer/multi-consumer queue.  All metadata is
    sequentialized through the scheduler.

    Elements of the Queue must be either Futures or msgpack-encodable data
    (ints, strings, lists, dicts).  All data is sent through the scheduler so
    it is wise not to send large objects.  To share large objects scatter the
    data and share the future instead.

    .. warning::

       This object is experimental and has known issues in Python 2

    Parameters
    ----------
    name: string (optional)
        Name used by other clients and the scheduler to identify the queue. If
        not given, a random name will be generated.
    client: Client (optional)
        Client used for communication with the scheduler. Defaults to the
        value of ``Client.current()``.
    maxsize: int (optional)
        Number of items allowed in the queue. If 0 (the default), the queue
        size is unbounded.

    Examples
    --------
    >>> from dask.distributed import Client, Queue  # doctest: +SKIP
    >>> client = Client()  # doctest: +SKIP
    >>> queue = Queue('x')  # doctest: +SKIP
    >>> future = client.submit(f, x)  # doctest: +SKIP
    >>> queue.put(future)  # doctest: +SKIP

    See Also
    --------
    Variable: shared variable between clients
    """

    def __init__(self, name=None, client=None, maxsize=0):
        self.client = client or Client.current()
        self.name = name or "queue-" + uuid.uuid4().hex
        self._event_started = asyncio.Event()
        if self.client.asynchronous or getattr(
            thread_state, "on_event_loop_thread", False
        ):

            async def _create_queue():
                await self.client.scheduler.queue_create(
                    name=self.name, maxsize=maxsize
                )
                self._event_started.set()

            self.client.loop.add_callback(_create_queue)
        else:
            sync(
                self.client.loop,
                self.client.scheduler.queue_create,
                name=self.name,
                maxsize=maxsize,
            )
            self._event_started.set()

    def __await__(self):
        async def _():
            await self._event_started.wait()
            return self

        return _().__await__()

    async def _put(self, value, timeout=None):
        if isinstance(value, Future):
            await self.client.scheduler.queue_put(
                key=tokey(value.key), timeout=timeout, name=self.name
            )
        else:
            await self.client.scheduler.queue_put(
                data=value, timeout=timeout, name=self.name
            )

    def put(self, value, timeout=None, **kwargs):
        """Put data into the queue

        Parameters
        ----------
        timeout: number or string or timedelta, optional
            Time in seconds to wait before timing out.
            Instead of number of seconds, it is also possible to specify
            a timedelta in string format, e.g. "200ms".
        """
        timeout = parse_timedelta(timeout)
        return self.client.sync(self._put, value, timeout=timeout, **kwargs)

    def get(self, timeout=None, batch=False, **kwargs):
        """Get data from the queue

        Parameters
        ----------
        timeout: number or string or timedelta, optional
            Time in seconds to wait before timing out.
            Instead of number of seconds, it is also possible to specify
            a timedelta in string format, e.g. "200ms".
        batch: boolean, int (optional)
            If True then return all elements currently waiting in the queue.
            If an integer than return that many elements from the queue
            If False (default) then return one item at a time
        """
        timeout = parse_timedelta(timeout)
        return self.client.sync(self._get, timeout=timeout, batch=batch, **kwargs)

    def qsize(self, **kwargs):
        """ Current number of elements in the queue """
        return self.client.sync(self._qsize, **kwargs)

    async def _get(self, timeout=None, batch=False):
        resp = await self.client.scheduler.queue_get(
            timeout=timeout, name=self.name, batch=batch
        )

        def process(d):
            if d["type"] == "Future":
                value = Future(d["value"], self.client, inform=True, state=d["state"])
                if d["state"] == "erred":
                    value._state.set_error(d["exception"], d["traceback"])
                self.client._send_to_scheduler(
                    {"op": "queue-future-release", "name": self.name, "key": d["value"]}
                )
            else:
                value = d["value"]

            return value

        if batch is False:
            result = process(resp)
        else:
            result = list(map(process, resp))

        return result

    async def _qsize(self):
        result = await self.client.scheduler.queue_qsize(name=self.name)
        return result

    def close(self):
        if self.client.status == "running":  # TODO: can leave zombie futures
            self.client._send_to_scheduler({"op": "queue_release", "name": self.name})

    def __getstate__(self):
        return (self.name, self.client.scheduler.address)

    def __setstate__(self, state):
        name, address = state
        try:
            client = get_client(address)
            assert client.scheduler.address == address
        except (AttributeError, AssertionError):
            client = Client(address, set_as_default=False)
        self.__init__(name=name, client=client)