/
bundle_processor.py
1794 lines (1572 loc) · 64.6 KB
/
bundle_processor.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.
#
"""SDK harness for executing Python Fns via the Fn API."""
# pytype: skip-file
from __future__ import absolute_import
from __future__ import division
from __future__ import print_function
import base64
import bisect
import collections
import json
import logging
import random
import threading
from builtins import next
from builtins import object
from typing import TYPE_CHECKING
from typing import Any
from typing import Callable
from typing import Container
from typing import DefaultDict
from typing import Dict
from typing import Iterable
from typing import Iterator
from typing import List
from typing import Mapping
from typing import Optional
from typing import Set
from typing import Tuple
from typing import Type
from typing import TypeVar
from typing import Union
from typing import cast
from future.utils import itervalues
from google.protobuf import duration_pb2
from google.protobuf import timestamp_pb2
import apache_beam as beam
from apache_beam import coders
from apache_beam.coders import WindowedValueCoder
from apache_beam.coders import coder_impl
from apache_beam.internal import pickler
from apache_beam.io import iobase
from apache_beam.metrics import monitoring_infos
from apache_beam.portability import common_urns
from apache_beam.portability import python_urns
from apache_beam.portability.api import beam_fn_api_pb2
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.runners import common
from apache_beam.runners import pipeline_context
from apache_beam.runners.worker import operation_specs
from apache_beam.runners.worker import operations
from apache_beam.runners.worker import statesampler
from apache_beam.transforms import sideinputs
from apache_beam.transforms import userstate
from apache_beam.utils import counters
from apache_beam.utils import proto_utils
from apache_beam.utils import timestamp
if TYPE_CHECKING:
from google.protobuf import message # pylint: disable=ungrouped-imports
from apache_beam import pvalue
from apache_beam.portability.api import metrics_pb2
from apache_beam.runners.sdf_utils import SplitResultPrimary
from apache_beam.runners.sdf_utils import SplitResultResidual
from apache_beam.runners.worker import data_plane
from apache_beam.runners.worker import sdk_worker
from apache_beam.transforms import window
from apache_beam.utils import windowed_value
# This module is experimental. No backwards-compatibility guarantees.
T = TypeVar('T')
ConstructorFn = Callable[[
'BeamTransformFactory',
Any,
beam_runner_api_pb2.PTransform,
Union['message.Message', bytes],
Dict[str, List[operations.Operation]]
],
operations.Operation]
OperationT = TypeVar('OperationT', bound=operations.Operation)
DATA_INPUT_URN = 'beam:runner:source:v1'
DATA_OUTPUT_URN = 'beam:runner:sink:v1'
IDENTITY_DOFN_URN = 'beam:dofn:identity:0.1'
# TODO(vikasrk): Fix this once runner sends appropriate common_urns.
OLD_DATAFLOW_RUNNER_HARNESS_PARDO_URN = 'beam:dofn:javasdk:0.1'
OLD_DATAFLOW_RUNNER_HARNESS_READ_URN = 'beam:source:java:0.1'
URNS_NEEDING_PCOLLECTIONS = set([
monitoring_infos.ELEMENT_COUNT_URN, monitoring_infos.SAMPLED_BYTE_SIZE_URN
])
_LOGGER = logging.getLogger(__name__)
class RunnerIOOperation(operations.Operation):
"""Common baseclass for runner harness IO operations."""
def __init__(self,
name_context, # type: Union[str, common.NameContext]
step_name,
consumers, # type: Mapping[Any, Iterable[operations.Operation]]
counter_factory,
state_sampler,
windowed_coder, # type: coders.Coder
transform_id, # type: str
data_channel # type: data_plane.DataChannel
):
# type: (...) -> None
super(RunnerIOOperation,
self).__init__(name_context, None, counter_factory, state_sampler)
self.windowed_coder = windowed_coder
self.windowed_coder_impl = windowed_coder.get_impl()
# transform_id represents the consumer for the bytes in the data plane for a
# DataInputOperation or a producer of these bytes for a DataOutputOperation.
self.transform_id = transform_id
self.data_channel = data_channel
for _, consumer_ops in consumers.items():
for consumer in consumer_ops:
self.add_receiver(consumer, 0)
class DataOutputOperation(RunnerIOOperation):
"""A sink-like operation that gathers outputs to be sent back to the runner.
"""
def set_output_stream(self, output_stream):
# type: (data_plane.ClosableOutputStream) -> None
self.output_stream = output_stream
def process(self, windowed_value):
# type: (windowed_value.WindowedValue) -> None
self.windowed_coder_impl.encode_to_stream(
windowed_value, self.output_stream, True)
self.output_stream.maybe_flush()
def finish(self):
# type: () -> None
self.output_stream.close()
super(DataOutputOperation, self).finish()
class DataInputOperation(RunnerIOOperation):
"""A source-like operation that gathers input from the runner."""
def __init__(self,
operation_name, # type: Union[str, common.NameContext]
step_name,
consumers, # type: Mapping[Any, Iterable[operations.Operation]]
counter_factory,
state_sampler,
windowed_coder, # type: coders.Coder
transform_id,
data_channel # type: data_plane.GrpcClientDataChannel
):
# type: (...) -> None
super(DataInputOperation, self).__init__(
operation_name,
step_name,
consumers,
counter_factory,
state_sampler,
windowed_coder,
transform_id=transform_id,
data_channel=data_channel)
# We must do this manually as we don't have a spec or spec.output_coders.
self.receivers = [
operations.ConsumerSet.create(
self.counter_factory,
self.name_context.step_name,
0,
next(iter(itervalues(consumers))),
self.windowed_coder)
]
self.splitting_lock = threading.Lock()
self.index = -1
self.stop = float('inf')
self.started = False
def start(self):
# type: () -> None
super(DataInputOperation, self).start()
with self.splitting_lock:
self.started = True
def process(self, windowed_value):
# type: (windowed_value.WindowedValue) -> None
self.output(windowed_value)
def process_encoded(self, encoded_windowed_values):
# type: (bytes) -> None
input_stream = coder_impl.create_InputStream(encoded_windowed_values)
while input_stream.size() > 0:
with self.splitting_lock:
if self.index == self.stop - 1:
return
self.index += 1
decoded_value = self.windowed_coder_impl.decode_from_stream(
input_stream, True)
self.output(decoded_value)
def monitoring_infos(self, transform_id, tag_to_pcollection_id):
# type: (str, Dict[str, str]) -> Dict[FrozenSet, metrics_pb2.MonitoringInfo]
all_monitoring_infos = super(DataInputOperation, self).monitoring_infos(
transform_id, tag_to_pcollection_id)
read_progress_info = monitoring_infos.int64_counter(
monitoring_infos.DATA_CHANNEL_READ_INDEX,
self.index,
ptransform=transform_id)
all_monitoring_infos[monitoring_infos.to_key(
read_progress_info)] = read_progress_info
return all_monitoring_infos
def try_split(
self, fraction_of_remainder, total_buffer_size, allowed_split_points):
# type: (...) -> Optional[Tuple[int, Optional[operations.SdfSplitResultsPrimary], Optional[operations.SdfSplitResultsResidual], int]]
with self.splitting_lock:
if not self.started:
return None
if self.index == -1:
# We are "finished" with the (non-existent) previous element.
current_element_progress = 1.0
else:
current_element_progress_object = (
self.receivers[0].current_element_progress())
if current_element_progress_object is None:
current_element_progress = 0.5
else:
current_element_progress = (
current_element_progress_object.fraction_completed)
# Now figure out where to split.
split = self._compute_split(
self.index,
current_element_progress,
self.stop,
fraction_of_remainder,
total_buffer_size,
allowed_split_points,
self.receivers[0].try_split)
if split:
self.stop = split[-1]
return split
@staticmethod
def _compute_split(
index,
current_element_progress,
stop,
fraction_of_remainder,
total_buffer_size,
allowed_split_points=(),
try_split=lambda fraction: None):
def is_valid_split_point(index):
return not allowed_split_points or index in allowed_split_points
if total_buffer_size < index + 1:
total_buffer_size = index + 1
elif total_buffer_size > stop:
total_buffer_size = stop
# The units here (except for keep_of_element_remainder) are all in
# terms of number of (possibly fractional) elements.
remainder = total_buffer_size - index - current_element_progress
keep = remainder * fraction_of_remainder
if current_element_progress < 1:
keep_of_element_remainder = keep / (1 - current_element_progress)
# If it's less than what's left of the current element,
# try splitting at the current element.
if (keep_of_element_remainder < 1 and is_valid_split_point(index) and
is_valid_split_point(index + 1)):
split = try_split(
keep_of_element_remainder
) # type: Optional[Tuple[operations.SdfSplitResultsPrimary, operations.SdfSplitResultsResidual]]
if split:
element_primary, element_residual = split
return index - 1, element_primary, element_residual, index + 1
# Otherwise, split at the closest element boundary.
# pylint: disable=round-builtin
stop_index = index + max(1, int(round(current_element_progress + keep)))
if allowed_split_points and stop_index not in allowed_split_points:
# Choose the closest allowed split point.
allowed_split_points = sorted(allowed_split_points)
closest = bisect.bisect(allowed_split_points, stop_index)
if closest == 0:
stop_index = allowed_split_points[0]
elif closest == len(allowed_split_points):
stop_index = allowed_split_points[-1]
else:
prev = allowed_split_points[closest - 1]
next = allowed_split_points[closest]
if index < prev and stop_index - prev < next - stop_index:
stop_index = prev
else:
stop_index = next
if index < stop_index < stop:
return stop_index - 1, None, None, stop_index
else:
return None
def finish(self):
# type: () -> None
with self.splitting_lock:
self.index += 1
self.started = False
def reset(self):
# type: () -> None
self.index = -1
self.stop = float('inf')
super(DataInputOperation, self).reset()
class _StateBackedIterable(object):
def __init__(self,
state_handler, # type: sdk_worker.CachingStateHandler
state_key, # type: beam_fn_api_pb2.StateKey
coder_or_impl, # type: Union[coders.Coder, coder_impl.CoderImpl]
is_cached=False
):
# type: (...) -> None
self._state_handler = state_handler
self._state_key = state_key
if isinstance(coder_or_impl, coders.Coder):
self._coder_impl = coder_or_impl.get_impl()
else:
self._coder_impl = coder_or_impl
self._is_cached = is_cached
def __iter__(self):
# type: () -> Iterator[Any]
return self._state_handler.blocking_get(
self._state_key, self._coder_impl, is_cached=self._is_cached)
def __reduce__(self):
return list, (list(self), )
coder_impl.FastPrimitivesCoderImpl.register_iterable_like_type(
_StateBackedIterable)
class StateBackedSideInputMap(object):
def __init__(self,
state_handler, # type: sdk_worker.CachingStateHandler
transform_id, # type: str
tag, # type: Optional[str]
side_input_data, # type: pvalue.SideInputData
coder # type: WindowedValueCoder
):
# type: (...) -> None
self._state_handler = state_handler
self._transform_id = transform_id
self._tag = tag
self._side_input_data = side_input_data
self._element_coder = coder.wrapped_value_coder
self._target_window_coder = coder.window_coder
# TODO(robertwb): Limit the cache size.
self._cache = {} # type: Dict[window.BoundedWindow, Any]
def __getitem__(self, window):
target_window = self._side_input_data.window_mapping_fn(window)
if target_window not in self._cache:
state_handler = self._state_handler
access_pattern = self._side_input_data.access_pattern
if access_pattern == common_urns.side_inputs.ITERABLE.urn:
state_key = beam_fn_api_pb2.StateKey(
iterable_side_input=beam_fn_api_pb2.StateKey.IterableSideInput(
transform_id=self._transform_id,
side_input_id=self._tag,
window=self._target_window_coder.encode(target_window)))
raw_view = _StateBackedIterable(
state_handler, state_key, self._element_coder)
elif access_pattern == common_urns.side_inputs.MULTIMAP.urn:
state_key = beam_fn_api_pb2.StateKey(
multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput(
transform_id=self._transform_id,
side_input_id=self._tag,
window=self._target_window_coder.encode(target_window),
key=b''))
cache = {}
key_coder_impl = self._element_coder.key_coder().get_impl()
value_coder = self._element_coder.value_coder()
class MultiMap(object):
def __getitem__(self, key):
if key not in cache:
keyed_state_key = beam_fn_api_pb2.StateKey()
keyed_state_key.CopyFrom(state_key)
keyed_state_key.multimap_side_input.key = (
key_coder_impl.encode_nested(key))
cache[key] = _StateBackedIterable(
state_handler, keyed_state_key, value_coder)
return cache[key]
def __reduce__(self):
# TODO(robertwb): Figure out how to support this.
raise TypeError(common_urns.side_inputs.MULTIMAP.urn)
raw_view = MultiMap()
else:
raise ValueError("Unknown access pattern: '%s'" % access_pattern)
self._cache[target_window] = self._side_input_data.view_fn(raw_view)
return self._cache[target_window]
def is_globally_windowed(self):
# type: () -> bool
return (
self._side_input_data.window_mapping_fn ==
sideinputs._global_window_mapping_fn)
def reset(self):
# type: () -> None
# TODO(BEAM-5428): Cross-bundle caching respecting cache tokens.
self._cache = {}
class CombiningValueRuntimeState(userstate.CombiningValueRuntimeState):
def __init__(self, underlying_bag_state, combinefn):
self._combinefn = combinefn
self._underlying_bag_state = underlying_bag_state
def _read_accumulator(self, rewrite=True):
merged_accumulator = self._combinefn.merge_accumulators(
self._underlying_bag_state.read())
if rewrite:
self._underlying_bag_state.clear()
self._underlying_bag_state.add(merged_accumulator)
return merged_accumulator
def read(self):
# type: () -> Iterable[Any]
return self._combinefn.extract_output(self._read_accumulator())
def add(self, value):
# type: (Any) -> None
# Prefer blind writes, but don't let them grow unboundedly.
# This should be tuned to be much lower, but for now exercise
# both paths well.
if random.random() < 0.5:
accumulator = self._read_accumulator(False)
self._underlying_bag_state.clear()
else:
accumulator = self._combinefn.create_accumulator()
self._underlying_bag_state.add(
self._combinefn.add_input(accumulator, value))
def clear(self):
# type: () -> None
self._underlying_bag_state.clear()
def commit(self):
self._underlying_bag_state.commit()
class _ConcatIterable(object):
"""An iterable that is the concatination of two iterables.
Unlike itertools.chain, this allows reiteration.
"""
def __init__(self, first, second):
# type: (Iterable[Any], Iterable[Any]) -> None
self.first = first
self.second = second
def __iter__(self):
# type: () -> Iterator[Any]
for elem in self.first:
yield elem
for elem in self.second:
yield elem
coder_impl.FastPrimitivesCoderImpl.register_iterable_like_type(_ConcatIterable)
class SynchronousBagRuntimeState(userstate.BagRuntimeState):
def __init__(self,
state_handler, # type: sdk_worker.CachingStateHandler
state_key, # type: beam_fn_api_pb2.StateKey
value_coder # type: coders.Coder
):
# type: (...) -> None
self._state_handler = state_handler
self._state_key = state_key
self._value_coder = value_coder
self._cleared = False
self._added_elements = [] # type: List[Any]
def read(self):
# type: () -> Iterable[Any]
return _ConcatIterable([] if self._cleared else cast(
'Iterable[Any]',
_StateBackedIterable(
self._state_handler,
self._state_key,
self._value_coder,
is_cached=True)),
self._added_elements)
def add(self, value):
# type: (Any) -> None
self._added_elements.append(value)
def clear(self):
# type: () -> None
self._cleared = True
self._added_elements = []
def commit(self):
to_await = None
if self._cleared:
to_await = self._state_handler.clear(self._state_key, is_cached=True)
if self._added_elements:
to_await = self._state_handler.extend(
self._state_key,
self._value_coder.get_impl(),
self._added_elements,
is_cached=True)
if to_await:
# To commit, we need to wait on the last state request future to complete.
to_await.get()
class SynchronousSetRuntimeState(userstate.SetRuntimeState):
def __init__(self,
state_handler, # type: sdk_worker.CachingStateHandler
state_key, # type: beam_fn_api_pb2.StateKey
value_coder # type: coders.Coder
):
# type: (...) -> None
self._state_handler = state_handler
self._state_key = state_key
self._value_coder = value_coder
self._cleared = False
self._added_elements = set() # type: Set[Any]
def _compact_data(self, rewrite=True):
accumulator = set(
_ConcatIterable(
set() if self._cleared else _StateBackedIterable(
self._state_handler,
self._state_key,
self._value_coder,
is_cached=True),
self._added_elements))
if rewrite and accumulator:
self._state_handler.clear(self._state_key, is_cached=True)
self._state_handler.extend(
self._state_key,
self._value_coder.get_impl(),
accumulator,
is_cached=True)
# Since everthing is already committed so we can safely reinitialize
# added_elements here.
self._added_elements = set()
return accumulator
def read(self):
# type: () -> Set[Any]
return self._compact_data(rewrite=False)
def add(self, value):
# type: (Any) -> None
if self._cleared:
# This is a good time explicitly clear.
self._state_handler.clear(self._state_key, is_cached=True)
self._cleared = False
self._added_elements.add(value)
if random.random() > 0.5:
self._compact_data()
def clear(self):
# type: () -> None
self._cleared = True
self._added_elements = set()
def commit(self):
# type: () -> None
to_await = None
if self._cleared:
to_await = self._state_handler.clear(self._state_key, is_cached=True)
if self._added_elements:
to_await = self._state_handler.extend(
self._state_key,
self._value_coder.get_impl(),
self._added_elements,
is_cached=True)
if to_await:
# To commit, we need to wait on the last state request future to complete.
to_await.get()
class OutputTimer(object):
def __init__(self,
key,
window, # type: windowed_value.BoundedWindow
timestamp, # type: timestamp.Timestamp
paneinfo, # type: windowed_value.PaneInfo
timer_family_id, # type: str
timer_coder_impl, # type: coder_impl.TimerCoderImpl
output_stream # type: data_plane.ClosableOutputStream
):
self._key = key
self._window = window
self._input_timestamp = timestamp
self._paneinfo = paneinfo
self._timer_family_id = timer_family_id
self._output_stream = output_stream
self._timer_coder_impl = timer_coder_impl
def set(self, ts):
ts = timestamp.Timestamp.of(ts)
timer = userstate.Timer(
user_key=self._key,
dynamic_timer_tag='',
windows=(self._window, ),
clear_bit=False,
fire_timestamp=ts,
hold_timestamp=self._input_timestamp,
paneinfo=self._paneinfo)
self._timer_coder_impl.encode_to_stream(timer, self._output_stream, True)
self._output_stream.maybe_flush()
def clear(self):
# type: () -> None
timer = userstate.Timer(
user_key=self._key,
dynamic_timer_tag='',
windows=(self._window, ),
clear_bit=True,
fire_timestamp=None,
hold_timestamp=None,
paneinfo=None)
self._timer_coder_impl.encode_to_stream(timer, self._output_stream, True)
self._output_stream.maybe_flush()
class TimerInfo(object):
"""A data class to store information related to a timer."""
def __init__(self, timer_coder_impl, output_stream=None):
self.timer_coder_impl = timer_coder_impl
self.output_stream = output_stream
class FnApiUserStateContext(userstate.UserStateContext):
"""Interface for state and timers from SDK to Fn API servicer of state.."""
def __init__(self,
state_handler, # type: sdk_worker.CachingStateHandler
transform_id, # type: str
key_coder, # type: coders.Coder
window_coder, # type: coders.Coder
):
# type: (...) -> None
"""Initialize a ``FnApiUserStateContext``.
Args:
state_handler: A StateServicer object.
transform_id: The name of the PTransform that this context is associated.
key_coder:
window_coder:
timer_family_specs: A list of ``userstate.TimerSpec`` objects specifying
the timers associated with this operation.
"""
self._state_handler = state_handler
self._transform_id = transform_id
self._key_coder = key_coder
self._window_coder = window_coder
# A mapping of {timer_family_id: TimerInfo}
self._timers_info = {}
self._all_states = {
} # type: Dict[tuple, userstate.AccumulatingRuntimeState]
def add_timer_info(self, timer_family_id, timer_info):
self._timers_info[timer_family_id] = timer_info
def get_timer(
self,
timer_spec,
key,
window, # type: windowed_value.BoundedWindow
timestamp,
pane):
# type: (...) -> OutputTimer
assert self._timers_info[timer_spec.name].output_stream is not None
return OutputTimer(
key,
window,
timestamp,
pane,
timer_spec.name,
self._timers_info[timer_spec.name].timer_coder_impl,
self._timers_info[timer_spec.name].output_stream)
def get_state(self, *args):
state_handle = self._all_states.get(args)
if state_handle is None:
state_handle = self._all_states[args] = self._create_state(*args)
return state_handle
def _create_state(self,
state_spec, # type: userstate.StateSpec
key,
window # type: windowed_value.BoundedWindow
):
# type: (...) -> userstate.AccumulatingRuntimeState
if isinstance(state_spec,
(userstate.BagStateSpec, userstate.CombiningValueStateSpec)):
bag_state = SynchronousBagRuntimeState(
self._state_handler,
state_key=beam_fn_api_pb2.StateKey(
bag_user_state=beam_fn_api_pb2.StateKey.BagUserState(
transform_id=self._transform_id,
user_state_id=state_spec.name,
window=self._window_coder.encode(window),
# State keys are expected in nested encoding format
key=self._key_coder.encode_nested(key))),
value_coder=state_spec.coder)
if isinstance(state_spec, userstate.BagStateSpec):
return bag_state
else:
return CombiningValueRuntimeState(bag_state, state_spec.combine_fn)
elif isinstance(state_spec, userstate.SetStateSpec):
return SynchronousSetRuntimeState(
self._state_handler,
state_key=beam_fn_api_pb2.StateKey(
bag_user_state=beam_fn_api_pb2.StateKey.BagUserState(
transform_id=self._transform_id,
user_state_id=state_spec.name,
window=self._window_coder.encode(window),
# State keys are expected in nested encoding format
key=self._key_coder.encode_nested(key))),
value_coder=state_spec.coder)
else:
raise NotImplementedError(state_spec)
def commit(self):
# type: () -> None
for state in self._all_states.values():
state.commit()
def reset(self):
# type: () -> None
# TODO(BEAM-5428): Implement cross-bundle state caching.
self._all_states = {}
self._timer_output_streams = {}
self._timer_coders_impl = {}
def memoize(func):
cache = {}
missing = object()
def wrapper(*args):
result = cache.get(args, missing)
if result is missing:
result = cache[args] = func(*args)
return result
return wrapper
def only_element(iterable):
# type: (Iterable[T]) -> T
element, = iterable
return element
class BundleProcessor(object):
""" A class for processing bundles of elements. """
def __init__(self,
process_bundle_descriptor, # type: beam_fn_api_pb2.ProcessBundleDescriptor
state_handler, # type: sdk_worker.CachingStateHandler
data_channel_factory # type: data_plane.DataChannelFactory
):
# type: (...) -> None
"""Initialize a bundle processor.
Args:
process_bundle_descriptor (``beam_fn_api_pb2.ProcessBundleDescriptor``):
a description of the stage that this ``BundleProcessor``is to execute.
state_handler (CachingStateHandler).
data_channel_factory (``data_plane.DataChannelFactory``).
"""
self.process_bundle_descriptor = process_bundle_descriptor
self.state_handler = state_handler
self.data_channel_factory = data_channel_factory
# There is no guarantee that the runner only set
# timer_api_service_descriptor when having timers. So this field cannot be
# used as an indicator of timers.
if self.process_bundle_descriptor.timer_api_service_descriptor:
self.timer_data_channel = (
data_channel_factory.create_data_channel_from_url(
self.process_bundle_descriptor.timer_api_service_descriptor.url))
else:
self.timer_data_channel = None
# A mapping of
# {(transform_id, timer_family_id): TimerInfo}
# The mapping is empty when there is no timer_family_specs in the
# ProcessBundleDescriptor.
self.timers_info = {}
# TODO(robertwb): Figure out the correct prefix to use for output counters
# from StateSampler.
self.counter_factory = counters.CounterFactory()
self.state_sampler = statesampler.StateSampler(
'fnapi-step-%s' % self.process_bundle_descriptor.id,
self.counter_factory)
self.ops = self.create_execution_tree(self.process_bundle_descriptor)
for op in self.ops.values():
op.setup()
self.splitting_lock = threading.Lock()
def create_execution_tree(
self,
descriptor # type: beam_fn_api_pb2.ProcessBundleDescriptor
):
# type: (...) -> collections.OrderedDict[str, operations.Operation]
transform_factory = BeamTransformFactory(
descriptor,
self.data_channel_factory,
self.counter_factory,
self.state_sampler,
self.state_handler)
self.timers_info = transform_factory.extract_timers_info()
def is_side_input(transform_proto, tag):
if transform_proto.spec.urn == common_urns.primitives.PAR_DO.urn:
return tag in proto_utils.parse_Bytes(
transform_proto.spec.payload,
beam_runner_api_pb2.ParDoPayload).side_inputs
pcoll_consumers = collections.defaultdict(
list) # type: DefaultDict[str, List[str]]
for transform_id, transform_proto in descriptor.transforms.items():
for tag, pcoll_id in transform_proto.inputs.items():
if not is_side_input(transform_proto, tag):
pcoll_consumers[pcoll_id].append(transform_id)
@memoize
def get_operation(transform_id):
# type: (str) -> operations.Operation
transform_consumers = {
tag: [get_operation(op) for op in pcoll_consumers[pcoll_id]]
for tag,
pcoll_id in descriptor.transforms[transform_id].outputs.items()
}
return transform_factory.create_operation(
transform_id, transform_consumers)
# Operations must be started (hence returned) in order.
@memoize
def topological_height(transform_id):
# type: (str) -> int
return 1 + max([0] + [
topological_height(consumer)
for pcoll in descriptor.transforms[transform_id].outputs.values()
for consumer in pcoll_consumers[pcoll]
])
return collections.OrderedDict([
(transform_id, get_operation(transform_id)) for transform_id in sorted(
descriptor.transforms, key=topological_height, reverse=True)
])
def reset(self):
# type: () -> None
self.counter_factory.reset()
self.state_sampler.reset()
# Side input caches.
for op in self.ops.values():
op.reset()
def process_bundle(self, instruction_id):
# type: (str) -> Tuple[List[beam_fn_api_pb2.DelayedBundleApplication], bool]
expected_inputs = []
for op in self.ops.values():
if isinstance(op, DataOutputOperation):
# TODO(robertwb): Is there a better way to pass the instruction id to
# the operation?
op.set_output_stream(
op.data_channel.output_stream(instruction_id, op.transform_id))
elif isinstance(op, DataInputOperation):
# We must wait until we receive "end of stream" for each of these ops.
expected_inputs.append(op)
try:
execution_context = ExecutionContext()
self.state_sampler.start()
# Start all operations.
for op in reversed(self.ops.values()):
_LOGGER.debug('start %s', op)
op.execution_context = execution_context
op.start()
# Each data_channel is mapped to a list of expected inputs which includes
# both data input and timer input. The data input is identied by
# transform_id. The data input is identified by
# (transform_id, timer_family_id).
data_channels = collections.defaultdict(
list
) # type: DefaultDict[data_plane.GrpcClientDataChannel, List[str]]
# Add expected data inputs for each data channel.
input_op_by_transform_id = {}
for input_op in expected_inputs:
data_channels[input_op.data_channel].append(input_op.transform_id)
input_op_by_transform_id[input_op.transform_id] = input_op
# Update timer_data channel with expected timer inputs.
if self.timer_data_channel:
data_channels[self.timer_data_channel].extend(
list(self.timers_info.keys()))
# Set up timer output stream for DoOperation.
for ((transform_id, timer_family_id),
timer_info) in self.timers_info.items():
output_stream = self.timer_data_channel.output_timer_stream(
instruction_id, transform_id, timer_family_id)
timer_info.output_stream = output_stream
self.ops[transform_id].add_timer_info(timer_family_id, timer_info)
# Process data and timer inputs
for data_channel, expected_inputs in data_channels.items():
for element in data_channel.input_elements(instruction_id,
expected_inputs):
if isinstance(element, beam_fn_api_pb2.Elements.Timer):
timer_coder_impl = (
self.timers_info[(
element.transform_id,
element.timer_family_id)].timer_coder_impl)
for timer_data in timer_coder_impl.decode_all(element.timers):
self.ops[element.transform_id].process_timer(
element.timer_family_id, timer_data)
elif isinstance(element, beam_fn_api_pb2.Elements.Data):
input_op_by_transform_id[element.transform_id].process_encoded(
element.data)
# Finish all operations.
for op in self.ops.values():
_LOGGER.debug('finish %s', op)
op.finish()
# Close every timer output stream
for timer_info in self.timers_info.values():
assert timer_info.output_stream is not None
timer_info.output_stream.close()
return ([
self.delayed_bundle_application(op, residual) for op,
residual in execution_context.delayed_applications
],
self.requires_finalization())
finally:
# Ensure any in-flight split attempts complete.
with self.splitting_lock:
pass
self.state_sampler.stop_if_still_running()
def finalize_bundle(self):
# type: () -> beam_fn_api_pb2.FinalizeBundleResponse
for op in self.ops.values():
op.finalize_bundle()
return beam_fn_api_pb2.FinalizeBundleResponse()