-
Notifications
You must be signed in to change notification settings - Fork 150
Expand file tree
/
Copy pathcore.py
More file actions
1965 lines (1645 loc) · 60.7 KB
/
core.py
File metadata and controls
1965 lines (1645 loc) · 60.7 KB
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
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
import asyncio
from collections import deque, defaultdict
from datetime import timedelta
import functools
import logging
import six
import sys
import threading
from time import time
from typing import Any, Callable, Hashable, Union
import weakref
import toolz
from tornado import gen
from tornado.locks import Condition
from tornado.ioloop import IOLoop
from tornado.queues import Queue
try:
from distributed.client import default_client as _dask_default_client
except ImportError: # pragma: no cover
_dask_default_client = None
from collections.abc import Iterable
from threading import get_ident as get_thread_identity
from .orderedweakset import OrderedWeakrefSet
no_default = '--no-default--'
_html_update_streams = set()
thread_state = threading.local()
logger = logging.getLogger(__name__)
_io_loops = []
def get_io_loop(asynchronous=None):
if asynchronous:
return IOLoop.current()
if _dask_default_client is not None:
try:
client = _dask_default_client()
except ValueError:
# No dask client found; continue
pass
else:
return client.loop
if not _io_loops:
loop = IOLoop()
thread = threading.Thread(target=loop.start)
thread.daemon = True
thread.start()
_io_loops.append(loop)
return _io_loops[-1]
def identity(x):
return x
class RefCounter:
""" A counter to track references to data
This class is used to track how many nodes in the DAG are referencing
a particular element in the pipeline. When the count reaches zero,
then parties interested in knowing if data is done being processed are
notified
Parameters
----------
initial: int, optional
The initial value of the reference counter
cb: callable
The function to use a callback when the reference count reaches zero
loop: tornado.ioloop.IOLoop
The loop on which to create a callback when the reference count
reaches zero
"""
def __init__(self, initial=0, cb=None, loop=None):
self.loop = loop if loop else get_io_loop()
self.count = initial
self.cb = cb
def retain(self, n=1):
"""Retain the reference
Parameters
----------
n: The number of times to retain the reference
"""
self.count += n
def release(self, n=1):
"""Release the reference
If the reference count is equal to or less than zero, the callback, if
provided will added to the provided loop or default loop
Parameters
----------
n: The number of references to release
"""
self.count -= n
if self.count <= 0 and self.cb:
self.loop.add_callback(self.cb)
def __str__(self):
return '<RefCounter count={}>'.format(self.count)
__repr__ = __str__
class APIRegisterMixin(object):
def _new_node(self, cls, args, kwargs):
""" Constructor for downstream nodes.
Examples
--------
To provide inheritance through nodes :
>>> class MyStream(Stream):
>>>
>>> def _new_node(self, cls, args, kwargs):
>>> if not issubclass(cls, MyStream):
>>> cls = type(cls.__name__, (cls, MyStream), dict(cls.__dict__))
>>> return cls(*args, **kwargs)
"""
return cls(*args, **kwargs)
@classmethod
def register_api(cls, modifier=identity, attribute_name=None):
""" Add callable to Stream API
This allows you to register a new method onto this class. You can use
it as a decorator.::
>>> @Stream.register_api()
... class foo(Stream):
... ...
>>> Stream().foo(...) # this works now
It attaches the callable as a normal attribute to the class object. In
doing so it respects inheritance (all subclasses of Stream will also
get the foo attribute).
By default callables are assumed to be instance methods. If you like
you can include modifiers to apply before attaching to the class as in
the following case where we construct a ``staticmethod``.
>>> @Stream.register_api(staticmethod)
... class foo(Stream):
... ...
>>> Stream.foo(...) # Foo operates as a static method
You can also provide an optional ``attribute_name`` argument to control
the name of the attribute your callable will be attached as.
>>> @Stream.register_api(attribute_name="bar")
... class foo(Stream):
... ...
>> Stream().bar(...) # foo was actually attached as bar
"""
def _(func):
@functools.wraps(func)
def wrapped(*args, **kwargs):
if identity is not staticmethod and args:
self = args[0]
if isinstance(self, APIRegisterMixin):
return self._new_node(func, args, kwargs)
return func(*args, **kwargs)
name = attribute_name if attribute_name else func.__name__
setattr(cls, name, modifier(wrapped))
return func
return _
@classmethod
def register_plugin_entry_point(cls, entry_point, modifier=identity):
if hasattr(cls, entry_point.name):
raise ValueError(
f"Can't add {entry_point.name} from {entry_point.module_name} "
f"to {cls.__name__}: duplicate method name."
)
def stub(*args, **kwargs):
""" Entrypoints-based streamz plugin. Will be loaded on first call. """
node = entry_point.load()
if not issubclass(node, Stream):
raise TypeError(
f"Error loading {entry_point.name} "
f"from module {entry_point.module_name}: "
f"{node.__class__.__name__} must be a subclass of Stream"
)
if getattr(cls, entry_point.name).__name__ == "stub":
cls.register_api(
modifier=modifier, attribute_name=entry_point.name
)(node)
return node(*args, **kwargs)
cls.register_api(modifier=modifier, attribute_name=entry_point.name)(stub)
class Stream(APIRegisterMixin):
""" A Stream is an infinite sequence of data.
Streams subscribe to each other passing and transforming data between them.
A Stream object listens for updates from upstream, reacts to these updates,
and then emits more data to flow downstream to all Stream objects that
subscribe to it. Downstream Stream objects may connect at any point of a
Stream graph to get a full view of the data coming off of that point to do
with as they will.
Parameters
----------
stream_name: str or None
This is the name of the stream.
asynchronous: boolean or None
Whether or not this stream will be used in asynchronous functions or
normal Python functions. Leave as None if you don't know.
True will cause operations like emit to return awaitable Futures
False will use an Event loop in another thread (starts it if necessary)
ensure_io_loop: boolean
Ensure that some IOLoop will be created. If asynchronous is None or
False then this will be in a separate thread, otherwise it will be
IOLoop.current
Examples
--------
>>> def inc(x):
... return x + 1
>>> source = Stream() # Create a stream object
>>> s = source.map(inc).map(str) # Subscribe to make new streams
>>> s.sink(print) # take an action whenever an element reaches the end
>>> L = list()
>>> s.sink(L.append) # or take multiple actions (streams can branch)
>>> for i in range(5):
... source.emit(i) # push data in at the source
'1'
'2'
'3'
'4'
'5'
>>> L # and the actions happen at the sinks
['1', '2', '3', '4', '5']
"""
_graphviz_shape = 'ellipse'
_graphviz_style = 'rounded,filled'
_graphviz_fillcolor = 'white'
_graphviz_orientation = 0
str_list = ['func', 'predicate', 'n', 'interval']
def __init__(self, upstream=None, upstreams=None, stream_name=None,
loop=None, asynchronous=None, ensure_io_loop=False):
self.name = stream_name
self.downstreams = OrderedWeakrefSet()
self.current_value = None
self.current_metadata = None
if upstreams is not None:
self.upstreams = list(upstreams)
elif upstream is not None:
self.upstreams = [upstream]
else:
self.upstreams = []
self._set_asynchronous(asynchronous)
self._set_loop(loop)
if ensure_io_loop and not self.loop:
self._set_asynchronous(False)
if self.loop is None and self.asynchronous is not None:
self._set_loop(get_io_loop(self.asynchronous))
for upstream in self.upstreams:
if upstream:
upstream.downstreams.add(self)
def _set_loop(self, loop):
self.loop = None
if loop is not None:
self._inform_loop(loop)
else:
for upstream in self.upstreams:
if upstream and upstream.loop:
self.loop = upstream.loop
break
def _inform_loop(self, loop):
"""
Percolate information about an event loop to the rest of the stream
"""
if self.loop is not None:
if self.loop is not loop:
raise ValueError("Two different event loops active")
else:
self.loop = loop
for upstream in self.upstreams:
if upstream:
upstream._inform_loop(loop)
for downstream in self.downstreams:
if downstream:
downstream._inform_loop(loop)
def _set_asynchronous(self, asynchronous):
self.asynchronous = None
if asynchronous is not None:
self._inform_asynchronous(asynchronous)
else:
for upstream in self.upstreams:
if upstream and upstream.asynchronous:
self.asynchronous = upstream.asynchronous
break
def _inform_asynchronous(self, asynchronous):
"""
Percolate information about an event loop to the rest of the stream
"""
if self.asynchronous is not None:
if self.asynchronous is not asynchronous:
raise ValueError("Stream has both asynchronous and synchronous elements")
else:
self.asynchronous = asynchronous
for upstream in self.upstreams:
if upstream:
upstream._inform_asynchronous(asynchronous)
for downstream in self.downstreams:
if downstream:
downstream._inform_asynchronous(asynchronous)
def _add_upstream(self, upstream):
"""Add upstream to current upstreams, this method is overridden for
classes which handle stream specific buffers/caches"""
self.upstreams.append(upstream)
def _add_downstream(self, downstream):
"""Add downstream to current downstreams"""
self.downstreams.add(downstream)
def _remove_downstream(self, downstream):
"""Remove downstream from current downstreams"""
self.downstreams.remove(downstream)
def _remove_upstream(self, upstream):
"""Remove upstream from current upstreams, this method is overridden for
classes which handle stream specific buffers/caches"""
self.upstreams.remove(upstream)
def start(self):
""" Start any upstream sources """
for upstream in self.upstreams:
upstream.start()
def stop(self):
""" Stop upstream sources """
for upstream in self.upstreams:
upstream.stop()
def __str__(self):
s_list = []
if self.name:
s_list.append('{}; {}'.format(self.name, self.__class__.__name__))
else:
s_list.append(self.__class__.__name__)
for m in self.str_list:
s = ''
at = getattr(self, m, None)
if at:
if not callable(at):
s = str(at)
elif hasattr(at, '__name__'):
s = getattr(self, m).__name__
else:
s = None
if s:
s_list.append('{}={}'.format(m, s))
if len(s_list) <= 2:
s_list = [term.split('=')[-1] for term in s_list]
text = "<"
text += s_list[0]
if len(s_list) > 1:
text += ': '
text += ', '.join(s_list[1:])
text += '>'
return text
__repr__ = __str__
def _ipython_display_(self, **kwargs): # pragma: no cover
try:
import ipywidgets
from IPython.core.interactiveshell import InteractiveShell
output = ipywidgets.Output(_view_count=0)
except ImportError:
# since this function is only called by jupyter, this import must succeed
from IPython.display import display, HTML
if hasattr(self, '_repr_html_'):
return display(HTML(self._repr_html_()))
else:
return display(self.__repr__())
output_ref = weakref.ref(output)
def update_cell(val):
output = output_ref()
if output is None:
return
with output:
content, *_ = InteractiveShell.instance().display_formatter.format(val)
output.outputs = ({'output_type': 'display_data',
'data': content,
'metadata': {}},)
s = self.map(update_cell)
_html_update_streams.add(s)
self.output_ref = output_ref
s_ref = weakref.ref(s)
def remove_stream(change):
output = output_ref()
if output is None:
return
if output._view_count == 0:
ss = s_ref()
ss.destroy()
_html_update_streams.remove(ss) # trigger gc
output.observe(remove_stream, '_view_count')
return output._ipython_display_(**kwargs)
def _emit(self, x, metadata=None):
"""
Push data into the stream at this point
Parameters
----------
x: any
an element of data
metadata: list[dict], optional
Various types of metadata associated with the data element in `x`.
ref: RefCounter
A reference counter used to check when data is done
"""
self.current_value = x
self.current_metadata = metadata
if metadata:
self._retain_refs(metadata, len(self.downstreams))
else:
metadata = []
result = []
for downstream in list(self.downstreams):
r = downstream.update(x, who=self, metadata=metadata)
if type(r) is list:
result.extend(r)
else:
result.append(r)
self._release_refs(metadata)
return [element for element in result if element is not None]
def emit(self, x, asynchronous=False, metadata=None):
""" Push data into the stream at this point
This is typically done only at source Streams but can theoretically be
done at any point
Parameters
----------
x: any
an element of data
asynchronous:
emit asynchronously
metadata: list[dict], optional
Various types of metadata associated with the data element in `x`.
ref: RefCounter
A reference counter used to check when data is done
"""
ts_async = getattr(thread_state, 'asynchronous', False)
if self.loop is None or asynchronous or self.asynchronous or ts_async:
if not ts_async:
thread_state.asynchronous = True
try:
result = self._emit(x, metadata=metadata)
if self.loop:
return gen.convert_yielded(result)
finally:
thread_state.asynchronous = ts_async
else:
async def _():
thread_state.asynchronous = True
try:
result = await asyncio.gather(*self._emit(x, metadata=metadata))
finally:
del thread_state.asynchronous
return result
sync(self.loop, _)
def update(self, x, who=None, metadata=None):
return self._emit(x, metadata=metadata)
def gather(self):
""" This is a no-op for core streamz
This allows gather to be used in both dask and core streams
"""
return self
def connect(self, downstream):
""" Connect this stream to a downstream element.
Parameters
----------
downstream: Stream
The downstream stream to connect to
"""
self._add_downstream(downstream)
downstream._add_upstream(self)
def disconnect(self, downstream):
""" Disconnect this stream to a downstream element.
Parameters
----------
downstream: Stream
The downstream stream to disconnect from
"""
self._remove_downstream(downstream)
downstream._remove_upstream(self)
@property
def upstream(self):
if len(self.upstreams) > 1:
raise ValueError("Stream has multiple upstreams")
elif len(self.upstreams) == 0:
return None
else:
return self.upstreams[0]
def destroy(self, streams=None):
"""
Disconnect this stream from any upstream sources
"""
if streams is None:
streams = self.upstreams
for upstream in list(streams):
upstream._remove_downstream(self)
self._remove_upstream(upstream)
def scatter(self, **kwargs):
from .dask import scatter
return scatter(self, **kwargs)
def remove(self, predicate):
""" Only pass through elements for which the predicate returns False """
return self.filter(lambda x: not predicate(x))
@property
def scan(self):
return self.accumulate
@property
def concat(self):
return self.flatten
def sink_to_list(self):
""" Append all elements of a stream to a list as they come in
Examples
--------
>>> source = Stream()
>>> L = source.map(lambda x: 10 * x).sink_to_list()
>>> for i in range(5):
... source.emit(i)
>>> L
[0, 10, 20, 30, 40]
"""
L = []
self.sink(L.append)
return L
def frequencies(self, **kwargs):
""" Count occurrences of elements """
def update_frequencies(last, x):
return toolz.assoc(last, x, last.get(x, 0) + 1)
return self.scan(update_frequencies, start={}, **kwargs)
def visualize(self, filename='mystream.png', **kwargs):
"""Render the computation of this object's task graph using graphviz.
Requires ``graphviz`` and ``networkx`` to be installed.
Parameters
----------
filename : str, optional
The name of the file to write to disk.
kwargs:
Graph attributes to pass to graphviz like ``rankdir="LR"``
"""
from .graph import visualize
return visualize(self, filename, **kwargs)
def to_dataframe(self, example):
""" Convert a stream of Pandas dataframes to a DataFrame
Examples
--------
>>> source = Stream()
>>> sdf = source.to_dataframe()
>>> L = sdf.groupby(sdf.x).y.mean().stream.sink_to_list()
>>> source.emit(pd.DataFrame(...)) # doctest: +SKIP
>>> source.emit(pd.DataFrame(...)) # doctest: +SKIP
>>> source.emit(pd.DataFrame(...)) # doctest: +SKIP
"""
from .dataframe import DataFrame
return DataFrame(stream=self, example=example)
def to_batch(self, **kwargs):
""" Convert a stream of lists to a Batch
All elements of the stream are assumed to be lists or tuples
Examples
--------
>>> source = Stream()
>>> batches = source.to_batch()
>>> L = batches.pluck('value').map(inc).sum().stream.sink_to_list()
>>> source.emit([{'name': 'Alice', 'value': 1},
... {'name': 'Bob', 'value': 2},
... {'name': 'Charlie', 'value': 3}])
>>> source.emit([{'name': 'Alice', 'value': 4},
... {'name': 'Bob', 'value': 5},
... {'name': 'Charlie', 'value': 6}])
"""
from .batch import Batch
return Batch(stream=self, **kwargs)
def _retain_refs(self, metadata, n=1):
""" Retain all references in the provided metadata `n` number of times
Parameters
----------
metadata: list[dict], optional
Various types of metadata associated with the data element in `x`.
ref: RefCounter
A reference counter used to check when data is done
n: The number of times to retain the provided references
"""
for m in metadata:
if 'ref' in m:
m['ref'].retain(n)
def _release_refs(self, metadata, n=1):
""" Release all references in the provided metadata `n` number of times
Parameters
----------
metadata: list[dict], optional
Various types of metadata associated with the data element in `x`.
ref: RefCounter
A reference counter used to check when data is done
n: The number of times to retain the provided references
"""
for m in metadata:
if 'ref' in m:
m['ref'].release(n)
@Stream.register_api()
class map(Stream):
""" Apply a function to every element in the stream
Parameters
----------
func: callable
*args :
The arguments to pass to the function.
**kwargs:
Keyword arguments to pass to func
Examples
--------
>>> source = Stream()
>>> source.map(lambda x: 2*x).sink(print)
>>> for i in range(5):
... source.emit(i)
0
2
4
6
8
"""
def __init__(self, upstream, func, *args, **kwargs):
self.func = func
# this is one of a few stream specific kwargs
stream_name = kwargs.pop('stream_name', None)
self.kwargs = kwargs
self.args = args
Stream.__init__(self, upstream, stream_name=stream_name)
def update(self, x, who=None, metadata=None):
try:
result = self.func(x, *self.args, **self.kwargs)
except Exception as e:
logger.exception(e)
raise
else:
return self._emit(result, metadata=metadata)
@Stream.register_api()
class starmap(Stream):
""" Apply a function to every element in the stream, splayed out
See ``itertools.starmap``
Parameters
----------
func: callable
*args :
The arguments to pass to the function.
**kwargs:
Keyword arguments to pass to func
Examples
--------
>>> source = Stream()
>>> source.starmap(lambda a, b: a + b).sink(print)
>>> for i in range(5):
... source.emit((i, i))
0
2
4
6
8
"""
def __init__(self, upstream, func, *args, **kwargs):
self.func = func
# this is one of a few stream specific kwargs
stream_name = kwargs.pop('stream_name', None)
self.kwargs = kwargs
self.args = args
Stream.__init__(self, upstream, stream_name=stream_name)
def update(self, x, who=None, metadata=None):
y = x + self.args
try:
result = self.func(*y, **self.kwargs)
except Exception as e:
logger.exception(e)
raise
else:
return self._emit(result, metadata=metadata)
def _truthy(x):
return not not x
@Stream.register_api()
class filter(Stream):
""" Only pass through elements that satisfy the predicate
Parameters
----------
predicate : function
The predicate. Should return True or False, where
True means that the predicate is satisfied.
*args :
The arguments to pass to the predicate.
**kwargs:
Keyword arguments to pass to predicate
Examples
--------
>>> source = Stream()
>>> source.filter(lambda x: x % 2 == 0).sink(print)
>>> for i in range(5):
... source.emit(i)
0
2
4
"""
def __init__(self, upstream, predicate, *args, **kwargs):
if predicate is None:
predicate = _truthy
self.predicate = predicate
stream_name = kwargs.pop("stream_name", None)
self.kwargs = kwargs
self.args = args
Stream.__init__(self, upstream, stream_name=stream_name)
def update(self, x, who=None, metadata=None):
if self.predicate(x, *self.args, **self.kwargs):
return self._emit(x, metadata=metadata)
@Stream.register_api()
class accumulate(Stream):
""" Accumulate results with previous state
This performs running or cumulative reductions, applying the function
to the previous total and the new element. The function should take
two arguments, the previous accumulated state and the next element and
it should return a new accumulated state,
- ``state = func(previous_state, new_value)`` (returns_state=False)
- ``state, result = func(previous_state, new_value)`` (returns_state=True)
where the new_state is passed to the next invocation. The state or result
is emitted downstream for the two cases.
Parameters
----------
func: callable
start: object
Initial value, passed as the value of ``previous_state`` on the first
invocation. Defaults to the first submitted element
returns_state: boolean
If true then func should return both the state and the value to emit
If false then both values are the same, and func returns one value
**kwargs:
Keyword arguments to pass to func
Examples
--------
A running total, producing triangular numbers
>>> source = Stream()
>>> source.accumulate(lambda acc, x: acc + x).sink(print)
>>> for i in range(5):
... source.emit(i)
0
1
3
6
10
A count of number of events (including the current one)
>>> source = Stream()
>>> source.accumulate(lambda acc, x: acc + 1, start=0).sink(print)
>>> for _ in range(5):
... source.emit(0)
1
2
3
4
5
Like the builtin "enumerate".
>>> source = Stream()
>>> source.accumulate(lambda acc, x: ((acc[0] + 1, x), (acc[0], x)),
... start=(0, 0), returns_state=True
... ).sink(print)
>>> for i in range(3):
... source.emit(0)
(0, 0)
(1, 0)
(2, 0)
"""
_graphviz_shape = 'box'
def __init__(self, upstream, func, start=no_default, returns_state=False,
**kwargs):
self.func = func
self.kwargs = kwargs
self.state = start
self.returns_state = returns_state
# this is one of a few stream specific kwargs
stream_name = kwargs.pop('stream_name', None)
self.with_state = kwargs.pop('with_state', False)
Stream.__init__(self, upstream, stream_name=stream_name)
def update(self, x, who=None, metadata=None):
if self.state is no_default:
self.state = x
if self.with_state:
return self._emit((self.state, x), metadata=metadata)
else:
return self._emit(x, metadata=metadata)
else:
try:
result = self.func(self.state, x, **self.kwargs)
except Exception as e:
logger.exception(e)
raise
if self.returns_state:
state, result = result
else:
state = result
self.state = state
if self.with_state:
return self._emit((self.state, result), metadata=metadata)
else:
return self._emit(result, metadata=metadata)
@Stream.register_api()
class slice(Stream):
"""
Get only some events in a stream by position. Works like list[] syntax.
Parameters
----------
start : int
First event to use. If None, start from the beginnning
end : int
Last event to use (non-inclusive). If None, continue without stopping.
Does not support negative indexing.
step : int
Pass on every Nth event. If None, pass every one.
Examples
--------
>>> source = Stream()
>>> source.slice(2, 6, 2).sink(print)
>>> for i in range(5):
... source.emit(0)
2
4
"""
def __init__(self, upstream, start=None, end=None, step=None, **kwargs):
self.state = 0
self.star = start or 0
self.end = end
self.step = step or 1
if any((_ or 0) < 0 for _ in [start, end, step]):
raise ValueError("Negative indices not supported by slice")
stream_name = kwargs.pop('stream_name', None)
Stream.__init__(self, upstream, stream_name=stream_name)
self._check_end()
def update(self, x, who=None, metadata=None):
if self.state >= self.star and self.state % self.step == 0:
self.emit(x, metadata=metadata)
self.state += 1
self._check_end()
def _check_end(self):
if self.end and self.state >= self.end:
# we're done
for upstream in self.upstreams:
upstream._remove_downstream(self)
@Stream.register_api()
class partition(Stream):
""" Partition stream into tuples of equal size
Parameters
----------
n: int
Maximum partition size
timeout: int or float, optional
Number of seconds after which a partition will be emitted,
even if its size is less than ``n``. If ``None`` (default),
a partition will be emitted only when its size reaches ``n``.
key: hashable or callable, optional
Emit items with the same key together as a separate partition.
If ``key`` is callable, partition will be identified by ``key(x)``,
otherwise by ``x[key]``. Defaults to ``None``.
Examples
--------
>>> source = Stream()
>>> source.partition(3).sink(print)
>>> for i in range(10):
... source.emit(i)