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
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467 | import asyncio
from collections import defaultdict, deque
from contextlib import suppress
import logging
import threading
import weakref
from .core import CommClosedError
from .metrics import time
from .utils import sync, TimeoutError, parse_timedelta
from .protocol.serialize import to_serialize
logger = logging.getLogger(__name__)
class PubSubSchedulerExtension:
""" Extend Dask's scheduler with routes to handle PubSub machinery """
def __init__(self, scheduler):
self.scheduler = scheduler
self.publishers = defaultdict(set)
self.subscribers = defaultdict(set)
self.client_subscribers = defaultdict(set)
self.scheduler.handlers.update({"pubsub_add_publisher": self.add_publisher})
self.scheduler.stream_handlers.update(
{
"pubsub-add-subscriber": self.add_subscriber,
"pubsub-remove-publisher": self.remove_publisher,
"pubsub-remove-subscriber": self.remove_subscriber,
"pubsub-msg": self.handle_message,
}
)
self.scheduler.extensions["pubsub"] = self
def add_publisher(self, comm=None, name=None, worker=None):
logger.debug("Add publisher: %s %s", name, worker)
self.publishers[name].add(worker)
return {
"subscribers": {addr: {} for addr in self.subscribers[name]},
"publish-scheduler": name in self.client_subscribers
and len(self.client_subscribers[name]) > 0,
}
def add_subscriber(self, comm=None, name=None, worker=None, client=None):
if worker:
logger.debug("Add worker subscriber: %s %s", name, worker)
self.subscribers[name].add(worker)
for pub in self.publishers[name]:
self.scheduler.worker_send(
pub,
{"op": "pubsub-add-subscriber", "address": worker, "name": name},
)
elif client:
logger.debug("Add client subscriber: %s %s", name, client)
for pub in self.publishers[name]:
self.scheduler.worker_send(
pub,
{"op": "pubsub-publish-scheduler", "name": name, "publish": True},
)
self.client_subscribers[name].add(client)
def remove_publisher(self, comm=None, name=None, worker=None):
if worker in self.publishers[name]:
logger.debug("Remove publisher: %s %s", name, worker)
self.publishers[name].remove(worker)
if not self.subscribers[name] and not self.publishers[name]:
del self.subscribers[name]
del self.publishers[name]
def remove_subscriber(self, comm=None, name=None, worker=None, client=None):
if worker:
logger.debug("Remove worker subscriber: %s %s", name, worker)
self.subscribers[name].remove(worker)
for pub in self.publishers[name]:
self.scheduler.worker_send(
pub,
{"op": "pubsub-remove-subscriber", "address": worker, "name": name},
)
elif client:
logger.debug("Remove client subscriber: %s %s", name, client)
self.client_subscribers[name].remove(client)
if not self.client_subscribers[name]:
del self.client_subscribers[name]
for pub in self.publishers[name]:
self.scheduler.worker_send(
pub,
{
"op": "pubsub-publish-scheduler",
"name": name,
"publish": False,
},
)
if not self.subscribers[name] and not self.publishers[name]:
logger.debug("Remove PubSub topic %s", name)
del self.subscribers[name]
del self.publishers[name]
def handle_message(self, name=None, msg=None, worker=None, client=None):
for c in list(self.client_subscribers[name]):
try:
self.scheduler.client_comms[c].send(
{"op": "pubsub-msg", "name": name, "msg": msg}
)
except (KeyError, CommClosedError):
self.remove_subscriber(name=name, client=c)
if client:
for sub in self.subscribers[name]:
self.scheduler.worker_send(
sub, {"op": "pubsub-msg", "name": name, "msg": msg}
)
class PubSubWorkerExtension:
""" Extend Dask's Worker with routes to handle PubSub machinery """
def __init__(self, worker):
self.worker = worker
self.worker.stream_handlers.update(
{
"pubsub-add-subscriber": self.add_subscriber,
"pubsub-remove-subscriber": self.remove_subscriber,
"pubsub-msg": self.handle_message,
"pubsub-publish-scheduler": self.publish_scheduler,
}
)
self.subscribers = defaultdict(weakref.WeakSet)
self.publishers = defaultdict(weakref.WeakSet)
self.publish_to_scheduler = defaultdict(lambda: False)
self.worker.extensions["pubsub"] = self # circular reference
def add_subscriber(self, name=None, address=None, **info):
for pub in self.publishers[name]:
pub.subscribers[address] = info
def remove_subscriber(self, name=None, address=None):
for pub in self.publishers[name]:
del pub.subscribers[address]
def publish_scheduler(self, name=None, publish=None):
self.publish_to_scheduler[name] = publish
async def handle_message(self, name=None, msg=None):
for sub in self.subscribers.get(name, []):
await sub._put(msg)
def trigger_cleanup(self):
self.worker.loop.add_callback(self.cleanup)
def cleanup(self):
for name, s in dict(self.subscribers).items():
if not len(s):
msg = {"op": "pubsub-remove-subscriber", "name": name}
self.worker.batched_stream.send(msg)
del self.subscribers[name]
for name, p in dict(self.publishers).items():
if not len(p):
msg = {"op": "pubsub-remove-publisher", "name": name}
self.worker.batched_stream.send(msg)
del self.publishers[name]
del self.publish_to_scheduler[name]
class PubSubClientExtension:
""" Extend Dask's Client with handlers to handle PubSub machinery """
def __init__(self, client):
self.client = client
self.client._stream_handlers.update({"pubsub-msg": self.handle_message})
self.subscribers = defaultdict(weakref.WeakSet)
self.client.extensions["pubsub"] = self # TODO: circular reference
async def handle_message(self, name=None, msg=None):
for sub in self.subscribers[name]:
await sub._put(msg)
if not self.subscribers[name]:
self.client.scheduler_comm.send(
{"op": "pubsub-remove-subscribers", "name": name}
)
def trigger_cleanup(self):
self.client.loop.add_callback(self.cleanup)
def cleanup(self):
for name, s in self.subscribers.items():
if not s:
msg = {"op": "pubsub-remove-subscriber", "name": name}
self.client.scheduler_comm.send(msg)
class Pub:
"""Publish data with Publish-Subscribe pattern
This allows clients and workers to directly communicate data between each
other with a typical Publish-Subscribe pattern. This involves two
components,
Pub objects, into which we put data:
>>> pub = Pub('my-topic')
>>> pub.put(123)
And Sub objects, from which we collect data:
>>> sub = Sub('my-topic')
>>> sub.get()
123
Many Pub and Sub objects can exist for the same topic. All data sent from
any Pub will be sent to all Sub objects on that topic that are currently
connected. Pub's and Sub's find each other using the scheduler, but they
communicate directly with each other without coordination from the
scheduler.
Pubs and Subs use the central scheduler to find each other, but not to
mediate the communication. This means that there is very little additional
latency or overhead, and they are appropriate for very frequent data
transfers. For context, most data transfer first checks with the scheduler to find which
workers should participate, and then does direct worker-to-worker
transfers. This checking in with the scheduler provides some stability
guarantees, but also adds in a few extra network hops. PubSub doesn't do
this, and so is faster, but also can easily drop messages if Pubs or Subs
disappear without notice.
When using a Pub or Sub from a Client all communications will be routed
through the scheduler. This can cause some performance degradation. Pubs
and Subs only operate at top-speed when they are both on workers.
Parameters
----------
name: object (msgpack serializable)
The name of the group of Pubs and Subs on which to participate.
worker: Worker (optional)
The worker to be used for publishing data. Defaults to the value of
```get_worker()```. If given, ``client`` must be ``None``.
client: Client (optional)
Client used for communication with the scheduler. Defaults to
the value of ``get_client()``. If given, ``worker`` must be ``None``.
Examples
--------
>>> pub = Pub('my-topic')
>>> sub = Sub('my-topic')
>>> pub.put([1, 2, 3])
>>> sub.get()
[1, 2, 3]
You can also use sub within a for loop:
>>> for msg in sub: # doctest: +SKIP
... print(msg)
or an async for loop
>>> async for msg in sub: # doctest: +SKIP
... print(msg)
Similarly the ``.get`` method will return an awaitable if used by an async
client or within the IOLoop thread of a worker
>>> await sub.get() # doctest: +SKIP
You can see the set of connected worker subscribers by looking at the
``.subscribers`` attribute:
>>> pub.subscribers
{'tcp://...': {},
'tcp://...': {}}
See Also
--------
Sub
"""
def __init__(self, name, worker=None, client=None):
if worker is None and client is None:
from distributed import get_worker, get_client
try:
worker = get_worker()
except Exception:
client = get_client()
self.subscribers = dict()
self.worker = worker
self.client = client
assert client or worker
if self.worker:
self.scheduler = self.worker.scheduler
self.loop = self.worker.loop
elif self.client:
self.scheduler = self.client.scheduler
self.loop = self.client.loop
self.name = name
self._started = False
self._buffer = []
self.loop.add_callback(self._start)
if self.worker:
pubsub = self.worker.extensions["pubsub"]
self.loop.add_callback(pubsub.publishers[name].add, self)
weakref.finalize(self, pubsub.trigger_cleanup)
async def _start(self):
if self.worker:
result = await self.scheduler.pubsub_add_publisher(
name=self.name, worker=self.worker.address
)
pubsub = self.worker.extensions["pubsub"]
self.subscribers.update(result["subscribers"])
pubsub.publish_to_scheduler[self.name] = result["publish-scheduler"]
self._started = True
for msg in self._buffer:
self.put(msg)
del self._buffer[:]
def _put(self, msg):
if not self._started:
self._buffer.append(msg)
return
data = {"op": "pubsub-msg", "name": self.name, "msg": to_serialize(msg)}
if self.worker:
for sub in self.subscribers:
self.worker.send_to_worker(sub, data)
if self.worker.extensions["pubsub"].publish_to_scheduler[self.name]:
self.worker.batched_stream.send(data)
elif self.client:
self.client.scheduler_comm.send(data)
def put(self, msg):
""" Publish a message to all subscribers of this topic """
self.loop.add_callback(self._put, msg)
def __repr__(self):
return "<Pub: {}>".format(self.name)
__str__ = __repr__
class Sub:
"""Subscribe to a Publish/Subscribe topic
See Also
--------
Pub: for full docstring
"""
def __init__(self, name, worker=None, client=None):
if worker is None and client is None:
from distributed.worker import get_worker, get_client
try:
worker = get_worker()
except Exception:
client = get_client()
self.worker = worker
self.client = client
if self.worker:
self.loop = self.worker.loop
elif self.client:
self.loop = self.client.loop
self.name = name
self.buffer = deque()
if self.worker:
pubsub = self.worker.extensions["pubsub"]
elif self.client:
pubsub = self.client.extensions["pubsub"]
self.loop.add_callback(pubsub.subscribers[name].add, self)
msg = {"op": "pubsub-add-subscriber", "name": self.name}
if self.worker:
self.loop.add_callback(self.worker.batched_stream.send, msg)
elif self.client:
self.loop.add_callback(self.client.scheduler_comm.send, msg)
else:
raise Exception()
weakref.finalize(self, pubsub.trigger_cleanup)
@property
def condition(self):
try:
return self._condition
except AttributeError:
self._condition = asyncio.Condition()
return self._condition
async def _get(self, timeout=None):
start = time()
while not self.buffer:
if timeout is not None:
timeout2 = timeout - (time() - start)
if timeout2 < 0:
raise TimeoutError()
else:
timeout2 = None
async def _():
await self.condition.acquire()
await self.condition.wait()
try:
await asyncio.wait_for(_(), timeout2)
finally:
with suppress(RuntimeError): # Python 3.6 fails here sometimes
self.condition.release()
return self.buffer.popleft()
__anext__ = _get
def get(self, timeout=None):
"""Get a single message
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)
if self.client:
return self.client.sync(self._get, timeout=timeout)
elif self.worker.thread_id == threading.get_ident():
return self._get()
else:
if self.buffer: # fastpath
return self.buffer.popleft()
return sync(self.loop, self._get, timeout=timeout)
next = __next__ = get
def __iter__(self):
return self
def __aiter__(self):
return self
async def _put(self, msg):
self.buffer.append(msg)
async with self.condition:
self.condition.notify()
def __repr__(self):
return "<Sub: {}>".format(self.name)
__str__ = __repr__
|