forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathsdk_worker.py
More file actions
1467 lines (1270 loc) · 52.8 KB
/
sdk_worker.py
File metadata and controls
1467 lines (1270 loc) · 52.8 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
#
# 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
# mypy: disallow-untyped-defs
import abc
import collections
import contextlib
import functools
import json
import logging
import queue
import sys
import threading
import time
import traceback
from concurrent import futures
from typing import TYPE_CHECKING
from typing import Any
from typing import Callable
from typing import DefaultDict
from typing import Dict
from typing import FrozenSet
from typing import Generic
from typing import Iterable
from typing import Iterator
from typing import List
from typing import MutableMapping
from typing import Optional
from typing import Tuple
from typing import TypeVar
from typing import Union
import grpc
from apache_beam.coders import coder_impl
from apache_beam.metrics import monitoring_infos
from apache_beam.metrics.execution import MetricsEnvironment
from apache_beam.portability.api import beam_fn_api_pb2
from apache_beam.portability.api import beam_fn_api_pb2_grpc
from apache_beam.portability.api import metrics_pb2
from apache_beam.runners.worker import bundle_processor
from apache_beam.runners.worker import data_plane
from apache_beam.runners.worker import data_sampler
from apache_beam.runners.worker import statesampler
from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
from apache_beam.runners.worker.data_plane import PeriodicThread
from apache_beam.runners.worker.statecache import CacheAware
from apache_beam.runners.worker.statecache import StateCache
from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor
from apache_beam.runners.worker.worker_status import FnApiWorkerStatusHandler
from apache_beam.utils import thread_pool_executor
from apache_beam.utils.sentinel import Sentinel
from apache_beam.version import __version__ as beam_version
if TYPE_CHECKING:
from apache_beam.portability.api import endpoints_pb2
from apache_beam.utils.profiler import Profile
T = TypeVar('T')
_KT = TypeVar('_KT')
_VT = TypeVar('_VT')
_LOGGER = logging.getLogger(__name__)
DEFAULT_BUNDLE_PROCESSOR_CACHE_SHUTDOWN_THRESHOLD_S = 60
# The number of ProcessBundleRequest instruction ids the BundleProcessorCache
# will remember for not running instructions.
MAX_KNOWN_NOT_RUNNING_INSTRUCTIONS = 1000
# The number of ProcessBundleRequest instruction ids that BundleProcessorCache
# will remember for failed instructions.
MAX_FAILED_INSTRUCTIONS = 10000
# retry on transient UNAVAILABLE grpc error from state channels.
_GRPC_SERVICE_CONFIG = json.dumps({
"methodConfig": [{
"name": [{
"service": "org.apache.beam.model.fn_execution.v1.BeamFnState"
}],
"retryPolicy": {
"maxAttempts": 5,
"initialBackoff": "0.1s",
"maxBackoff": "5s",
"backoffMultiplier": 2,
"retryableStatusCodes": ["UNAVAILABLE"],
},
}]
})
class ShortIdCache(object):
""" Cache for MonitoringInfo "short ids"
"""
def __init__(self):
# type: () -> None
self._lock = threading.Lock()
self._last_short_id = 0
self._info_key_to_short_id = {} # type: Dict[FrozenSet, str]
self._short_id_to_info = {} # type: Dict[str, metrics_pb2.MonitoringInfo]
def get_short_id(self, monitoring_info):
# type: (metrics_pb2.MonitoringInfo) -> str
""" Returns the assigned shortId for a given MonitoringInfo, assigns one if
not assigned already.
"""
key = monitoring_infos.to_key(monitoring_info)
with self._lock:
try:
return self._info_key_to_short_id[key]
except KeyError:
self._last_short_id += 1
# Convert to a hex string (and drop the '0x') for some compression
shortId = hex(self._last_short_id)[2:]
payload_cleared = metrics_pb2.MonitoringInfo()
payload_cleared.CopyFrom(monitoring_info)
payload_cleared.ClearField('payload')
self._info_key_to_short_id[key] = shortId
self._short_id_to_info[shortId] = payload_cleared
return shortId
def get_infos(self, short_ids):
#type: (Iterable[str]) -> Dict[str, metrics_pb2.MonitoringInfo]
""" Gets the base MonitoringInfo (with payload cleared) for each short ID.
Throws KeyError if an unassigned short ID is encountered.
"""
return {
short_id: self._short_id_to_info[short_id]
for short_id in short_ids
}
SHORT_ID_CACHE = ShortIdCache()
class SdkHarness(object):
REQUEST_METHOD_PREFIX = '_request_'
def __init__(
self,
control_address, # type: str
credentials=None, # type: Optional[grpc.ChannelCredentials]
worker_id=None, # type: Optional[str]
# Caching is disabled by default
state_cache_size=0, # type: int
# time-based data buffering is disabled by default
data_buffer_time_limit_ms=0, # type: int
profiler_factory=None, # type: Optional[Callable[..., Profile]]
status_address=None, # type: Optional[str]
# Heap dump through status api is disabled by default
enable_heap_dump=False, # type: bool
data_sampler=None, # type: Optional[data_sampler.DataSampler]
# Unrecoverable SDK harness initialization error (if any)
# that should be reported to the runner when proocessing the first bundle.
deferred_exception=None, # type: Optional[Exception]
runner_capabilities=frozenset(), # type: FrozenSet[str]
element_processing_timeout_minutes=None, # type: Optional[int]
):
# type: (...) -> None
self._alive = True
self._worker_index = 0
self._worker_id = worker_id
self._state_cache = StateCache(state_cache_size)
self._deferred_exception = deferred_exception
options = [('grpc.max_receive_message_length', -1),
('grpc.max_send_message_length', -1)]
if credentials is None:
_LOGGER.info('Creating insecure control channel for %s.', control_address)
self._control_channel = GRPCChannelFactory.insecure_channel(
control_address, options=options)
else:
_LOGGER.info('Creating secure control channel for %s.', control_address)
self._control_channel = GRPCChannelFactory.secure_channel(
control_address, credentials, options=options)
grpc.channel_ready_future(self._control_channel).result(timeout=60)
_LOGGER.info('Control channel established.')
self._control_channel = grpc.intercept_channel(
self._control_channel, WorkerIdInterceptor(self._worker_id))
self._data_channel_factory = data_plane.GrpcClientDataChannelFactory(
credentials, self._worker_id, data_buffer_time_limit_ms)
self._state_handler_factory = GrpcStateHandlerFactory(
state_cache=self._state_cache,
credentials=credentials,
worker_id=self._worker_id)
self._profiler_factory = profiler_factory
self.data_sampler = data_sampler
self.runner_capabilities = runner_capabilities
self._element_processing_timeout_minutes = element_processing_timeout_minutes
def default_factory(id):
# type: (str) -> beam_fn_api_pb2.ProcessBundleDescriptor
return self._control_stub.GetProcessBundleDescriptor(
beam_fn_api_pb2.GetProcessBundleDescriptorRequest(
process_bundle_descriptor_id=id))
self._fns = KeyedDefaultDict(default_factory)
# BundleProcessor cache across all workers.
self._bundle_processor_cache = BundleProcessorCache(
self.runner_capabilities,
state_handler_factory=self._state_handler_factory,
data_channel_factory=self._data_channel_factory,
fns=self._fns,
data_sampler=self.data_sampler,
)
self._status_handler = None # type: Optional[FnApiWorkerStatusHandler]
if status_address:
try:
self._status_handler = FnApiWorkerStatusHandler(
status_address,
self._bundle_processor_cache,
self._state_cache,
enable_heap_dump,
element_processing_timeout_minutes=self.
_element_processing_timeout_minutes)
except Exception:
traceback_string = traceback.format_exc()
_LOGGER.warning(
'Error creating worker status request handler, '
'skipping status report. Trace back: %s' % traceback_string)
# TODO(BEAM-8998) use common
# thread_pool_executor.shared_unbounded_instance() to process bundle
# progress once dataflow runner's excessive progress polling is removed.
self._report_progress_executor = futures.ThreadPoolExecutor(max_workers=1)
self._worker_thread_pool = thread_pool_executor.shared_unbounded_instance()
self._responses = queue.Queue(
) # type: queue.Queue[Union[beam_fn_api_pb2.InstructionResponse, Sentinel]]
_LOGGER.info('Initializing SDKHarness with unbounded number of workers.')
def run(self):
# type: () -> None
self._control_stub = beam_fn_api_pb2_grpc.BeamFnControlStub(
self._control_channel)
no_more_work = Sentinel.sentinel
def get_responses():
# type: () -> Iterator[beam_fn_api_pb2.InstructionResponse]
while True:
response = self._responses.get()
if response is no_more_work:
return
yield response
self._alive = True
try:
for work_request in self._control_stub.Control(get_responses()):
_LOGGER.debug('Got work %s', work_request.instruction_id)
request_type = work_request.WhichOneof('request')
if request_type is None:
raise RuntimeError(
"Cannot interpret a request received over control channel. "
"This is not expected. "
"Verify that SDK was not accidentally downgraded at runtime. "
f"SDK version: {beam_version}, "
f"instruction id: {work_request.instruction_id}, "
f"raw request: {str(work_request.SerializeToString())}")
# Name spacing the request method with 'request_'. The called method
# will be like self.request_register(request)
getattr(self, SdkHarness.REQUEST_METHOD_PREFIX + request_type)(
work_request)
finally:
self._alive = False
if self.data_sampler:
self.data_sampler.stop()
_LOGGER.info('No more requests from control plane')
_LOGGER.info('SDK Harness waiting for in-flight requests to complete')
# Wait until existing requests are processed.
self._worker_thread_pool.shutdown()
# get_responses may be blocked on responses.get(), but we need to return
# control to its caller.
self._responses.put(no_more_work)
# Stop all the workers and clean all the associated resources
self._data_channel_factory.close()
self._state_handler_factory.close()
self._bundle_processor_cache.shutdown()
if self._status_handler:
self._status_handler.close()
_LOGGER.info('Done consuming work.')
def _execute(
self,
task, # type: Callable[[], beam_fn_api_pb2.InstructionResponse]
request # type: beam_fn_api_pb2.InstructionRequest
):
# type: (...) -> None
with statesampler.instruction_id(request.instruction_id):
try:
response = task()
except: # pylint: disable=bare-except
traceback_string = traceback.format_exc()
print(traceback_string, file=sys.stderr)
_LOGGER.error(
'Error processing instruction %s. Original traceback is\n%s\n',
request.instruction_id,
traceback_string)
response = beam_fn_api_pb2.InstructionResponse(
instruction_id=request.instruction_id, error=traceback_string)
self._responses.put(response)
def _request_register(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
# registration request is handled synchronously
self._execute(lambda: self.create_worker().do_instruction(request), request)
def _request_process_bundle(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
if self._deferred_exception:
raise self._deferred_exception
self._bundle_processor_cache.activate(request.instruction_id)
self._request_execute(request)
def _request_process_bundle_split(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
self._request_process_bundle_action(request)
def _request_process_bundle_progress(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
self._request_process_bundle_action(request)
def _request_process_bundle_action(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
def task():
# type: () -> None
self._execute(
lambda: self.create_worker().do_instruction(request), request)
self._report_progress_executor.submit(task)
def _request_finalize_bundle(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
self._request_execute(request)
def _request_harness_monitoring_infos(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
process_wide_monitoring_infos = MetricsEnvironment.process_wide_container(
).to_runner_api_monitoring_infos(None).values()
self._execute(
lambda: beam_fn_api_pb2.InstructionResponse(
instruction_id=request.instruction_id, harness_monitoring_infos=(
beam_fn_api_pb2.HarnessMonitoringInfosResponse(
monitoring_data={
SHORT_ID_CACHE.get_short_id(info): info.payload
for info in process_wide_monitoring_infos
}))),
request)
def _request_monitoring_infos(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
self._execute(
lambda: beam_fn_api_pb2.InstructionResponse(
instruction_id=request.instruction_id, monitoring_infos=
beam_fn_api_pb2.MonitoringInfosMetadataResponse(
monitoring_info=SHORT_ID_CACHE.get_infos(
request.monitoring_infos.monitoring_info_id))),
request)
def _request_execute(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
def task():
# type: () -> None
self._execute(
lambda: self.create_worker().do_instruction(request), request)
self._worker_thread_pool.submit(task)
_LOGGER.debug(
"Currently using %s threads." % len(self._worker_thread_pool._workers))
def _request_sample_data(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
def get_samples(request):
# type: (beam_fn_api_pb2.InstructionRequest) -> beam_fn_api_pb2.InstructionResponse
samples = beam_fn_api_pb2.SampleDataResponse()
if self.data_sampler is not None:
samples = self.data_sampler.samples(request.sample_data.pcollection_ids)
return beam_fn_api_pb2.InstructionResponse(
instruction_id=request.instruction_id, sample_data=samples)
self._execute(lambda: get_samples(request), request)
def create_worker(self):
# type: () -> SdkWorker
return SdkWorker(
self._bundle_processor_cache, profiler_factory=self._profiler_factory)
class BundleProcessorCache(object):
"""A cache for ``BundleProcessor``s.
``BundleProcessor`` objects are cached by the id of their
``beam_fn_api_pb2.ProcessBundleDescriptor``.
Attributes:
fns (dict): A dictionary that maps bundle descriptor IDs to instances of
``beam_fn_api_pb2.ProcessBundleDescriptor``.
state_handler_factory (``StateHandlerFactory``): Used to create state
handlers to be used by a ``bundle_processor.BundleProcessor`` during
processing.
data_channel_factory (``data_plane.DataChannelFactory``)
active_bundle_processors (dict): A dictionary, indexed by instruction IDs,
containing ``bundle_processor.BundleProcessor`` objects that are currently
active processing the corresponding instruction.
cached_bundle_processors (dict): A dictionary, indexed by bundle processor
id, of cached ``bundle_processor.BundleProcessor`` that are not currently
performing processing.
"""
periodic_shutdown = None # type: Optional[PeriodicThread]
def __init__(
self,
runner_capabilities, # type: FrozenSet[str]
state_handler_factory, # type: StateHandlerFactory
data_channel_factory, # type: data_plane.DataChannelFactory
fns, # type: MutableMapping[str, beam_fn_api_pb2.ProcessBundleDescriptor]
data_sampler=None, # type: Optional[data_sampler.DataSampler]
):
# type: (...) -> None
self.runner_capabilities = runner_capabilities
self.fns = fns
self.state_handler_factory = state_handler_factory
self.data_channel_factory = data_channel_factory
self.known_not_running_instruction_ids = collections.OrderedDict(
) # type: collections.OrderedDict[str, bool]
self.failed_instruction_ids = collections.OrderedDict(
) # type: collections.OrderedDict[str, Exception]
self.active_bundle_processors = {
} # type: Dict[str, Tuple[str, bundle_processor.BundleProcessor]]
self.cached_bundle_processors = collections.defaultdict(
list) # type: DefaultDict[str, List[bundle_processor.BundleProcessor]]
self.last_access_times = collections.defaultdict(
float) # type: DefaultDict[str, float]
self._schedule_periodic_shutdown()
self._lock = threading.Lock()
self.data_sampler = data_sampler
def register(self, bundle_descriptor):
# type: (beam_fn_api_pb2.ProcessBundleDescriptor) -> None
"""Register a ``beam_fn_api_pb2.ProcessBundleDescriptor`` by its id."""
self.fns[bundle_descriptor.id] = bundle_descriptor
def activate(self, instruction_id):
# type: (str) -> None
"""Makes the ``instruction_id`` known to the bundle processor.
Allows ``lookup`` to return ``None``. Necessary if ``lookup`` can occur
before ``get``.
"""
with self._lock:
self.known_not_running_instruction_ids[instruction_id] = True
def get(self, instruction_id, bundle_descriptor_id):
# type: (str, str) -> bundle_processor.BundleProcessor
"""
Return the requested ``BundleProcessor``, creating it if necessary.
Moves the ``BundleProcessor`` from the inactive to the active cache.
"""
with self._lock:
try:
# pop() is threadsafe
processor = self.cached_bundle_processors[bundle_descriptor_id].pop()
self.active_bundle_processors[
instruction_id] = bundle_descriptor_id, processor
try:
del self.known_not_running_instruction_ids[instruction_id]
except KeyError:
# The instruction may have not been pre-registered before execution
# since activate() may have never been invoked
pass
return processor
except IndexError:
pass
# Make sure we instantiate the processor while not holding the lock.
# Reduce risks of concurrent modifications of the same protos
# captured in bundle descriptor when the same bundle descriptor is used
# in different instructions.
pbd = beam_fn_api_pb2.ProcessBundleDescriptor()
pbd.MergeFrom(self.fns[bundle_descriptor_id])
processor = bundle_processor.BundleProcessor(
self.runner_capabilities,
pbd,
self.state_handler_factory.create_state_handler(
pbd.state_api_service_descriptor),
self.data_channel_factory,
self.data_sampler)
with self._lock:
self.active_bundle_processors[
instruction_id] = bundle_descriptor_id, processor
try:
del self.known_not_running_instruction_ids[instruction_id]
except KeyError:
# The instruction may have not been pre-registered before execution
# since activate() may have never been invoked
pass
return processor
def lookup(self, instruction_id):
# type: (str) -> Optional[bundle_processor.BundleProcessor]
"""
Return the requested ``BundleProcessor`` from the cache.
Will return ``None`` if the BundleProcessor is known but not yet ready. Will
raise an error if the ``instruction_id`` is not known or has been discarded.
"""
with self._lock:
if instruction_id in self.failed_instruction_ids:
e = self.failed_instruction_ids[instruction_id]
raise RuntimeError(
'Bundle processing associated with %s has failed. '
'Check prior failing response and attached exception for details.' %
instruction_id) from e
processor = self.active_bundle_processors.get(
instruction_id, (None, None))[-1]
if processor:
return processor
if instruction_id in self.known_not_running_instruction_ids:
return None
raise RuntimeError('Unknown process bundle id %s.' % instruction_id)
def discard(self, instruction_id, exception):
# type: (str, Exception) -> None
"""
Marks the instruction id as failed shutting down the ``BundleProcessor``.
"""
with self._lock:
self.failed_instruction_ids[instruction_id] = exception
while len(self.failed_instruction_ids) > MAX_FAILED_INSTRUCTIONS:
self.failed_instruction_ids.popitem(last=False)
processor = self.active_bundle_processors[instruction_id][1]
del self.active_bundle_processors[instruction_id]
# Perform the shutdown while not holding the lock.
processor.shutdown()
def release(self, instruction_id):
# type: (str) -> None
"""
Release the requested ``BundleProcessor``.
Resets the ``BundleProcessor`` and moves it from the active to the
inactive cache.
"""
with self._lock:
self.known_not_running_instruction_ids[instruction_id] = True
while len(self.known_not_running_instruction_ids
) > MAX_KNOWN_NOT_RUNNING_INSTRUCTIONS:
self.known_not_running_instruction_ids.popitem(last=False)
descriptor_id, processor = (
self.active_bundle_processors.pop(instruction_id))
# Make sure that we reset the processor while not holding the lock.
processor.reset()
with self._lock:
self.last_access_times[descriptor_id] = time.time()
self.cached_bundle_processors[descriptor_id].append(processor)
def shutdown(self):
# type: () -> None
"""
Shutdown all ``BundleProcessor``s in the cache.
"""
if self.periodic_shutdown:
self.periodic_shutdown.cancel()
self.periodic_shutdown.join()
self.periodic_shutdown = None
for instruction_id in list(self.active_bundle_processors.keys()):
self.discard(instruction_id, RuntimeError('Shutdown invoked'))
for cached_bundle_processors in self.cached_bundle_processors.values():
BundleProcessorCache._shutdown_cached_bundle_processors(
cached_bundle_processors)
def _schedule_periodic_shutdown(self):
# type: () -> None
def shutdown_inactive_bundle_processors():
# type: () -> None
inactive_descriptor_ids = []
inactive_time = time.time(
) - DEFAULT_BUNDLE_PROCESSOR_CACHE_SHUTDOWN_THRESHOLD_S
with self._lock:
for descriptor_id, last_access_time in self.last_access_times.items():
if (inactive_time > last_access_time):
inactive_descriptor_ids.append(descriptor_id)
# Shutdown can be expensive, keep out of lock
for descriptor_id in inactive_descriptor_ids:
BundleProcessorCache._shutdown_cached_bundle_processors(
self.cached_bundle_processors[descriptor_id])
self.periodic_shutdown = PeriodicThread(
DEFAULT_BUNDLE_PROCESSOR_CACHE_SHUTDOWN_THRESHOLD_S,
shutdown_inactive_bundle_processors)
self.periodic_shutdown.daemon = True
self.periodic_shutdown.start()
@staticmethod
def _shutdown_cached_bundle_processors(cached_bundle_processors):
# type: (List[bundle_processor.BundleProcessor]) -> None
try:
while True:
# pop() is threadsafe
bundle_processor = cached_bundle_processors.pop()
bundle_processor.shutdown()
except IndexError:
pass
class SdkWorker(object):
def __init__(
self,
bundle_processor_cache, # type: BundleProcessorCache
profiler_factory=None, # type: Optional[Callable[..., Profile]]
):
# type: (...) -> None
self.bundle_processor_cache = bundle_processor_cache
self.profiler_factory = profiler_factory
def do_instruction(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> beam_fn_api_pb2.InstructionResponse
request_type = request.WhichOneof('request')
if request_type:
# E.g. if register is set, this will call self.register(request.register))
return getattr(self, request_type)(
getattr(request, request_type), request.instruction_id)
else:
raise NotImplementedError
def register(
self,
request, # type: beam_fn_api_pb2.RegisterRequest
instruction_id # type: str
):
# type: (...) -> beam_fn_api_pb2.InstructionResponse
"""Registers a set of ``beam_fn_api_pb2.ProcessBundleDescriptor``s.
This set of ``beam_fn_api_pb2.ProcessBundleDescriptor`` come as part of a
``beam_fn_api_pb2.RegisterRequest``, which the runner sends to the SDK
worker before starting processing to register stages.
"""
for process_bundle_descriptor in request.process_bundle_descriptor:
self.bundle_processor_cache.register(process_bundle_descriptor)
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id,
register=beam_fn_api_pb2.RegisterResponse())
def process_bundle(
self,
request, # type: beam_fn_api_pb2.ProcessBundleRequest
instruction_id # type: str
):
# type: (...) -> beam_fn_api_pb2.InstructionResponse
bundle_processor = self.bundle_processor_cache.get(
instruction_id, request.process_bundle_descriptor_id)
try:
with bundle_processor.state_handler.process_instruction_id(
instruction_id, request.cache_tokens):
with self.maybe_profile(instruction_id):
delayed_applications, requests_finalization = (
bundle_processor.process_bundle(instruction_id))
monitoring_infos = bundle_processor.monitoring_infos()
response = beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id,
process_bundle=beam_fn_api_pb2.ProcessBundleResponse(
residual_roots=delayed_applications,
monitoring_infos=monitoring_infos,
monitoring_data={
SHORT_ID_CACHE.get_short_id(info): info.payload
for info in monitoring_infos
},
requires_finalization=requests_finalization))
# Don't release here if finalize is needed.
if not requests_finalization:
self.bundle_processor_cache.release(instruction_id)
return response
except Exception as e: # pylint: disable=bare-except
# Don't re-use bundle processors on failure.
self.bundle_processor_cache.discard(instruction_id, e)
raise
def process_bundle_split(
self,
request, # type: beam_fn_api_pb2.ProcessBundleSplitRequest
instruction_id # type: str
):
# type: (...) -> beam_fn_api_pb2.InstructionResponse
try:
processor = self.bundle_processor_cache.lookup(request.instruction_id)
except RuntimeError:
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id, error=traceback.format_exc())
# Return an empty response if we aren't running. This can happen
# if the ProcessBundleRequest has not started or already finished.
process_bundle_split = (
processor.try_split(request)
if processor else beam_fn_api_pb2.ProcessBundleSplitResponse())
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id,
process_bundle_split=process_bundle_split)
def process_bundle_progress(
self,
request, # type: beam_fn_api_pb2.ProcessBundleProgressRequest
instruction_id # type: str
):
# type: (...) -> beam_fn_api_pb2.InstructionResponse
try:
processor = self.bundle_processor_cache.lookup(request.instruction_id)
except RuntimeError:
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id, error=traceback.format_exc())
if processor:
monitoring_infos = processor.monitoring_infos()
consuming_received_data = processor.consuming_received_data
else:
# Return an empty response if we aren't running. This can happen
# if the ProcessBundleRequest has not started or already finished.
monitoring_infos = []
consuming_received_data = False
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id,
process_bundle_progress=beam_fn_api_pb2.ProcessBundleProgressResponse(
monitoring_infos=monitoring_infos,
monitoring_data={
SHORT_ID_CACHE.get_short_id(info): info.payload
for info in monitoring_infos
},
consuming_received_data=consuming_received_data))
def finalize_bundle(
self,
request, # type: beam_fn_api_pb2.FinalizeBundleRequest
instruction_id # type: str
):
# type: (...) -> beam_fn_api_pb2.InstructionResponse
try:
processor = self.bundle_processor_cache.lookup(request.instruction_id)
except RuntimeError:
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id, error=traceback.format_exc())
if processor:
try:
finalize_response = processor.finalize_bundle()
self.bundle_processor_cache.release(request.instruction_id)
return beam_fn_api_pb2.InstructionResponse(
instruction_id=instruction_id, finalize_bundle=finalize_response)
except Exception as e:
self.bundle_processor_cache.discard(request.instruction_id, e)
raise
# We can reach this state if there was an erroneous request to finalize
# the bundle while it is being initialized or has already been finalized
# and released.
raise RuntimeError(
'Bundle is not in a finalizable state for %s' % instruction_id)
@contextlib.contextmanager
def maybe_profile(self, instruction_id):
# type: (str) -> Iterator[None]
if self.profiler_factory:
profiler = self.profiler_factory(instruction_id)
if profiler:
with profiler:
yield
else:
yield
else:
yield
class StateHandler(metaclass=abc.ABCMeta):
"""An abstract object representing a ``StateHandler``."""
@abc.abstractmethod
def get_raw(
self,
state_key, # type: beam_fn_api_pb2.StateKey
continuation_token=None # type: Optional[bytes]
):
# type: (...) -> Tuple[bytes, Optional[bytes]]
"""Gets the contents of state for the given state key.
State is associated to a state key, AND an instruction_id, which is set
when calling process_instruction_id.
Returns a tuple with the contents in state, and an optional continuation
token, which is used to page the API.
"""
raise NotImplementedError(type(self))
@abc.abstractmethod
def append_raw(
self,
state_key, # type: beam_fn_api_pb2.StateKey
data # type: bytes
):
# type: (...) -> _Future
"""Append the input data into the state key.
Returns a future that allows one to wait for the completion of the call.
State is associated to a state key, AND an instruction_id, which is set
when calling process_instruction_id.
"""
raise NotImplementedError(type(self))
@abc.abstractmethod
def clear(self, state_key):
# type: (beam_fn_api_pb2.StateKey) -> _Future
"""Clears the contents of a cell for the input state key.
Returns a future that allows one to wait for the completion of the call.
State is associated to a state key, AND an instruction_id, which is set
when calling process_instruction_id.
"""
raise NotImplementedError(type(self))
@abc.abstractmethod
@contextlib.contextmanager
def process_instruction_id(self, bundle_id):
# type: (str) -> Iterator[None]
"""Switch the context of the state handler to a specific instruction.
This must be called before performing any write or read operations on the
existing state.
"""
raise NotImplementedError(type(self))
@abc.abstractmethod
def done(self):
# type: () -> None
"""Mark the state handler as done, and potentially delete all context."""
raise NotImplementedError(type(self))
class StateHandlerFactory(metaclass=abc.ABCMeta):
"""An abstract factory for creating ``DataChannel``."""
@abc.abstractmethod
def create_state_handler(self, api_service_descriptor):
# type: (endpoints_pb2.ApiServiceDescriptor) -> CachingStateHandler
"""Returns a ``StateHandler`` from the given ApiServiceDescriptor."""
raise NotImplementedError(type(self))
@abc.abstractmethod
def close(self):
# type: () -> None
"""Close all channels that this factory owns."""
raise NotImplementedError(type(self))
class GrpcStateHandlerFactory(StateHandlerFactory):
"""A factory for ``GrpcStateHandler``.
Caches the created channels by ``state descriptor url``.
"""
def __init__(self, state_cache, credentials=None, worker_id=None):
# type: (StateCache, Optional[grpc.ChannelCredentials], Optional[str]) -> None
self._state_handler_cache = {} # type: Dict[str, CachingStateHandler]
self._lock = threading.Lock()
self._throwing_state_handler = ThrowingStateHandler()
self._credentials = credentials
self._state_cache = state_cache
self._worker_id = worker_id
def create_state_handler(self, api_service_descriptor):
# type: (endpoints_pb2.ApiServiceDescriptor) -> CachingStateHandler
if not api_service_descriptor:
return self._throwing_state_handler
url = api_service_descriptor.url
if url not in self._state_handler_cache:
with self._lock:
if url not in self._state_handler_cache:
# Options to have no limits (-1) on the size of the messages
# received or sent over the data plane. The actual buffer size is
# controlled in a layer above.
options = [('grpc.max_receive_message_length', -1),
('grpc.max_send_message_length', -1),
('grpc.service_config', _GRPC_SERVICE_CONFIG)]
if self._credentials is None:
_LOGGER.info('Creating insecure state channel for %s.', url)
grpc_channel = GRPCChannelFactory.insecure_channel(
url, options=options)
else:
_LOGGER.info('Creating secure state channel for %s.', url)
grpc_channel = GRPCChannelFactory.secure_channel(
url, self._credentials, options=options)
_LOGGER.info('State channel established.')
# Add workerId to the grpc channel
grpc_channel = grpc.intercept_channel(
grpc_channel, WorkerIdInterceptor(self._worker_id))
self._state_handler_cache[url] = GlobalCachingStateHandler(
self._state_cache,
GrpcStateHandler(
beam_fn_api_pb2_grpc.BeamFnStateStub(grpc_channel)))
return self._state_handler_cache[url]
def close(self):
# type: () -> None
_LOGGER.info('Closing all cached gRPC state handlers.')
for _, state_handler in self._state_handler_cache.items():
state_handler.done()
self._state_handler_cache.clear()
self._state_cache.invalidate_all()
class CachingStateHandler(metaclass=abc.ABCMeta):
@abc.abstractmethod
@contextlib.contextmanager
def process_instruction_id(self, bundle_id, cache_tokens):
# type: (str, Iterable[beam_fn_api_pb2.ProcessBundleRequest.CacheToken]) -> Iterator[None]
raise NotImplementedError(type(self))
@abc.abstractmethod
def blocking_get(
self,
state_key, # type: beam_fn_api_pb2.StateKey
coder, # type: coder_impl.CoderImpl
):
# type: (...) -> Iterable[Any]
raise NotImplementedError(type(self))
@abc.abstractmethod
def extend(
self,
state_key, # type: beam_fn_api_pb2.StateKey
coder, # type: coder_impl.CoderImpl
elements, # type: Iterable[Any]
):
# type: (...) -> _Future
raise NotImplementedError(type(self))
@abc.abstractmethod
def clear(self, state_key):
# type: (beam_fn_api_pb2.StateKey) -> _Future
raise NotImplementedError(type(self))
@abc.abstractmethod
def done(self):
# type: () -> None
raise NotImplementedError(type(self))
class ThrowingStateHandler(CachingStateHandler):
"""A caching state handler that errors on any requests."""
@contextlib.contextmanager
def process_instruction_id(self, bundle_id, cache_tokens):
# type: (str, Iterable[beam_fn_api_pb2.ProcessBundleRequest.CacheToken]) -> Iterator[None]
raise RuntimeError(
'Unable to handle state requests for ProcessBundleDescriptor '
'for bundle id %s.' % bundle_id)
def blocking_get(
self,
state_key, # type: beam_fn_api_pb2.StateKey
coder, # type: coder_impl.CoderImpl