-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
fn_api_runner.py
1963 lines (1685 loc) · 75.7 KB
/
fn_api_runner.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.
#
"""A PipelineRunner using the SDK harness.
"""
from __future__ import absolute_import
from __future__ import print_function
import collections
import contextlib
import copy
import itertools
import logging
import os
import queue
import subprocess
import sys
import threading
import time
import uuid
from builtins import object
from concurrent import futures
import grpc
import apache_beam as beam # pylint: disable=ungrouped-imports
from apache_beam import coders
from apache_beam.coders.coder_impl import create_InputStream
from apache_beam.coders.coder_impl import create_OutputStream
from apache_beam.metrics import metric
from apache_beam.metrics import monitoring_infos
from apache_beam.metrics.execution import MetricResult
from apache_beam.options import pipeline_options
from apache_beam.options.value_provider import RuntimeValueProvider
from apache_beam.portability import common_urns
from apache_beam.portability import python_urns
from apache_beam.portability.api import beam_artifact_api_pb2
from apache_beam.portability.api import beam_artifact_api_pb2_grpc
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 beam_provision_api_pb2
from apache_beam.portability.api import beam_provision_api_pb2_grpc
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.portability.api import endpoints_pb2
from apache_beam.runners import pipeline_context
from apache_beam.runners import runner
from apache_beam.runners.portability import artifact_service
from apache_beam.runners.portability import fn_api_runner_transforms
from apache_beam.runners.portability import portable_metrics
from apache_beam.runners.portability.fn_api_runner_transforms import create_buffer_id
from apache_beam.runners.portability.fn_api_runner_transforms import only_element
from apache_beam.runners.portability.fn_api_runner_transforms import split_buffer_id
from apache_beam.runners.portability.fn_api_runner_transforms import unique_name
from apache_beam.runners.worker import bundle_processor
from apache_beam.runners.worker import data_plane
from apache_beam.runners.worker import sdk_worker
from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
from apache_beam.runners.worker.sdk_worker import _Future
from apache_beam.runners.worker.statecache import StateCache
from apache_beam.transforms import environments
from apache_beam.transforms import trigger
from apache_beam.transforms.window import GlobalWindow
from apache_beam.transforms.window import GlobalWindows
from apache_beam.utils import profiler
from apache_beam.utils import proto_utils
from apache_beam.utils import windowed_value
# This module is experimental. No backwards-compatibility guarantees.
ENCODED_IMPULSE_VALUE = beam.coders.WindowedValueCoder(
beam.coders.BytesCoder(),
beam.coders.coders.GlobalWindowCoder()).get_impl().encode_nested(
beam.transforms.window.GlobalWindows.windowed_value(b''))
# State caching is enabled in the fn_api_runner for testing, except for one
# test which runs without state caching (FnApiRunnerTestWithDisabledCaching).
# The cache is disabled in production for other runners.
STATE_CACHE_SIZE = 100
class ControlConnection(object):
_uid_counter = 0
_lock = threading.Lock()
def __init__(self):
self._push_queue = queue.Queue()
self._input = None
self._futures_by_id = dict()
self._read_thread = threading.Thread(
name='beam_control_read', target=self._read)
self._state = BeamFnControlServicer.UNSTARTED_STATE
def _read(self):
for data in self._input:
self._futures_by_id.pop(data.instruction_id).set(data)
def push(self, req):
if req == BeamFnControlServicer._DONE_MARKER:
self._push_queue.put(req)
return None
if not req.instruction_id:
with ControlConnection._lock:
ControlConnection._uid_counter += 1
req.instruction_id = 'control_%s' % ControlConnection._uid_counter
future = ControlFuture(req.instruction_id)
self._futures_by_id[req.instruction_id] = future
self._push_queue.put(req)
return future
def get_req(self):
return self._push_queue.get()
def set_input(self, input):
with ControlConnection._lock:
if self._input:
raise RuntimeError('input is already set.')
self._input = input
self._read_thread.start()
self._state = BeamFnControlServicer.STARTED_STATE
def close(self):
with ControlConnection._lock:
if self._state == BeamFnControlServicer.STARTED_STATE:
self.push(BeamFnControlServicer._DONE_MARKER)
self._read_thread.join()
self._state = BeamFnControlServicer.DONE_STATE
class BeamFnControlServicer(beam_fn_api_pb2_grpc.BeamFnControlServicer):
"""Implementation of BeamFnControlServicer for clients."""
UNSTARTED_STATE = 'unstarted'
STARTED_STATE = 'started'
DONE_STATE = 'done'
_DONE_MARKER = object()
def __init__(self):
self._lock = threading.Lock()
self._uid_counter = 0
self._state = self.UNSTARTED_STATE
# following self._req_* variables are used for debugging purpose, data is
# added only when self._log_req is True.
self._req_sent = collections.defaultdict(int)
self._req_worker_mapping = {}
self._log_req = logging.getLogger().getEffectiveLevel() <= logging.DEBUG
self._connections_by_worker_id = collections.defaultdict(ControlConnection)
def get_conn_by_worker_id(self, worker_id):
with self._lock:
return self._connections_by_worker_id[worker_id]
def Control(self, iterator, context):
with self._lock:
if self._state == self.DONE_STATE:
return
else:
self._state = self.STARTED_STATE
worker_id = dict(context.invocation_metadata()).get('worker_id')
if not worker_id:
raise RuntimeError('All workers communicate through gRPC should have '
'worker_id. Received None.')
control_conn = self.get_conn_by_worker_id(worker_id)
control_conn.set_input(iterator)
while True:
to_push = control_conn.get_req()
if to_push is self._DONE_MARKER:
return
yield to_push
if self._log_req:
self._req_sent[to_push.instruction_id] += 1
def done(self):
self._state = self.DONE_STATE
logging.debug('Runner: Requests sent by runner: %s',
[(str(req), cnt) for req, cnt in self._req_sent.items()])
logging.debug('Runner: Requests multiplexing info: %s',
[(str(req), worker) for req, worker
in self._req_worker_mapping.items()])
class _ListBuffer(list):
"""Used to support parititioning of a list."""
def partition(self, n):
return [self[k::n] for k in range(n)]
class _GroupingBuffer(object):
"""Used to accumulate groupded (shuffled) results."""
def __init__(self, pre_grouped_coder, post_grouped_coder, windowing):
self._key_coder = pre_grouped_coder.key_coder()
self._pre_grouped_coder = pre_grouped_coder
self._post_grouped_coder = post_grouped_coder
self._table = collections.defaultdict(list)
self._windowing = windowing
self._grouped_output = None
def append(self, elements_data):
if self._grouped_output:
raise RuntimeError('Grouping table append after read.')
input_stream = create_InputStream(elements_data)
coder_impl = self._pre_grouped_coder.get_impl()
key_coder_impl = self._key_coder.get_impl()
# TODO(robertwb): We could optimize this even more by using a
# window-dropping coder for the data plane.
is_trivial_windowing = self._windowing.is_default()
while input_stream.size() > 0:
windowed_key_value = coder_impl.decode_from_stream(input_stream, True)
key, value = windowed_key_value.value
self._table[key_coder_impl.encode(key)].append(
value if is_trivial_windowing
else windowed_key_value.with_value(value))
def partition(self, n):
""" It is used to partition _GroupingBuffer to N parts. Once it is
partitioned, it would not be re-partitioned with diff N. Re-partition
is not supported now.
"""
if not self._grouped_output:
if self._windowing.is_default():
globally_window = GlobalWindows.windowed_value(
None,
timestamp=GlobalWindow().max_timestamp(),
pane_info=windowed_value.PaneInfo(
is_first=True,
is_last=True,
timing=windowed_value.PaneInfoTiming.ON_TIME,
index=0,
nonspeculative_index=0)).with_value
windowed_key_values = lambda key, values: [
globally_window((key, values))]
else:
# TODO(pabloem, BEAM-7514): Trigger driver needs access to the clock
# note that this only comes through if windowing is default - but what
# about having multiple firings on the global window.
# May need to revise.
trigger_driver = trigger.create_trigger_driver(self._windowing, True)
windowed_key_values = trigger_driver.process_entire_key
coder_impl = self._post_grouped_coder.get_impl()
key_coder_impl = self._key_coder.get_impl()
self._grouped_output = [[] for _ in range(n)]
output_stream_list = []
for _ in range(n):
output_stream_list.append(create_OutputStream())
for idx, (encoded_key, windowed_values) in enumerate(self._table.items()):
key = key_coder_impl.decode(encoded_key)
for wkvs in windowed_key_values(key, windowed_values):
coder_impl.encode_to_stream(wkvs, output_stream_list[idx % n], True)
for ix, output_stream in enumerate(output_stream_list):
self._grouped_output[ix] = [output_stream.get()]
self._table = None
return self._grouped_output
def __iter__(self):
""" Since partition() returns a list of lists, add this __iter__ to return
a list to simplify code when we need to iterate through ALL elements of
_GroupingBuffer.
"""
return itertools.chain(*self.partition(1))
class _WindowGroupingBuffer(object):
"""Used to partition windowed side inputs."""
def __init__(self, access_pattern, coder):
# Here's where we would use a different type of partitioning
# (e.g. also by key) for a different access pattern.
if access_pattern.urn == common_urns.side_inputs.ITERABLE.urn:
self._kv_extrator = lambda value: ('', value)
self._key_coder = coders.SingletonCoder('')
self._value_coder = coder.wrapped_value_coder
elif access_pattern.urn == common_urns.side_inputs.MULTIMAP.urn:
self._kv_extrator = lambda value: value
self._key_coder = coder.wrapped_value_coder.key_coder()
self._value_coder = (
coder.wrapped_value_coder.value_coder())
else:
raise ValueError(
"Unknown access pattern: '%s'" % access_pattern.urn)
self._windowed_value_coder = coder
self._window_coder = coder.window_coder
self._values_by_window = collections.defaultdict(list)
def append(self, elements_data):
input_stream = create_InputStream(elements_data)
while input_stream.size() > 0:
windowed_value = self._windowed_value_coder.get_impl(
).decode_from_stream(input_stream, True)
key, value = self._kv_extrator(windowed_value.value)
for window in windowed_value.windows:
self._values_by_window[key, window].append(value)
def encoded_items(self):
value_coder_impl = self._value_coder.get_impl()
key_coder_impl = self._key_coder.get_impl()
for (key, window), values in self._values_by_window.items():
encoded_window = self._window_coder.encode(window)
encoded_key = key_coder_impl.encode_nested(key)
output_stream = create_OutputStream()
for value in values:
value_coder_impl.encode_to_stream(value, output_stream, True)
yield encoded_key, encoded_window, output_stream.get()
class FnApiRunner(runner.PipelineRunner):
def __init__(
self,
default_environment=None,
bundle_repeat=0,
use_state_iterables=False,
provision_info=None,
progress_request_frequency=None):
"""Creates a new Fn API Runner.
Args:
default_environment: the default environment to use for UserFns.
bundle_repeat: replay every bundle this many extra times, for profiling
and debugging
use_state_iterables: Intentionally split gbk iterables over state API
(for testing)
provision_info: provisioning info to make available to workers, or None
progress_request_frequency: The frequency (in seconds) that the runner
waits before requesting progress from the SDK.
"""
super(FnApiRunner, self).__init__()
self._last_uid = -1
self._default_environment = (
default_environment
or environments.EmbeddedPythonEnvironment())
self._bundle_repeat = bundle_repeat
self._num_workers = 1
self._progress_frequency = progress_request_frequency
self._profiler_factory = None
self._use_state_iterables = use_state_iterables
self._provision_info = provision_info or ExtendedProvisionInfo(
beam_provision_api_pb2.ProvisionInfo(
job_id='unknown-job-id',
job_name='unknown-job-name',
retrieval_token='unused-retrieval-token'))
def _next_uid(self):
self._last_uid += 1
return str(self._last_uid)
def run_pipeline(self, pipeline, options):
RuntimeValueProvider.set_runtime_options({})
# Setup "beam_fn_api" experiment options if lacked.
experiments = (options.view_as(pipeline_options.DebugOptions).experiments
or [])
if not 'beam_fn_api' in experiments:
experiments.append('beam_fn_api')
options.view_as(pipeline_options.DebugOptions).experiments = experiments
# This is sometimes needed if type checking is disabled
# to enforce that the inputs (and outputs) of GroupByKey operations
# are known to be KVs.
from apache_beam.runners.dataflow.dataflow_runner import DataflowRunner
# TODO: Move group_by_key_input_visitor() to a non-dataflow specific file.
pipeline.visit(DataflowRunner.group_by_key_input_visitor())
self._bundle_repeat = self._bundle_repeat or options.view_as(
pipeline_options.DirectOptions).direct_runner_bundle_repeat
self._num_workers = options.view_as(
pipeline_options.DirectOptions).direct_num_workers or self._num_workers
self._profiler_factory = profiler.Profile.factory_from_options(
options.view_as(pipeline_options.ProfilingOptions))
if 'use_sdf_bounded_source' in experiments:
pipeline.replace_all(DataflowRunner._SDF_PTRANSFORM_OVERRIDES)
self._latest_run_result = self.run_via_runner_api(pipeline.to_runner_api(
default_environment=self._default_environment))
return self._latest_run_result
def run_via_runner_api(self, pipeline_proto):
stage_context, stages = self.create_stages(pipeline_proto)
# TODO(pabloem, BEAM-7514): Create a watermark manager (that has access to
# the teststream (if any), and all the stages).
return self.run_stages(stage_context, stages)
@contextlib.contextmanager
def maybe_profile(self):
if self._profiler_factory:
try:
profile_id = 'direct-' + subprocess.check_output(
['git', 'rev-parse', '--abbrev-ref', 'HEAD']
).decode(errors='ignore').strip()
except subprocess.CalledProcessError:
profile_id = 'direct-unknown'
profiler = self._profiler_factory(profile_id, time_prefix='')
else:
profiler = None
if profiler:
with profiler:
yield
if not self._bundle_repeat:
logging.warning(
'The --direct_runner_bundle_repeat option is not set; '
'a significant portion of the profile may be one-time overhead.')
path = profiler.profile_output
print('CPU Profile written to %s' % path)
try:
import gprof2dot # pylint: disable=unused-import
if not subprocess.call([
sys.executable, '-m', 'gprof2dot',
'-f', 'pstats', path, '-o', path + '.dot']):
if not subprocess.call(
['dot', '-Tsvg', '-o', path + '.svg', path + '.dot']):
print('CPU Profile rendering at file://%s.svg'
% os.path.abspath(path))
except ImportError:
# pylint: disable=superfluous-parens
print('Please install gprof2dot and dot for profile renderings.')
else:
# Empty context.
yield
def create_stages(self, pipeline_proto):
return fn_api_runner_transforms.create_and_optimize_stages(
copy.deepcopy(pipeline_proto),
phases=[fn_api_runner_transforms.annotate_downstream_side_inputs,
fn_api_runner_transforms.fix_side_input_pcoll_coders,
fn_api_runner_transforms.lift_combiners,
fn_api_runner_transforms.expand_sdf,
fn_api_runner_transforms.expand_gbk,
fn_api_runner_transforms.sink_flattens,
fn_api_runner_transforms.greedily_fuse,
fn_api_runner_transforms.read_to_impulse,
fn_api_runner_transforms.impulse_to_input,
fn_api_runner_transforms.inject_timer_pcollections,
fn_api_runner_transforms.sort_stages,
fn_api_runner_transforms.window_pcollection_coders],
known_runner_urns=frozenset([
common_urns.primitives.FLATTEN.urn,
common_urns.primitives.GROUP_BY_KEY.urn]),
use_state_iterables=self._use_state_iterables)
def run_stages(self, stage_context, stages):
"""Run a list of topologically-sorted stages in batch mode.
Args:
stage_context (fn_api_runner_transforms.TransformContext)
stages (list[fn_api_runner_transforms.Stage])
"""
worker_handler_manager = WorkerHandlerManager(
stage_context.components.environments, self._provision_info)
metrics_by_stage = {}
monitoring_infos_by_stage = {}
try:
with self.maybe_profile():
pcoll_buffers = collections.defaultdict(_ListBuffer)
for stage in stages:
stage_results = self._run_stage(
worker_handler_manager.get_worker_handlers,
stage_context.components,
stage,
pcoll_buffers,
stage_context.safe_coders)
metrics_by_stage[stage.name] = stage_results.process_bundle.metrics
monitoring_infos_by_stage[stage.name] = (
stage_results.process_bundle.monitoring_infos)
finally:
worker_handler_manager.close_all()
return RunnerResult(
runner.PipelineState.DONE, monitoring_infos_by_stage, metrics_by_stage)
def _store_side_inputs_in_state(self,
worker_handler,
context,
pipeline_components,
data_side_input,
pcoll_buffers,
safe_coders):
for (transform_id, tag), (buffer_id, si) in data_side_input.items():
_, pcoll_id = split_buffer_id(buffer_id)
value_coder = context.coders[safe_coders[
pipeline_components.pcollections[pcoll_id].coder_id]]
elements_by_window = _WindowGroupingBuffer(si, value_coder)
for element_data in pcoll_buffers[buffer_id]:
elements_by_window.append(element_data)
for key, window, elements_data in elements_by_window.encoded_items():
state_key = beam_fn_api_pb2.StateKey(
multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput(
transform_id=transform_id,
side_input_id=tag,
window=window,
key=key))
worker_handler.state.append_raw(state_key, elements_data)
def _run_bundle_multiple_times_for_testing(
self, worker_handler_list, process_bundle_descriptor, data_input,
data_output, get_input_coder_callable, cache_token_generator):
# all workers share state, so use any worker_handler.
worker_handler = worker_handler_list[0]
for k in range(self._bundle_repeat):
try:
worker_handler.state.checkpoint()
testing_bundle_manager = ParallelBundleManager(
worker_handler_list, lambda pcoll_id: [],
get_input_coder_callable, process_bundle_descriptor,
self._progress_frequency, k,
num_workers=self._num_workers,
cache_token_generator=cache_token_generator
)
testing_bundle_manager.process_bundle(data_input, data_output)
finally:
worker_handler.state.restore()
def _collect_written_timers_and_add_to_deferred_inputs(self,
context,
pipeline_components,
stage,
get_buffer_callable,
deferred_inputs):
for transform_id, timer_writes in stage.timer_pcollections:
# Queue any set timers as new inputs.
windowed_timer_coder_impl = context.coders[
pipeline_components.pcollections[timer_writes].coder_id].get_impl()
written_timers = get_buffer_callable(
create_buffer_id(timer_writes, kind='timers'))
if written_timers:
# Keep only the "last" timer set per key and window.
timers_by_key_and_window = {}
for elements_data in written_timers:
input_stream = create_InputStream(elements_data)
while input_stream.size() > 0:
windowed_key_timer = windowed_timer_coder_impl.decode_from_stream(
input_stream, True)
key, _ = windowed_key_timer.value
# TODO: Explode and merge windows.
assert len(windowed_key_timer.windows) == 1
timers_by_key_and_window[
key, windowed_key_timer.windows[0]] = windowed_key_timer
out = create_OutputStream()
for windowed_key_timer in timers_by_key_and_window.values():
windowed_timer_coder_impl.encode_to_stream(
windowed_key_timer, out, True)
deferred_inputs[transform_id] = _ListBuffer([out.get()])
written_timers[:] = []
def _add_residuals_and_channel_splits_to_deferred_inputs(
self, splits, get_input_coder_callable,
input_for_callable, last_sent, deferred_inputs):
prev_stops = {}
for split in splits:
for delayed_application in split.residual_roots:
deferred_inputs[
input_for_callable(
delayed_application.application.transform_id,
delayed_application.application.input_id)
].append(delayed_application.application.element)
for channel_split in split.channel_splits:
coder_impl = get_input_coder_callable(channel_split.transform_id)
# TODO(SDF): This requires determanistic ordering of buffer iteration.
# TODO(SDF): The return split is in terms of indices. Ideally,
# a runner could map these back to actual positions to effectively
# describe the two "halves" of the now-split range. Even if we have
# to buffer each element we send (or at the very least a bit of
# metadata, like position, about each of them) this should be doable
# if they're already in memory and we are bounding the buffer size
# (e.g. to 10mb plus whatever is eagerly read from the SDK). In the
# case of non-split-points, we can either immediately replay the
# "non-split-position" elements or record them as we do the other
# delayed applications.
# Decode and recode to split the encoded buffer by element index.
all_elements = list(coder_impl.decode_all(b''.join(last_sent[
channel_split.transform_id])))
residual_elements = all_elements[
channel_split.first_residual_element : prev_stops.get(
channel_split.transform_id, len(all_elements)) + 1]
if residual_elements:
deferred_inputs[channel_split.transform_id].append(
coder_impl.encode_all(residual_elements))
prev_stops[
channel_split.transform_id] = channel_split.last_primary_element
@staticmethod
def _extract_stage_data_endpoints(
stage, pipeline_components, data_api_service_descriptor, pcoll_buffers):
# Returns maps of transform names to PCollection identifiers.
# Also mutates IO stages to point to the data ApiServiceDescriptor.
data_input = {}
data_side_input = {}
data_output = {}
for transform in stage.transforms:
if transform.spec.urn in (bundle_processor.DATA_INPUT_URN,
bundle_processor.DATA_OUTPUT_URN):
pcoll_id = transform.spec.payload
if transform.spec.urn == bundle_processor.DATA_INPUT_URN:
target = transform.unique_name, only_element(transform.outputs)
if pcoll_id == fn_api_runner_transforms.IMPULSE_BUFFER:
data_input[target] = _ListBuffer([ENCODED_IMPULSE_VALUE])
else:
data_input[target] = pcoll_buffers[pcoll_id]
coder_id = pipeline_components.pcollections[
only_element(transform.outputs.values())].coder_id
elif transform.spec.urn == bundle_processor.DATA_OUTPUT_URN:
target = transform.unique_name, only_element(transform.inputs)
data_output[target] = pcoll_id
coder_id = pipeline_components.pcollections[
only_element(transform.inputs.values())].coder_id
else:
raise NotImplementedError
data_spec = beam_fn_api_pb2.RemoteGrpcPort(coder_id=coder_id)
if data_api_service_descriptor:
data_spec.api_service_descriptor.url = (
data_api_service_descriptor.url)
transform.spec.payload = data_spec.SerializeToString()
elif transform.spec.urn in fn_api_runner_transforms.PAR_DO_URNS:
payload = proto_utils.parse_Bytes(
transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
for tag, si in payload.side_inputs.items():
data_side_input[transform.unique_name, tag] = (
create_buffer_id(transform.inputs[tag]), si.access_pattern)
return data_input, data_side_input, data_output
def _run_stage(self,
worker_handler_factory,
pipeline_components,
stage,
pcoll_buffers,
safe_coders):
"""Run an individual stage.
Args:
worker_handler_factory: A ``callable`` that takes in an environment, and
returns a ``WorkerHandler`` class.
pipeline_components (beam_runner_api_pb2.Components): TODO
stage (fn_api_runner_transforms.Stage)
pcoll_buffers (collections.defaultdict of str: list): Mapping of
PCollection IDs to list that functions as buffer for the
``beam.PCollection``.
safe_coders (dict): TODO
"""
def iterable_state_write(values, element_coder_impl):
token = unique_name(None, 'iter').encode('ascii')
out = create_OutputStream()
for element in values:
element_coder_impl.encode_to_stream(element, out, True)
worker_handler.state.append_raw(
beam_fn_api_pb2.StateKey(
runner=beam_fn_api_pb2.StateKey.Runner(key=token)),
out.get())
return token
worker_handler_list = worker_handler_factory(
stage.environment, self._num_workers)
# All worker_handlers share the same grpc server, so we can read grpc server
# info from any worker_handler and read from the first worker_handler.
worker_handler = next(iter(worker_handler_list))
context = pipeline_context.PipelineContext(
pipeline_components, iterable_state_write=iterable_state_write)
data_api_service_descriptor = worker_handler.data_api_service_descriptor()
logging.info('Running %s', stage.name)
data_input, data_side_input, data_output = self._extract_endpoints(
stage, pipeline_components, data_api_service_descriptor, pcoll_buffers)
process_bundle_descriptor = beam_fn_api_pb2.ProcessBundleDescriptor(
id=self._next_uid(),
transforms={transform.unique_name: transform
for transform in stage.transforms},
pcollections=dict(pipeline_components.pcollections.items()),
coders=dict(pipeline_components.coders.items()),
windowing_strategies=dict(
pipeline_components.windowing_strategies.items()),
environments=dict(pipeline_components.environments.items()))
if worker_handler.state_api_service_descriptor():
process_bundle_descriptor.state_api_service_descriptor.url = (
worker_handler.state_api_service_descriptor().url)
# Store the required side inputs into state so it is accessible for the
# worker when it runs this bundle.
self._store_side_inputs_in_state(worker_handler,
context,
pipeline_components,
data_side_input,
pcoll_buffers,
safe_coders)
def get_buffer(buffer_id):
"""Returns the buffer for a given (operation_type, PCollection ID).
For grouping-typed operations, we produce a ``_GroupingBuffer``. For
others, we produce a ``_ListBuffer``.
"""
kind, name = split_buffer_id(buffer_id)
if kind in ('materialize', 'timers'):
# If `buffer_id` is not a key in `pcoll_buffers`, it will be added by
# the `defaultdict`.
return pcoll_buffers[buffer_id]
elif kind == 'group':
# This is a grouping write, create a grouping buffer if needed.
if buffer_id not in pcoll_buffers:
original_gbk_transform = name
transform_proto = pipeline_components.transforms[
original_gbk_transform]
input_pcoll = only_element(list(transform_proto.inputs.values()))
output_pcoll = only_element(list(transform_proto.outputs.values()))
pre_gbk_coder = context.coders[safe_coders[
pipeline_components.pcollections[input_pcoll].coder_id]]
post_gbk_coder = context.coders[safe_coders[
pipeline_components.pcollections[output_pcoll].coder_id]]
windowing_strategy = context.windowing_strategies[
pipeline_components
.pcollections[output_pcoll].windowing_strategy_id]
pcoll_buffers[buffer_id] = _GroupingBuffer(
pre_gbk_coder, post_gbk_coder, windowing_strategy)
else:
# These should be the only two identifiers we produce for now,
# but special side input writes may go here.
raise NotImplementedError(buffer_id)
return pcoll_buffers[buffer_id]
def get_input_coder_impl(transform_id):
return context.coders[safe_coders[
beam_fn_api_pb2.RemoteGrpcPort.FromString(
process_bundle_descriptor.transforms[transform_id].spec.payload
).coder_id
]].get_impl()
# Change cache token across bundle repeats
cache_token_generator = FnApiRunner.get_cache_token_generator(static=False)
self._run_bundle_multiple_times_for_testing(
worker_handler_list, process_bundle_descriptor, data_input, data_output,
get_input_coder_impl, cache_token_generator=cache_token_generator)
bundle_manager = ParallelBundleManager(
worker_handler_list, get_buffer, get_input_coder_impl,
process_bundle_descriptor, self._progress_frequency,
num_workers=self._num_workers,
cache_token_generator=cache_token_generator)
result, splits = bundle_manager.process_bundle(data_input, data_output)
def input_for(transform_id, input_id):
input_pcoll = process_bundle_descriptor.transforms[
transform_id].inputs[input_id]
for read_id, proto in process_bundle_descriptor.transforms.items():
if (proto.spec.urn == bundle_processor.DATA_INPUT_URN
and input_pcoll in proto.outputs.values()):
return read_id
raise RuntimeError(
'No IO transform feeds %s' % transform_id)
last_result = result
last_sent = data_input
while True:
deferred_inputs = collections.defaultdict(_ListBuffer)
self._collect_written_timers_and_add_to_deferred_inputs(
context, pipeline_components, stage, get_buffer, deferred_inputs)
# Queue any process-initiated delayed bundle applications.
for delayed_application in last_result.process_bundle.residual_roots:
deferred_inputs[
input_for(
delayed_application.application.transform_id,
delayed_application.application.input_id)
].append(delayed_application.application.element)
# Queue any runner-initiated delayed bundle applications.
self._add_residuals_and_channel_splits_to_deferred_inputs(
splits, get_input_coder_impl, input_for, last_sent, deferred_inputs)
if deferred_inputs:
# The worker will be waiting on these inputs as well.
for other_input in data_input:
if other_input not in deferred_inputs:
deferred_inputs[other_input] = _ListBuffer([])
# TODO(robertwb): merge results
# We cannot split deferred_input until we include residual_roots to
# merged results. Without residual_roots, pipeline stops earlier and we
# may miss some data.
bundle_manager._num_workers = 1
bundle_manager._skip_registration = True
last_result, splits = bundle_manager.process_bundle(
deferred_inputs, data_output)
last_sent = deferred_inputs
result = beam_fn_api_pb2.InstructionResponse(
process_bundle=beam_fn_api_pb2.ProcessBundleResponse(
monitoring_infos=monitoring_infos.consolidate(
itertools.chain(
result.process_bundle.monitoring_infos,
last_result.process_bundle.monitoring_infos))),
error=result.error or last_result.error)
else:
break
return result
@staticmethod
def _extract_endpoints(stage,
pipeline_components,
data_api_service_descriptor,
pcoll_buffers):
"""Returns maps of transform names to PCollection identifiers.
Also mutates IO stages to point to the data ApiServiceDescriptor.
Args:
stage (fn_api_runner_transforms.Stage): The stage to extract endpoints
for.
pipeline_components (beam_runner_api_pb2.Components): Components of the
pipeline to include coders, transforms, PCollections, etc.
data_api_service_descriptor: A GRPC endpoint descriptor for data plane.
pcoll_buffers (dict): A dictionary containing buffers for PCollection
elements.
Returns:
A tuple of (data_input, data_side_input, data_output) dictionaries.
`data_input` is a dictionary mapping (transform_name, output_name) to a
PCollection buffer; `data_output` is a dictionary mapping
(transform_name, output_name) to a PCollection ID.
"""
data_input = {}
data_side_input = {}
data_output = {}
for transform in stage.transforms:
if transform.spec.urn in (bundle_processor.DATA_INPUT_URN,
bundle_processor.DATA_OUTPUT_URN):
pcoll_id = transform.spec.payload
if transform.spec.urn == bundle_processor.DATA_INPUT_URN:
if pcoll_id == fn_api_runner_transforms.IMPULSE_BUFFER:
data_input[transform.unique_name] = _ListBuffer(
[ENCODED_IMPULSE_VALUE])
else:
data_input[transform.unique_name] = pcoll_buffers[pcoll_id]
coder_id = pipeline_components.pcollections[
only_element(transform.outputs.values())].coder_id
elif transform.spec.urn == bundle_processor.DATA_OUTPUT_URN:
data_output[transform.unique_name] = pcoll_id
coder_id = pipeline_components.pcollections[
only_element(transform.inputs.values())].coder_id
else:
raise NotImplementedError
data_spec = beam_fn_api_pb2.RemoteGrpcPort(coder_id=coder_id)
if data_api_service_descriptor:
data_spec.api_service_descriptor.url = (
data_api_service_descriptor.url)
transform.spec.payload = data_spec.SerializeToString()
elif transform.spec.urn in fn_api_runner_transforms.PAR_DO_URNS:
payload = proto_utils.parse_Bytes(
transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
for tag, si in payload.side_inputs.items():
data_side_input[transform.unique_name, tag] = (
create_buffer_id(transform.inputs[tag]), si.access_pattern)
return data_input, data_side_input, data_output
# These classes are used to interact with the worker.
class StateServicer(beam_fn_api_pb2_grpc.BeamFnStateServicer):
class CopyOnWriteState(object):
def __init__(self, underlying):
self._underlying = underlying
self._overlay = {}
def __getitem__(self, key):
if key in self._overlay:
return self._overlay[key]
else:
return FnApiRunner.StateServicer.CopyOnWriteList(
self._underlying, self._overlay, key)
def __delitem__(self, key):
self._overlay[key] = []
def commit(self):
self._underlying.update(self._overlay)
return self._underlying
class CopyOnWriteList(object):
def __init__(self, underlying, overlay, key):
self._underlying = underlying
self._overlay = overlay
self._key = key
def __iter__(self):
if self._key in self._overlay:
return iter(self._overlay[self._key])
else:
return iter(self._underlying[self._key])
def append(self, item):
if self._key not in self._overlay:
self._overlay[self._key] = list(self._underlying[self._key])
self._overlay[self._key].append(item)
def __init__(self):
self._lock = threading.Lock()
self._state = collections.defaultdict(list)
self._checkpoint = None
self._use_continuation_tokens = False
self._continuations = {}
def checkpoint(self):
assert self._checkpoint is None
self._checkpoint = self._state
self._state = FnApiRunner.StateServicer.CopyOnWriteState(self._state)
def commit(self):
self._state.commit()
self._state = self._checkpoint.commit()
self._checkpoint = None
def restore(self):
self._state = self._checkpoint
self._checkpoint = None
@contextlib.contextmanager
def process_instruction_id(self, unused_instruction_id):
yield
def get_raw(self, state_key, continuation_token=None):
with self._lock:
full_state = self._state[self._to_key(state_key)]
if self._use_continuation_tokens:
# The token is "nonce:index".
if not continuation_token:
token_base = 'token_%x' % len(self._continuations)
self._continuations[token_base] = tuple(full_state)
return b'', '%s:0' % token_base
else:
token_base, index = continuation_token.split(':')
ix = int(index)
full_state = self._continuations[token_base]
if ix == len(full_state):
return b'', None
else:
return full_state[ix], '%s:%d' % (token_base, ix + 1)
else:
assert not continuation_token
return b''.join(full_state), None
def append_raw(self, state_key, data):
with self._lock:
self._state[self._to_key(state_key)].append(data)
return _Future.done()
def clear(self, state_key):
with self._lock:
try:
del self._state[self._to_key(state_key)]
except KeyError:
# This may happen with the caching layer across bundles. Caching may
# skip this storage layer for a blocking_get(key) request. Without
# the caching, the state for a key would be initialized via the
# defaultdict that _state uses.
pass
return _Future.done()
@staticmethod
def _to_key(state_key):
return state_key.SerializeToString()
class GrpcStateServicer(beam_fn_api_pb2_grpc.BeamFnStateServicer):
def __init__(self, state):
self._state = state
def State(self, request_stream, context=None):
# Note that this eagerly mutates state, assuming any failures are fatal.
# Thus it is safe to ignore instruction_id.
for request in request_stream:
request_type = request.WhichOneof('request')
if request_type == 'get':
data, continuation_token = self._state.get_raw(
request.state_key, request.get.continuation_token)
yield beam_fn_api_pb2.StateResponse(
id=request.id,
get=beam_fn_api_pb2.StateGetResponse(
data=data, continuation_token=continuation_token))