-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
trigger.py
1646 lines (1317 loc) · 53.3 KB
/
trigger.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
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
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
"""Support for Apache Beam triggers.
Triggers control when in processing time windows get emitted.
"""
# pytype: skip-file
import collections
import copy
import logging
import numbers
from abc import ABCMeta
from abc import abstractmethod
from collections import abc as collections_abc # ambiguty with direct abc
from enum import Flag
from enum import auto
from itertools import zip_longest
from apache_beam.coders import coder_impl
from apache_beam.coders import observable
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.transforms import combiners
from apache_beam.transforms import core
from apache_beam.transforms.timeutil import TimeDomain
from apache_beam.transforms.window import GlobalWindow
from apache_beam.transforms.window import GlobalWindows
from apache_beam.transforms.window import TimestampCombiner
from apache_beam.transforms.window import WindowedValue
from apache_beam.transforms.window import WindowFn
from apache_beam.utils import windowed_value
from apache_beam.utils.timestamp import MAX_TIMESTAMP
from apache_beam.utils.timestamp import MIN_TIMESTAMP
from apache_beam.utils.timestamp import TIME_GRANULARITY
__all__ = [
'AccumulationMode',
'TriggerFn',
'DefaultTrigger',
'AfterWatermark',
'AfterProcessingTime',
'AfterCount',
'Repeatedly',
'AfterAny',
'AfterAll',
'AfterEach',
'OrFinally',
]
_LOGGER = logging.getLogger(__name__)
class AccumulationMode(object):
"""Controls what to do with data when a trigger fires multiple times."""
DISCARDING = beam_runner_api_pb2.AccumulationMode.DISCARDING
ACCUMULATING = beam_runner_api_pb2.AccumulationMode.ACCUMULATING
# TODO(robertwb): Provide retractions of previous outputs.
# RETRACTING = 3
class _StateTag(metaclass=ABCMeta):
"""An identifier used to store and retrieve typed, combinable state.
The given tag must be unique for this step."""
def __init__(self, tag):
self.tag = tag
class _ReadModifyWriteStateTag(_StateTag):
"""StateTag pointing to an element."""
def __repr__(self):
return 'ValueStateTag(%s)' % (self.tag)
def with_prefix(self, prefix):
return _ReadModifyWriteStateTag(prefix + self.tag)
class _SetStateTag(_StateTag):
"""StateTag pointing to an element."""
def __repr__(self):
return 'SetStateTag({tag})'.format(tag=self.tag)
def with_prefix(self, prefix):
return _SetStateTag(prefix + self.tag)
class _CombiningValueStateTag(_StateTag):
"""StateTag pointing to an element, accumulated with a combiner.
The given tag must be unique for this step. The given CombineFn will be
applied (possibly incrementally and eagerly) when adding elements."""
# TODO(robertwb): Also store the coder (perhaps extracted from the combine_fn)
def __init__(self, tag, combine_fn):
super().__init__(tag)
if not combine_fn:
raise ValueError('combine_fn must be specified.')
if not isinstance(combine_fn, core.CombineFn):
combine_fn = core.CombineFn.from_callable(combine_fn)
self.combine_fn = combine_fn
def __repr__(self):
return 'CombiningValueStateTag(%s, %s)' % (self.tag, self.combine_fn)
def with_prefix(self, prefix):
return _CombiningValueStateTag(prefix + self.tag, self.combine_fn)
def without_extraction(self):
class NoExtractionCombineFn(core.CombineFn):
setup = self.combine_fn.setup
create_accumulator = self.combine_fn.create_accumulator
add_input = self.combine_fn.add_input
merge_accumulators = self.combine_fn.merge_accumulators
compact = self.combine_fn.compact
extract_output = staticmethod(lambda x: x)
teardown = self.combine_fn.teardown
return _CombiningValueStateTag(self.tag, NoExtractionCombineFn())
class _ListStateTag(_StateTag):
"""StateTag pointing to a list of elements."""
def __repr__(self):
return 'ListStateTag(%s)' % self.tag
def with_prefix(self, prefix):
return _ListStateTag(prefix + self.tag)
class _WatermarkHoldStateTag(_StateTag):
def __init__(self, tag, timestamp_combiner_impl):
super().__init__(tag)
self.timestamp_combiner_impl = timestamp_combiner_impl
def __repr__(self):
return 'WatermarkHoldStateTag(%s, %s)' % (
self.tag, self.timestamp_combiner_impl)
def with_prefix(self, prefix):
return _WatermarkHoldStateTag(
prefix + self.tag, self.timestamp_combiner_impl)
class DataLossReason(Flag):
"""Enum defining potential reasons that a trigger may cause data loss.
These flags should only cover when the trigger is the cause, though windowing
can be taken into account. For instance, AfterWatermark may not flag itself
as finishing if the windowing doesn't allow lateness.
"""
# Trigger will never be the source of data loss.
NO_POTENTIAL_LOSS = 0
# Trigger may finish. In this case, data that comes in after the trigger may
# be lost. Example: AfterCount(1) will stop firing after the first element.
MAY_FINISH = auto()
# Deprecated: Beam will emit buffered data at GC time. Any other behavior
# should be treated as a bug with the runner used.
CONDITION_NOT_GUARANTEED = auto()
# Convenience functions for checking if a flag is included. Each is equivalent
# to `reason & flag == flag`
def _IncludesMayFinish(reason: DataLossReason) -> bool:
return reason & DataLossReason.MAY_FINISH == DataLossReason.MAY_FINISH
# pylint: disable=unused-argument
# TODO(robertwb): Provisional API, Java likely to change as well.
class TriggerFn(metaclass=ABCMeta):
"""A TriggerFn determines when window (panes) are emitted.
See https://beam.apache.org/documentation/programming-guide/#triggers
"""
@abstractmethod
def on_element(self, element, window, context):
"""Called when a new element arrives in a window.
Args:
element: the element being added
window: the window to which the element is being added
context: a context (e.g. a TriggerContext instance) for managing state
and setting timers
"""
pass
@abstractmethod
def on_merge(self, to_be_merged, merge_result, context):
"""Called when multiple windows are merged.
Args:
to_be_merged: the set of windows to be merged
merge_result: the window into which the windows are being merged
context: a context (e.g. a TriggerContext instance) for managing state
and setting timers
"""
pass
@abstractmethod
def should_fire(self, time_domain, timestamp, window, context):
"""Whether this trigger should cause the window to fire.
Args:
time_domain: WATERMARK for event-time timers and REAL_TIME for
processing-time timers.
timestamp: for time_domain WATERMARK, it represents the
watermark: (a lower bound on) the watermark of the system
and for time_domain REAL_TIME, it represents the
trigger: timestamp of the processing-time timer.
window: the window whose trigger is being considered
context: a context (e.g. a TriggerContext instance) for managing state
and setting timers
Returns:
whether this trigger should cause a firing
"""
pass
@abstractmethod
def has_ontime_pane(self):
"""Whether this trigger creates an empty pane even if there are no elements.
Returns:
True if this trigger guarantees that there will always be an ON_TIME pane
even if there are no elements in that pane.
"""
pass
@abstractmethod
def on_fire(self, watermark, window, context):
"""Called when a trigger actually fires.
Args:
watermark: (a lower bound on) the watermark of the system
window: the window whose trigger is being fired
context: a context (e.g. a TriggerContext instance) for managing state
and setting timers
Returns:
whether this trigger is finished
"""
pass
@abstractmethod
def reset(self, window, context):
"""Clear any state and timers used by this TriggerFn."""
pass
def may_lose_data(self, unused_windowing: core.Windowing) -> DataLossReason:
"""Returns whether or not this trigger could cause data loss.
A trigger can cause data loss in the following scenarios:
* The trigger has a chance to finish. For instance, AfterWatermark()
without a late trigger would cause all late data to be lost. This
scenario is only accounted for if the windowing strategy allows
late data. Otherwise, the trigger is not responsible for the data
loss.
Note that this only returns the potential for loss. It does not mean that
there will be data loss. It also only accounts for loss related to the
trigger, not other potential causes.
Args:
windowing: The Windowing that this trigger belongs to. It does not need
to be the top-level trigger.
Returns:
The DataLossReason. If there is no potential loss,
DataLossReason.NO_POTENTIAL_LOSS is returned. Otherwise, all the
potential reasons are returned as a single value.
"""
# For backwards compatibility's sake, we're assuming the trigger is safe.
return DataLossReason.NO_POTENTIAL_LOSS
# pylint: enable=unused-argument
@staticmethod
def from_runner_api(proto, context):
return {
'after_all': AfterAll,
'after_any': AfterAny,
'after_each': AfterEach,
'after_end_of_window': AfterWatermark,
'after_processing_time': AfterProcessingTime,
# after_processing_time, after_synchronized_processing_time
'always': Always,
'default': DefaultTrigger,
'element_count': AfterCount,
'never': _Never,
'or_finally': OrFinally,
'repeat': Repeatedly,
}[proto.WhichOneof('trigger')].from_runner_api(proto, context)
@abstractmethod
def to_runner_api(self, unused_context):
pass
class DefaultTrigger(TriggerFn):
"""Semantically Repeatedly(AfterWatermark()), but more optimized."""
def __init__(self):
pass
def __repr__(self):
return 'DefaultTrigger()'
def on_element(self, element, window, context):
context.set_timer(str(window), TimeDomain.WATERMARK, window.end)
def on_merge(self, to_be_merged, merge_result, context):
for window in to_be_merged:
context.clear_timer(str(window), TimeDomain.WATERMARK)
def should_fire(self, time_domain, watermark, window, context):
if watermark >= window.end:
# Explicitly clear the timer so that late elements are not emitted again
# when the timer is fired.
context.clear_timer(str(window), TimeDomain.WATERMARK)
return watermark >= window.end
def on_fire(self, watermark, window, context):
return False
def reset(self, window, context):
context.clear_timer(str(window), TimeDomain.WATERMARK)
def may_lose_data(self, unused_windowing):
return DataLossReason.NO_POTENTIAL_LOSS
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
@staticmethod
def from_runner_api(proto, context):
return DefaultTrigger()
def to_runner_api(self, unused_context):
return beam_runner_api_pb2.Trigger(
default=beam_runner_api_pb2.Trigger.Default())
def has_ontime_pane(self):
return True
class AfterProcessingTime(TriggerFn):
"""Fire exactly once after a specified delay from processing time."""
STATE_TAG = _SetStateTag('has_timer')
def __init__(self, delay=0):
"""Initialize a processing time trigger with a delay in seconds."""
self.delay = delay
def __repr__(self):
return 'AfterProcessingTime(delay=%d)' % self.delay
def on_element(self, element, window, context):
if not context.get_state(self.STATE_TAG):
context.set_timer(
'', TimeDomain.REAL_TIME, context.get_current_time() + self.delay)
context.add_state(self.STATE_TAG, True)
def on_merge(self, to_be_merged, merge_result, context):
# timers will be kept through merging
pass
def should_fire(self, time_domain, timestamp, window, context):
if time_domain == TimeDomain.REAL_TIME:
return True
def on_fire(self, timestamp, window, context):
return True
def reset(self, window, context):
context.clear_state(self.STATE_TAG)
def may_lose_data(self, unused_windowing):
"""AfterProcessingTime may finish."""
return DataLossReason.MAY_FINISH
@staticmethod
def from_runner_api(proto, context):
return AfterProcessingTime(
delay=(
proto.after_processing_time.timestamp_transforms[0].delay.
delay_millis) // 1000)
def to_runner_api(self, context):
delay_proto = beam_runner_api_pb2.TimestampTransform(
delay=beam_runner_api_pb2.TimestampTransform.Delay(
delay_millis=self.delay * 1000))
return beam_runner_api_pb2.Trigger(
after_processing_time=beam_runner_api_pb2.Trigger.AfterProcessingTime(
timestamp_transforms=[delay_proto]))
def has_ontime_pane(self):
return False
class Always(TriggerFn):
"""Repeatedly invoke the given trigger, never finishing."""
def __init__(self):
pass
def __repr__(self):
return 'Always'
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return 1
def on_element(self, element, window, context):
pass
def on_merge(self, to_be_merged, merge_result, context):
pass
def has_ontime_pane(self):
return False
def reset(self, window, context):
pass
def should_fire(self, time_domain, watermark, window, context):
return True
def on_fire(self, watermark, window, context):
return False
def may_lose_data(self, unused_windowing):
"""No potential loss, since the trigger always fires."""
return DataLossReason.NO_POTENTIAL_LOSS
@staticmethod
def from_runner_api(proto, context):
return Always()
def to_runner_api(self, context):
return beam_runner_api_pb2.Trigger(
always=beam_runner_api_pb2.Trigger.Always())
class _Never(TriggerFn):
"""A trigger that never fires.
Data may still be released at window closing.
"""
def __init__(self):
pass
def __repr__(self):
return 'Never'
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
def on_element(self, element, window, context):
pass
def on_merge(self, to_be_merged, merge_result, context):
pass
def has_ontime_pane(self):
False
def reset(self, window, context):
pass
def should_fire(self, time_domain, watermark, window, context):
return False
def on_fire(self, watermark, window, context):
return True
def may_lose_data(self, unused_windowing):
"""No potential data loss.
Though Never doesn't explicitly trigger, it still collects data on
windowing closing.
"""
return DataLossReason.NO_POTENTIAL_LOSS
@staticmethod
def from_runner_api(proto, context):
return _Never()
def to_runner_api(self, context):
return beam_runner_api_pb2.Trigger(
never=beam_runner_api_pb2.Trigger.Never())
class AfterWatermark(TriggerFn):
"""Fire exactly once when the watermark passes the end of the window.
Args:
early: if not None, a speculative trigger to repeatedly evaluate before
the watermark passes the end of the window
late: if not None, a speculative trigger to repeatedly evaluate after
the watermark passes the end of the window
"""
LATE_TAG = _CombiningValueStateTag('is_late', any)
def __init__(self, early=None, late=None):
# TODO(zhoufek): Maybe don't wrap early/late if they are already Repeatedly
self.early = Repeatedly(early) if early else None
self.late = Repeatedly(late) if late else None
def __repr__(self):
qualifiers = []
if self.early:
qualifiers.append('early=%s' % self.early.underlying)
if self.late:
qualifiers.append('late=%s' % self.late.underlying)
return 'AfterWatermark(%s)' % ', '.join(qualifiers)
def is_late(self, context):
return self.late and context.get_state(self.LATE_TAG)
def on_element(self, element, window, context):
if self.is_late(context):
self.late.on_element(element, window, NestedContext(context, 'late'))
else:
context.set_timer('', TimeDomain.WATERMARK, window.end)
if self.early:
self.early.on_element(element, window, NestedContext(context, 'early'))
def on_merge(self, to_be_merged, merge_result, context):
# TODO(robertwb): Figure out whether the 'rewind' semantics could be used
# here.
if self.is_late(context):
self.late.on_merge(
to_be_merged, merge_result, NestedContext(context, 'late'))
else:
# Note: Timer clearing solely an optimization.
for window in to_be_merged:
if window.end != merge_result.end:
context.clear_timer('', TimeDomain.WATERMARK)
if self.early:
self.early.on_merge(
to_be_merged, merge_result, NestedContext(context, 'early'))
def should_fire(self, time_domain, watermark, window, context):
if self.is_late(context):
return self.late.should_fire(
time_domain, watermark, window, NestedContext(context, 'late'))
elif watermark >= window.end:
# Explicitly clear the timer so that late elements are not emitted again
# when the timer is fired.
context.clear_timer('', TimeDomain.WATERMARK)
return True
elif self.early:
return self.early.should_fire(
time_domain, watermark, window, NestedContext(context, 'early'))
return False
def on_fire(self, watermark, window, context):
if self.is_late(context):
return self.late.on_fire(
watermark, window, NestedContext(context, 'late'))
elif watermark >= window.end:
context.add_state(self.LATE_TAG, True)
return not self.late
elif self.early:
self.early.on_fire(watermark, window, NestedContext(context, 'early'))
return False
def reset(self, window, context):
if self.late:
context.clear_state(self.LATE_TAG)
if self.early:
self.early.reset(window, NestedContext(context, 'early'))
if self.late:
self.late.reset(window, NestedContext(context, 'late'))
def may_lose_data(self, windowing):
"""May cause data loss if lateness allowed and no late trigger set."""
if windowing.allowed_lateness == 0:
return DataLossReason.NO_POTENTIAL_LOSS
if self.late is None:
return DataLossReason.MAY_FINISH
return self.late.may_lose_data(windowing)
def __eq__(self, other):
return (
type(self) == type(other) and self.early == other.early and
self.late == other.late)
def __hash__(self):
return hash((type(self), self.early, self.late))
@staticmethod
def from_runner_api(proto, context):
return AfterWatermark(
early=TriggerFn.from_runner_api(
proto.after_end_of_window.early_firings, context)
if proto.after_end_of_window.HasField('early_firings') else None,
late=TriggerFn.from_runner_api(
proto.after_end_of_window.late_firings, context)
if proto.after_end_of_window.HasField('late_firings') else None)
def to_runner_api(self, context):
early_proto = self.early.underlying.to_runner_api(
context) if self.early else None
late_proto = self.late.underlying.to_runner_api(
context) if self.late else None
return beam_runner_api_pb2.Trigger(
after_end_of_window=beam_runner_api_pb2.Trigger.AfterEndOfWindow(
early_firings=early_proto, late_firings=late_proto))
def has_ontime_pane(self):
return True
class AfterCount(TriggerFn):
"""Fire when there are at least count elements in this window pane."""
COUNT_TAG = _CombiningValueStateTag('count', combiners.CountCombineFn())
def __init__(self, count):
if not isinstance(count, numbers.Integral) or count < 1:
raise ValueError("count (%d) must be a positive integer." % count)
self.count = count
def __repr__(self):
return 'AfterCount(%s)' % self.count
def __eq__(self, other):
return type(self) == type(other) and self.count == other.count
def __hash__(self):
return hash(self.count)
def on_element(self, element, window, context):
context.add_state(self.COUNT_TAG, 1)
def on_merge(self, to_be_merged, merge_result, context):
# states automatically merged
pass
def should_fire(self, time_domain, watermark, window, context):
return context.get_state(self.COUNT_TAG) >= self.count
def on_fire(self, watermark, window, context):
return True
def reset(self, window, context):
context.clear_state(self.COUNT_TAG)
def may_lose_data(self, unused_windowing):
"""AfterCount may finish."""
return DataLossReason.MAY_FINISH
@staticmethod
def from_runner_api(proto, unused_context):
return AfterCount(proto.element_count.element_count)
def to_runner_api(self, unused_context):
return beam_runner_api_pb2.Trigger(
element_count=beam_runner_api_pb2.Trigger.ElementCount(
element_count=self.count))
def has_ontime_pane(self):
return False
class Repeatedly(TriggerFn):
"""Repeatedly invoke the given trigger, never finishing."""
def __init__(self, underlying):
self.underlying = underlying
def __repr__(self):
return 'Repeatedly(%s)' % self.underlying
def __eq__(self, other):
return type(self) == type(other) and self.underlying == other.underlying
def __hash__(self):
return hash(self.underlying)
def on_element(self, element, window, context):
self.underlying.on_element(element, window, context)
def on_merge(self, to_be_merged, merge_result, context):
self.underlying.on_merge(to_be_merged, merge_result, context)
def should_fire(self, time_domain, watermark, window, context):
return self.underlying.should_fire(time_domain, watermark, window, context)
def on_fire(self, watermark, window, context):
if self.underlying.on_fire(watermark, window, context):
self.underlying.reset(window, context)
return False
def reset(self, window, context):
self.underlying.reset(window, context)
def may_lose_data(self, windowing):
"""Repeatedly will run in a loop and pick up whatever is left at GC."""
return DataLossReason.NO_POTENTIAL_LOSS
@staticmethod
def from_runner_api(proto, context):
return Repeatedly(
TriggerFn.from_runner_api(proto.repeat.subtrigger, context))
def to_runner_api(self, context):
return beam_runner_api_pb2.Trigger(
repeat=beam_runner_api_pb2.Trigger.Repeat(
subtrigger=self.underlying.to_runner_api(context)))
def has_ontime_pane(self):
return self.underlying.has_ontime_pane()
class _ParallelTriggerFn(TriggerFn, metaclass=ABCMeta):
def __init__(self, *triggers):
self.triggers = triggers
def __repr__(self):
return '%s(%s)' % (
self.__class__.__name__, ', '.join(str(t) for t in self.triggers))
def __eq__(self, other):
return type(self) == type(other) and self.triggers == other.triggers
def __hash__(self):
return hash(self.triggers)
@abstractmethod
def combine_op(self, trigger_results):
pass
def on_element(self, element, window, context):
for ix, trigger in enumerate(self.triggers):
trigger.on_element(element, window, self._sub_context(context, ix))
def on_merge(self, to_be_merged, merge_result, context):
for ix, trigger in enumerate(self.triggers):
trigger.on_merge(
to_be_merged, merge_result, self._sub_context(context, ix))
def should_fire(self, time_domain, watermark, window, context):
self._time_domain = time_domain
return self.combine_op(
trigger.should_fire(
time_domain, watermark, window, self._sub_context(context, ix))
for ix,
trigger in enumerate(self.triggers))
def on_fire(self, watermark, window, context):
finished = []
for ix, trigger in enumerate(self.triggers):
nested_context = self._sub_context(context, ix)
if trigger.should_fire(TimeDomain.WATERMARK,
watermark,
window,
nested_context):
finished.append(trigger.on_fire(watermark, window, nested_context))
return self.combine_op(finished)
def may_lose_data(self, windowing):
may_finish = self.combine_op(
_IncludesMayFinish(t.may_lose_data(windowing)) for t in self.triggers)
return (
DataLossReason.MAY_FINISH
if may_finish else DataLossReason.NO_POTENTIAL_LOSS)
def reset(self, window, context):
for ix, trigger in enumerate(self.triggers):
trigger.reset(window, self._sub_context(context, ix))
@staticmethod
def _sub_context(context, index):
return NestedContext(context, '%d/' % index)
@staticmethod
def from_runner_api(proto, context):
subtriggers = [
TriggerFn.from_runner_api(subtrigger, context) for subtrigger in
proto.after_all.subtriggers or proto.after_any.subtriggers
]
if proto.after_all.subtriggers:
return AfterAll(*subtriggers)
else:
return AfterAny(*subtriggers)
def to_runner_api(self, context):
subtriggers = [
subtrigger.to_runner_api(context) for subtrigger in self.triggers
]
if self.combine_op == all:
return beam_runner_api_pb2.Trigger(
after_all=beam_runner_api_pb2.Trigger.AfterAll(
subtriggers=subtriggers))
elif self.combine_op == any:
return beam_runner_api_pb2.Trigger(
after_any=beam_runner_api_pb2.Trigger.AfterAny(
subtriggers=subtriggers))
else:
raise NotImplementedError(self)
def has_ontime_pane(self):
return any(t.has_ontime_pane() for t in self.triggers)
class AfterAny(_ParallelTriggerFn):
"""Fires when any subtrigger fires.
Also finishes when any subtrigger finishes.
"""
combine_op = any
class AfterAll(_ParallelTriggerFn):
"""Fires when all subtriggers have fired.
Also finishes when all subtriggers have finished.
"""
combine_op = all
class AfterEach(TriggerFn):
INDEX_TAG = _CombiningValueStateTag(
'index', (lambda indices: 0 if not indices else max(indices)))
def __init__(self, *triggers):
self.triggers = triggers
def __repr__(self):
return '%s(%s)' % (
self.__class__.__name__, ', '.join(str(t) for t in self.triggers))
def __eq__(self, other):
return type(self) == type(other) and self.triggers == other.triggers
def __hash__(self):
return hash(self.triggers)
def on_element(self, element, window, context):
ix = context.get_state(self.INDEX_TAG)
if ix < len(self.triggers):
self.triggers[ix].on_element(
element, window, self._sub_context(context, ix))
def on_merge(self, to_be_merged, merge_result, context):
# This takes the furthest window on merging.
# TODO(robertwb): Revisit this when merging windows logic is settled for
# all possible merging situations.
ix = context.get_state(self.INDEX_TAG)
if ix < len(self.triggers):
self.triggers[ix].on_merge(
to_be_merged, merge_result, self._sub_context(context, ix))
def should_fire(self, time_domain, watermark, window, context):
ix = context.get_state(self.INDEX_TAG)
if ix < len(self.triggers):
return self.triggers[ix].should_fire(
time_domain, watermark, window, self._sub_context(context, ix))
def on_fire(self, watermark, window, context):
ix = context.get_state(self.INDEX_TAG)
if ix < len(self.triggers):
if self.triggers[ix].on_fire(watermark,
window,
self._sub_context(context, ix)):
ix += 1
context.add_state(self.INDEX_TAG, ix)
return ix == len(self.triggers)
def reset(self, window, context):
context.clear_state(self.INDEX_TAG)
for ix, trigger in enumerate(self.triggers):
trigger.reset(window, self._sub_context(context, ix))
def may_lose_data(self, windowing):
"""If all sub-triggers may finish, this may finish."""
may_finish = all(
_IncludesMayFinish(t.may_lose_data(windowing)) for t in self.triggers)
return (
DataLossReason.MAY_FINISH
if may_finish else DataLossReason.NO_POTENTIAL_LOSS)
@staticmethod
def _sub_context(context, index):
return NestedContext(context, '%d/' % index)
@staticmethod
def from_runner_api(proto, context):
return AfterEach(
*[
TriggerFn.from_runner_api(subtrigger, context)
for subtrigger in proto.after_each.subtriggers
])
def to_runner_api(self, context):
return beam_runner_api_pb2.Trigger(
after_each=beam_runner_api_pb2.Trigger.AfterEach(
subtriggers=[
subtrigger.to_runner_api(context)
for subtrigger in self.triggers
]))
def has_ontime_pane(self):
return any(t.has_ontime_pane() for t in self.triggers)
class OrFinally(AfterAny):
@staticmethod
def from_runner_api(proto, context):
return OrFinally(
TriggerFn.from_runner_api(proto.or_finally.main, context),
# getattr is used as finally is a keyword in Python
TriggerFn.from_runner_api(
getattr(proto.or_finally, 'finally'), context))
def to_runner_api(self, context):
return beam_runner_api_pb2.Trigger(
or_finally=beam_runner_api_pb2.Trigger.OrFinally(
main=self.triggers[0].to_runner_api(context),
# dict keyword argument is used as finally is a keyword in Python
**{'finally': self.triggers[1].to_runner_api(context)}))
class TriggerContext(object):
def __init__(self, outer, window, clock):
self._outer = outer
self._window = window
self._clock = clock
def get_current_time(self):
return self._clock.time()
def set_timer(self, name, time_domain, timestamp):
self._outer.set_timer(self._window, name, time_domain, timestamp)
def clear_timer(self, name, time_domain):
self._outer.clear_timer(self._window, name, time_domain)
def add_state(self, tag, value):
self._outer.add_state(self._window, tag, value)
def get_state(self, tag):
return self._outer.get_state(self._window, tag)
def clear_state(self, tag):
return self._outer.clear_state(self._window, tag)
class NestedContext(object):
"""Namespaced context useful for defining composite triggers."""
def __init__(self, outer, prefix):
self._outer = outer
self._prefix = prefix
def get_current_time(self):
return self._outer.get_current_time()
def set_timer(self, name, time_domain, timestamp):
self._outer.set_timer(self._prefix + name, time_domain, timestamp)
def clear_timer(self, name, time_domain):
self._outer.clear_timer(self._prefix + name, time_domain)
def add_state(self, tag, value):
self._outer.add_state(tag.with_prefix(self._prefix), value)
def get_state(self, tag):
return self._outer.get_state(tag.with_prefix(self._prefix))