-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
fn_runner.py
1550 lines (1378 loc) · 64.1 KB
/
fn_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.
"""
# pytype: skip-file
# mypy: check-untyped-defs
import contextlib
import copy
import itertools
import logging
import multiprocessing
import os
import subprocess
import sys
import threading
import time
from typing import TYPE_CHECKING
from typing import Callable
from typing import Dict
from typing import Iterable
from typing import Iterator
from typing import List
from typing import Mapping
from typing import MutableMapping
from typing import Optional
from typing import Set
from typing import Tuple
from typing import TypeVar
from typing import Union
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.metrics.execution import MetricsContainer
from apache_beam.metrics.metricbase import MetricName
from apache_beam.metrics.monitoring_infos import consolidate as consolidate_monitoring_infos
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.api import beam_fn_api_pb2
from apache_beam.portability.api import beam_provision_api_pb2
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.runners import runner
from apache_beam.runners.common import group_by_key_input_visitor
from apache_beam.runners.portability import portable_metrics
from apache_beam.runners.portability.fn_api_runner import execution
from apache_beam.runners.portability.fn_api_runner import translations
from apache_beam.runners.portability.fn_api_runner.execution import ListBuffer
from apache_beam.runners.portability.fn_api_runner.translations import BundleProcessResult
from apache_beam.runners.portability.fn_api_runner.translations import DataInput
from apache_beam.runners.portability.fn_api_runner.translations import DataOutput
from apache_beam.runners.portability.fn_api_runner.translations import OutputTimerData
from apache_beam.runners.portability.fn_api_runner.translations import OutputTimers
from apache_beam.runners.portability.fn_api_runner.translations import create_buffer_id
from apache_beam.runners.portability.fn_api_runner.translations import only_element
from apache_beam.runners.portability.fn_api_runner.worker_handlers import WorkerHandlerManager
from apache_beam.runners.worker import bundle_processor
from apache_beam.transforms import environments
from apache_beam.utils import proto_utils
from apache_beam.utils import thread_pool_executor
from apache_beam.utils import timestamp
from apache_beam.utils.profiler import Profile
if TYPE_CHECKING:
from apache_beam.pipeline import Pipeline
from apache_beam.portability.api import metrics_pb2
from apache_beam.runners.portability.fn_api_runner.worker_handlers import WorkerHandler
_LOGGER = logging.getLogger(__name__)
_BUNDLE_LOGGER = logging.getLogger(__name__ + ".run_bundle")
T = TypeVar('T')
# This module is experimental. No backwards-compatibility guarantees.
class FnApiRunner(runner.PipelineRunner):
NUM_FUSED_STAGES_COUNTER = "__num_fused_stages"
def __init__(
self,
default_environment=None, # type: Optional[environments.Environment]
bundle_repeat=0, # type: int
use_state_iterables=False, # type: bool
provision_info=None, # type: Optional[ExtendedProvisionInfo]
progress_request_frequency=None, # type: Optional[float]
is_drain=False # type: bool
):
# type: (...) -> 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.
is_drain: identify whether expand the sdf graph in the drain mode.
"""
super().__init__()
self._default_environment = (
default_environment or environments.EmbeddedPythonEnvironment.default())
self._bundle_repeat = bundle_repeat
self._num_workers = 1
self._progress_frequency = progress_request_frequency
self._profiler_factory: Optional[Callable[..., Profile]] = None
self._use_state_iterables = use_state_iterables
self._is_drain = is_drain
self._provision_info = provision_info or ExtendedProvisionInfo(
beam_provision_api_pb2.ProvisionInfo(
retrieval_token='unused-retrieval-token'))
@staticmethod
def supported_requirements():
# type: () -> Tuple[str, ...]
return (
common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn,
common_urns.requirements.REQUIRES_BUNDLE_FINALIZATION.urn,
common_urns.requirements.REQUIRES_SPLITTABLE_DOFN.urn,
)
def run_pipeline(self,
pipeline, # type: Pipeline
options # type: pipeline_options.PipelineOptions
):
# type: (...) -> RunnerResult
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.
pipeline.visit(
group_by_key_input_visitor(
not options.view_as(pipeline_options.TypeOptions).
allow_non_deterministic_key_coders))
self._bundle_repeat = self._bundle_repeat or options.view_as(
pipeline_options.DirectOptions).direct_runner_bundle_repeat
pipeline_direct_num_workers = options.view_as(
pipeline_options.DirectOptions).direct_num_workers
if pipeline_direct_num_workers == 0:
self._num_workers = multiprocessing.cpu_count()
else:
self._num_workers = pipeline_direct_num_workers or self._num_workers
# set direct workers running mode if it is defined with pipeline options.
running_mode = \
options.view_as(pipeline_options.DirectOptions).direct_running_mode
if running_mode == 'multi_threading':
self._default_environment = (
environments.EmbeddedPythonGrpcEnvironment.default())
elif running_mode == 'multi_processing':
command_string = '%s -m apache_beam.runners.worker.sdk_worker_main' \
% sys.executable
self._default_environment = (
environments.SubprocessSDKEnvironment.from_command_string(
command_string=command_string))
if running_mode == 'in_memory' and self._num_workers != 1:
_LOGGER.warning(
'If direct_num_workers is not equal to 1, direct_running_mode '
'should be `multi_processing` or `multi_threading` instead of '
'`in_memory` in order for it to have the desired worker parallelism '
'effect. direct_num_workers: %d ; running_mode: %s',
self._num_workers,
running_mode)
self._profiler_factory = Profile.factory_from_options(
options.view_as(pipeline_options.ProfilingOptions))
self._latest_run_result = self.run_via_runner_api(
pipeline.to_runner_api(default_environment=self._default_environment),
options)
return self._latest_run_result
def run_via_runner_api(self, pipeline_proto, options):
# type: (beam_runner_api_pb2.Pipeline, pipeline_options.PipelineOptions) -> RunnerResult
self._validate_requirements(pipeline_proto)
self._check_requirements(pipeline_proto)
if options.view_as(
pipeline_options.DirectOptions).direct_embed_docker_python:
pipeline_proto = self.embed_default_docker_image(pipeline_proto)
stage_context, stages = self.create_stages(pipeline_proto)
return self.run_stages(stage_context, stages)
def embed_default_docker_image(self, pipeline_proto):
# Context is unused for these types.
embedded_env = environments.EmbeddedPythonEnvironment.default(
).to_runner_api(None) # type: ignore[arg-type]
docker_env = environments.DockerEnvironment.from_container_image(
environments.DockerEnvironment.default_docker_image()).to_runner_api(
None) # type: ignore[arg-type]
for env_id, env in pipeline_proto.components.environments.items():
if env == docker_env:
docker_env_id = env_id
break
else:
# No matching docker environments.
return pipeline_proto
for env_id, env in pipeline_proto.components.environments.items():
if env.urn == embedded_env.urn:
embedded_env_id = env_id
break
else:
# No existing embedded environment.
pipeline_proto.components.environments[docker_env_id].CopyFrom(
embedded_env)
return pipeline_proto
for transform in pipeline_proto.components.transforms.values():
if transform.environment_id == docker_env_id:
transform.environment_id = embedded_env_id
return pipeline_proto
@contextlib.contextmanager
def maybe_profile(self):
# type: () -> Iterator[None]
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='') # type: Optional[Profile]
else:
profiler = None
if profiler:
with profiler:
yield
if not self._bundle_repeat:
_LOGGER.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 _validate_requirements(self, pipeline_proto):
# type: (beam_runner_api_pb2.Pipeline) -> None
"""As a test runner, validate requirements were set correctly."""
expected_requirements = set()
def add_requirements(transform_id):
# type: (str) -> None
transform = pipeline_proto.components.transforms[transform_id]
if transform.spec.urn in translations.PAR_DO_URNS:
payload = proto_utils.parse_Bytes(
transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
if payload.requests_finalization:
expected_requirements.add(
common_urns.requirements.REQUIRES_BUNDLE_FINALIZATION.urn)
if (payload.state_specs or payload.timer_family_specs):
expected_requirements.add(
common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn)
if payload.requires_stable_input:
expected_requirements.add(
common_urns.requirements.REQUIRES_STABLE_INPUT.urn)
if payload.requires_time_sorted_input:
expected_requirements.add(
common_urns.requirements.REQUIRES_TIME_SORTED_INPUT.urn)
if payload.restriction_coder_id:
expected_requirements.add(
common_urns.requirements.REQUIRES_SPLITTABLE_DOFN.urn)
else:
for sub in transform.subtransforms:
add_requirements(sub)
for root in pipeline_proto.root_transform_ids:
add_requirements(root)
if not expected_requirements.issubset(pipeline_proto.requirements):
raise ValueError(
'Missing requirement declaration: %s' %
(expected_requirements - set(pipeline_proto.requirements)))
def _check_requirements(self, pipeline_proto):
# type: (beam_runner_api_pb2.Pipeline) -> None
"""Check that this runner can satisfy all pipeline requirements."""
supported_requirements = set(self.supported_requirements())
for requirement in pipeline_proto.requirements:
if requirement not in supported_requirements:
raise ValueError(
'Unable to run pipeline with requirement: %s' % requirement)
for transform in pipeline_proto.components.transforms.values():
if transform.spec.urn == common_urns.primitives.TEST_STREAM.urn:
raise NotImplementedError(transform.spec.urn)
elif transform.spec.urn in translations.PAR_DO_URNS:
payload = proto_utils.parse_Bytes(
transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
for timer in payload.timer_family_specs.values():
if timer.time_domain not in (
beam_runner_api_pb2.TimeDomain.EVENT_TIME,
beam_runner_api_pb2.TimeDomain.PROCESSING_TIME):
raise NotImplementedError(timer.time_domain)
def create_stages(
self,
pipeline_proto # type: beam_runner_api_pb2.Pipeline
):
# type: (...) -> Tuple[translations.TransformContext, List[translations.Stage]]
return translations.create_and_optimize_stages(
copy.deepcopy(pipeline_proto),
phases=[
translations.annotate_downstream_side_inputs,
translations.fix_side_input_pcoll_coders,
translations.pack_combiners,
translations.lift_combiners,
translations.expand_sdf,
translations.expand_gbk,
translations.sink_flattens,
translations.greedily_fuse,
translations.read_to_impulse,
translations.impulse_to_input,
translations.sort_stages,
translations.add_impulse_to_dangling_transforms,
translations.setup_timer_mapping,
translations.populate_data_channel_coders,
],
known_runner_urns=frozenset([
common_urns.primitives.FLATTEN.urn,
common_urns.primitives.GROUP_BY_KEY.urn,
]),
use_state_iterables=self._use_state_iterables,
is_drain=self._is_drain)
def run_stages(self,
stage_context, # type: translations.TransformContext
stages # type: List[translations.Stage]
):
# type: (...) -> RunnerResult
"""Run a list of topologically-sorted stages in batch mode.
Args:
stage_context (translations.TransformContext)
stages (list[fn_api_runner.translations.Stage])
"""
worker_handler_manager = WorkerHandlerManager(
stage_context.components.environments, self._provision_info)
pipeline_metrics = MetricsContainer('')
pipeline_metrics.get_counter(
MetricName(
str(type(self)),
self.NUM_FUSED_STAGES_COUNTER,
urn='internal:' + self.NUM_FUSED_STAGES_COUNTER)).update(
len(stages))
monitoring_infos_by_stage: MutableMapping[
str, Iterable['metrics_pb2.MonitoringInfo']] = {}
runner_execution_context = execution.FnApiRunnerExecutionContext(
stages,
worker_handler_manager,
stage_context.components,
stage_context.safe_coders,
stage_context.data_channel_coders,
self._num_workers)
try:
with self.maybe_profile():
# Initialize Runner context:
# - Pipeline dictionaries, initial inputs and pipeline triggers
# - Replace Data API endpoints in protobufs.
runner_execution_context.setup()
bundle_counter = 0
# Start executing all ready bundles.
while len(runner_execution_context.queues.ready_inputs) > 0:
_LOGGER.debug(
"Remaining ready bundles: %s\n"
"\tWatermark pending bundles: %s\n"
"\tTime pending bundles: %s",
len(runner_execution_context.queues.ready_inputs),
len(runner_execution_context.queues.watermark_pending_inputs),
len(runner_execution_context.queues.time_pending_inputs))
consuming_stage_name, bundle_input = (
runner_execution_context.queues.ready_inputs.deque())
stage = runner_execution_context.stages[consuming_stage_name]
bundle_context_manager = runner_execution_context.bundle_manager_for(
stage)
_BUNDLE_LOGGER.debug(
'Running bundle for stage %s\n\tExpected outputs: %s timers: %s',
bundle_context_manager.stage.name,
bundle_context_manager.stage_data_outputs,
bundle_context_manager.stage_timer_outputs)
assert consuming_stage_name == bundle_context_manager.stage.name
bundle_counter += 1
bundle_results = self._execute_bundle(
runner_execution_context, bundle_context_manager, bundle_input)
if consuming_stage_name in monitoring_infos_by_stage:
monitoring_infos_by_stage[
consuming_stage_name] = consolidate_monitoring_infos(
itertools.chain(
bundle_results.process_bundle.monitoring_infos,
monitoring_infos_by_stage[consuming_stage_name]))
else:
assert isinstance(
bundle_results.process_bundle.monitoring_infos, Iterable)
monitoring_infos_by_stage[consuming_stage_name] = \
bundle_results.process_bundle.monitoring_infos
# Within monitoring_infos_by_stage we also keep monitoring information
# for the whole pipeline, which we key under ''.
if '' not in monitoring_infos_by_stage:
monitoring_infos_by_stage[''] = list(
pipeline_metrics.to_runner_api_monitoring_infos('').values())
else:
monitoring_infos_by_stage[''] = consolidate_monitoring_infos(
itertools.chain(
pipeline_metrics.to_runner_api_monitoring_infos(
'').values(),
monitoring_infos_by_stage['']))
# We only compute new ready bundles whenever we run out of current
# ready bundles, but we could do it after every new bundle and
# it should work either way.
if len(runner_execution_context.queues.ready_inputs) == 0:
self._schedule_ready_bundles(runner_execution_context)
assert len(runner_execution_context.queues.ready_inputs) == 0, (
'A total of %d ready bundles did not execute.'
% len(runner_execution_context.queues.ready_inputs))
assert len(
runner_execution_context.queues.watermark_pending_inputs) == 0, \
('A total of %d watermark-pending bundles did not execute.'
% len(runner_execution_context.queues.watermark_pending_inputs))
assert len(runner_execution_context.queues.time_pending_inputs) == 0, (
'A total of %d time-pending bundles did not execute.'
% len(runner_execution_context.queues.time_pending_inputs))
finally:
worker_handler_manager.close_all()
return RunnerResult(runner.PipelineState.DONE, monitoring_infos_by_stage)
def _schedule_ready_bundles(
self, runner_execution_context: execution.FnApiRunnerExecutionContext):
to_add_watermarks = []
while len(runner_execution_context.queues.watermark_pending_inputs) > 0:
(stage_name, bundle_watermark), data_input = (
runner_execution_context.queues.watermark_pending_inputs.deque())
current_watermark = (
runner_execution_context.watermark_manager.get_stage_node(
stage_name).input_watermark())
if current_watermark >= bundle_watermark:
_BUNDLE_LOGGER.debug(
'Watermark: %s. Enqueuing bundle scheduled for (%s) for stage %s',
current_watermark,
bundle_watermark,
stage_name)
_LOGGER.debug(
'Stage info:\n\t:%s\n',
runner_execution_context.watermark_manager.get_stage_node(
stage_name))
runner_execution_context.queues.ready_inputs.enque(
(stage_name, data_input))
else:
_BUNDLE_LOGGER.debug(
'Unable to add bundle for stage %s\n'
'\tStage input watermark: %s\n'
'\tBundle schedule watermark: %s',
stage_name,
current_watermark,
bundle_watermark)
_LOGGER.debug(
'Stage info:\n\t:%s\n',
runner_execution_context.watermark_manager.get_stage_node(
stage_name))
to_add_watermarks.append(((stage_name, bundle_watermark), data_input))
for elm in to_add_watermarks:
runner_execution_context.queues.watermark_pending_inputs.enque(elm)
to_add_real_time = []
while len(runner_execution_context.queues.time_pending_inputs) > 0:
current_time = runner_execution_context.clock.time()
(stage_name, work_timestamp), data_input = (
runner_execution_context.queues.time_pending_inputs.deque())
if current_time >= work_timestamp:
_LOGGER.debug(
'Time: %s. Enqueuing bundle scheduled for (%s) for stage %s',
current_time,
work_timestamp,
stage_name)
runner_execution_context.queues.ready_inputs.enque(
(stage_name, data_input))
else:
_LOGGER.debug(
'Unable to add bundle for stage %s\n'
'\tCurrent time: %s\n'
'\tBundle schedule time: %s\n',
stage_name,
current_time,
work_timestamp)
to_add_real_time.append(((stage_name, work_timestamp), data_input))
for elm in to_add_real_time:
runner_execution_context.queues.time_pending_inputs.enque(elm)
def _run_bundle_multiple_times_for_testing(
self,
runner_execution_context, # type: execution.FnApiRunnerExecutionContext
bundle_manager, # type: BundleManager
data_input, # type: MutableMapping[str, execution.PartitionableBuffer]
data_output, # type: DataOutput
fired_timers, # type: Mapping[translations.TimerFamilyId, execution.PartitionableBuffer]
expected_output_timers: OutputTimers,
) -> None:
"""
If bundle_repeat > 0, replay every bundle for profiling and debugging.
"""
# all workers share state, so use any worker_handler.
for _ in range(self._bundle_repeat):
try:
runner_execution_context.state_servicer.checkpoint()
bundle_manager.process_bundle(
data_input,
data_output,
fired_timers,
expected_output_timers,
dry_run=True)
finally:
runner_execution_context.state_servicer.restore()
@staticmethod
def _collect_written_timers(
bundle_context_manager: execution.BundleContextManager
) -> Tuple[Dict[translations.TimerFamilyId, timestamp.Timestamp],
OutputTimerData]:
"""Review output buffers, and collect written timers.
This function reviews a stage that has just been run. The stage will have
written timers to its output buffers. The function then takes the timers,
and adds them to the `newly_set_timers` dictionary, and the
timer_watermark_data dictionary.
The function then returns the following two elements in a tuple:
- timer_watermark_data: A dictionary mapping timer family to upcoming
timestamp to fire.
- newly_set_timers: A dictionary mapping timer family to timer buffers
to be passed to the SDK upon firing.
"""
timer_watermark_data = {}
newly_set_timers: OutputTimerData = {}
for (transform_id, timer_family_id) in bundle_context_manager.stage.timers:
written_timers = bundle_context_manager.get_buffer(
create_buffer_id(timer_family_id, kind='timers'), transform_id)
assert isinstance(written_timers, ListBuffer)
timer_coder_impl = bundle_context_manager.get_timer_coder_impl(
transform_id, timer_family_id)
if not written_timers.cleared:
timers_by_key_tag_and_window = {}
for elements_timers in written_timers:
for decoded_timer in timer_coder_impl.decode_all(elements_timers):
key_tag_win = (
decoded_timer.user_key,
decoded_timer.dynamic_timer_tag,
decoded_timer.windows[0])
if not decoded_timer.clear_bit:
timers_by_key_tag_and_window[key_tag_win] = decoded_timer
elif (decoded_timer.clear_bit and
key_tag_win in timers_by_key_tag_and_window):
del timers_by_key_tag_and_window[key_tag_win]
out = create_OutputStream()
for decoded_timer in timers_by_key_tag_and_window.values():
# Only add not cleared timer to fired timers.
if not decoded_timer.clear_bit:
timer_coder_impl.encode_to_stream(decoded_timer, out, True)
if (transform_id, timer_family_id) not in timer_watermark_data:
timer_watermark_data[(transform_id,
timer_family_id)] = timestamp.MAX_TIMESTAMP
timer_watermark_data[(transform_id, timer_family_id)] = min(
timer_watermark_data[(transform_id, timer_family_id)],
decoded_timer.hold_timestamp)
if (transform_id, timer_family_id) not in timer_watermark_data:
continue
newly_set_timers[(transform_id, timer_family_id)] = (
ListBuffer(coder_impl=timer_coder_impl),
timer_watermark_data[(transform_id, timer_family_id)])
newly_set_timers[(transform_id, timer_family_id)][0].append(out.get())
written_timers.clear()
return timer_watermark_data, newly_set_timers
def _add_sdk_delayed_applications_to_deferred_inputs(
self,
bundle_context_manager, # type: execution.BundleContextManager
bundle_result, # type: beam_fn_api_pb2.InstructionResponse
deferred_inputs # type: MutableMapping[str, execution.PartitionableBuffer]
):
# type: (...) -> Set[str]
"""Returns a set of PCollection IDs of PColls having delayed applications.
This transform inspects the bundle_context_manager, and bundle_result
objects, and adds all deferred inputs to the deferred_inputs object.
"""
pcolls_with_delayed_apps = set()
for delayed_application in bundle_result.process_bundle.residual_roots:
producer_name = bundle_context_manager.input_for(
delayed_application.application.transform_id,
delayed_application.application.input_id)
if producer_name not in deferred_inputs:
deferred_inputs[producer_name] = ListBuffer(
coder_impl=bundle_context_manager.get_input_coder_impl(
producer_name))
deferred_inputs[producer_name].append(
delayed_application.application.element)
transform = bundle_context_manager.process_bundle_descriptor.transforms[
producer_name]
# We take the output with tag 'out' from the producer transform. The
# producer transform is a GRPC read, and it has a single output.
pcolls_with_delayed_apps.add(only_element(transform.outputs.values()))
return pcolls_with_delayed_apps
def _add_residuals_and_channel_splits_to_deferred_inputs(
self,
splits, # type: List[beam_fn_api_pb2.ProcessBundleSplitResponse]
bundle_context_manager, # type: execution.BundleContextManager
last_sent, # type: MutableMapping[str, execution.PartitionableBuffer]
deferred_inputs # type: MutableMapping[str, execution.PartitionableBuffer]
):
# type: (...) -> Tuple[Set[str], Set[str]]
"""Returns a two sets representing PCollections with watermark holds.
The first set represents PCollections with delayed root applications.
The second set represents PTransforms with channel splits.
"""
pcolls_with_delayed_apps = set()
transforms_with_channel_splits = set()
prev_stops = {} # type: Dict[str, int]
for split in splits:
for delayed_application in split.residual_roots:
producer_name = bundle_context_manager.input_for(
delayed_application.application.transform_id,
delayed_application.application.input_id)
if producer_name not in deferred_inputs:
deferred_inputs[producer_name] = ListBuffer(
coder_impl=bundle_context_manager.get_input_coder_impl(
producer_name))
deferred_inputs[producer_name].append(
delayed_application.application.element)
# We take the output with tag 'out' from the producer transform. The
# producer transform is a GRPC read, and it has a single output.
pcolls_with_delayed_apps.add(
bundle_context_manager.process_bundle_descriptor.
transforms[producer_name].outputs['out'])
for channel_split in split.channel_splits:
coder_impl = bundle_context_manager.get_input_coder_impl(
channel_split.transform_id)
# TODO(SDF): This requires deterministic 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:
transform = (
bundle_context_manager.process_bundle_descriptor.transforms[
channel_split.transform_id])
assert transform.spec.urn == bundle_processor.DATA_INPUT_URN
transforms_with_channel_splits.add(transform.unique_name)
if channel_split.transform_id not in deferred_inputs:
coder_impl = bundle_context_manager.get_input_coder_impl(
channel_split.transform_id)
deferred_inputs[channel_split.transform_id] = ListBuffer(
coder_impl=coder_impl)
deferred_inputs[channel_split.transform_id].append(
coder_impl.encode_all(residual_elements))
prev_stops[
channel_split.transform_id] = channel_split.last_primary_element
return pcolls_with_delayed_apps, transforms_with_channel_splits
def _execute_bundle(self,
runner_execution_context, # type: execution.FnApiRunnerExecutionContext
bundle_context_manager, # type: execution.BundleContextManager
bundle_input: DataInput
) -> beam_fn_api_pb2.InstructionResponse:
"""Execute a bundle end-to-end.
Args:
runner_execution_context (execution.FnApiRunnerExecutionContext): An
object containing execution information for the pipeline.
bundle_context_manager (execution.BundleContextManager): A description of
the stage to execute, and its context.
bundle_input: The set of buffers to input into this bundle
"""
worker_handler_manager = runner_execution_context.worker_handler_manager
# TODO(pabloem): Should move this to be done once per stage
worker_handler_manager.register_process_bundle_descriptor(
bundle_context_manager.process_bundle_descriptor)
# We create the bundle manager here, as it can be reused for bundles of
# the same stage, but it may have to be created by-bundle later on.
bundle_manager = self._get_bundle_manager(bundle_context_manager)
last_result, deferred_inputs, newly_set_timers, watermark_updates = (
self._run_bundle(
runner_execution_context,
bundle_context_manager,
bundle_input,
bundle_context_manager.stage_data_outputs,
bundle_context_manager.stage_timer_outputs,
bundle_manager))
for pc_name, watermark in watermark_updates.items():
_BUNDLE_LOGGER.debug('Update: %s %s', pc_name, watermark)
runner_execution_context.watermark_manager.set_pcoll_watermark(
pc_name, watermark)
if deferred_inputs:
assert (runner_execution_context.watermark_manager.get_stage_node(
bundle_context_manager.stage.name).output_watermark()
< timestamp.MAX_TIMESTAMP), (
'wrong timestamp for %s. '
% runner_execution_context.watermark_manager.get_stage_node(
bundle_context_manager.stage.name))
runner_execution_context.queues.ready_inputs.enque(
(bundle_context_manager.stage.name, DataInput(deferred_inputs, {})))
self._enqueue_set_timers(
runner_execution_context,
bundle_context_manager,
newly_set_timers,
bundle_input)
if not deferred_inputs and not newly_set_timers:
# Marking all inputs as having been fully produced.
for _, output_pc in bundle_context_manager.stage_data_outputs.items():
_, update_output_pc = translations.split_buffer_id(output_pc)
runner_execution_context.watermark_manager.get_pcoll_node(
update_output_pc).set_produced_watermark(timestamp.MAX_TIMESTAMP)
# Store the required downstream side inputs into state so it is accessible
# for the worker when it runs bundles that consume this stage's output.
data_side_input = (
runner_execution_context.side_input_descriptors_by_stage.get(
bundle_context_manager.stage.name, {}))
runner_execution_context.commit_side_inputs_to_state(data_side_input)
buffers_to_clean = set()
known_consumers = set()
for _, buffer_id in bundle_context_manager.stage_data_outputs.items():
for (consuming_stage_name, consuming_transform) in \
runner_execution_context.buffer_id_to_consumer_pairs.get(buffer_id,
[]):
buffer = runner_execution_context.pcoll_buffers.get(buffer_id, None)
if (buffer_id in runner_execution_context.pcoll_buffers and
buffer_id not in buffers_to_clean):
buffers_to_clean.add(buffer_id)
elif buffer and buffer_id in buffers_to_clean:
# If the buffer_id has already been added to buffers_to_clean, this
# means that the buffer is being consumed by two separate stages,
# so we create a copy of the buffer for every new stage.
runner_execution_context.pcoll_buffers[buffer_id] = buffer.copy()
buffer = runner_execution_context.pcoll_buffers[buffer_id]
# If the buffer has already been added to be consumed by
# (stage, transform), then we don't need to add it again. This case
# can happen whenever we flatten the same PCollection with itself.
if (consuming_stage_name, consuming_transform,
buffer_id) in known_consumers:
continue
else:
known_consumers.add(
(consuming_stage_name, consuming_transform, buffer_id))
# We enqueue all of the pending output buffers to be scheduled at the
# MAX_TIMESTAMP for the downstream stage.
runner_execution_context.queues.watermark_pending_inputs.enque(
((consuming_stage_name, timestamp.MAX_TIMESTAMP),
DataInput({consuming_transform: buffer}, {}))) # type: ignore
for bid in buffers_to_clean:
if bid in runner_execution_context.pcoll_buffers:
del runner_execution_context.pcoll_buffers[bid]
return last_result
def _enqueue_set_timers(
self,
runner_execution_context: execution.FnApiRunnerExecutionContext,
bundle_context_manager: execution.BundleContextManager,
fired_timers: translations.OutputTimerData,
previous_bundle_input: DataInput):
empty_data_input: MutableMapping[str, execution.PartitionableBuffer] = {
k: ListBuffer(None)
for k in previous_bundle_input.data.keys()
}
current_time = runner_execution_context.clock.time()
current_watermark = \
runner_execution_context.watermark_manager.get_stage_node(
bundle_context_manager.stage.name).input_watermark()
for unique_timer_family in fired_timers:
timer_data, target_timestamp = fired_timers[unique_timer_family]
_, time_domain = bundle_context_manager.stage_timer_outputs[
unique_timer_family]
if time_domain == beam_runner_api_pb2.TimeDomain.PROCESSING_TIME:
if current_time >= target_timestamp:
runner_execution_context.queues.ready_inputs.enque((
bundle_context_manager.stage.name,
DataInput(empty_data_input, {unique_timer_family: timer_data})))
else:
runner_execution_context.queues.time_pending_inputs.enque(
((bundle_context_manager.stage.name, target_timestamp),
DataInput(empty_data_input, {unique_timer_family: timer_data})))
else:
assert time_domain == beam_runner_api_pb2.TimeDomain.EVENT_TIME
if current_watermark >= target_timestamp:
runner_execution_context.queues.ready_inputs.enque((
bundle_context_manager.stage.name,
DataInput(empty_data_input, {unique_timer_family: timer_data})))
else:
runner_execution_context.queues.watermark_pending_inputs.enque(
((bundle_context_manager.stage.name, target_timestamp),
DataInput(empty_data_input, {unique_timer_family: timer_data})))
def _get_bundle_manager(
self, bundle_context_manager: execution.BundleContextManager):
# TODO(pabloem): Consider moving this function to the BundleContextManager
cache_token_generator = FnApiRunner.get_cache_token_generator(static=False)
if bundle_context_manager.num_workers == 1:
# Avoid thread/processor pools for increased performance and debugability.
bundle_manager_type = BundleManager
elif bundle_context_manager.stage.is_stateful():
# State is keyed, and a single key cannot be processed concurrently.
# Alternatively, we could arrange to partition work by key.
# TODO(pabloem): Arrange to partition work by key.
bundle_manager_type = BundleManager
else:
bundle_manager_type = ParallelBundleManager
return bundle_manager_type(
bundle_context_manager,
self._progress_frequency,
cache_token_generator=cache_token_generator)
@staticmethod
def _build_watermark_updates(
runner_execution_context, # type: execution.FnApiRunnerExecutionContext
stage_inputs, # type: Iterable[str]
expected_timers, # type: Iterable[translations.TimerFamilyId]
pcolls_with_da, # type: Set[str]
transforms_w_splits, # type: Set[str]
watermarks_by_transform_and_timer_family # type: Dict[translations.TimerFamilyId, timestamp.Timestamp]
) -> Dict[Union[str, translations.TimerFamilyId], timestamp.Timestamp]:
"""Builds a dictionary of PCollection (or TimerFamilyId) to timestamp.
Args:
stage_inputs: represent the set of expected input PCollections for a
stage. These do not include timers.
expected_timers: represent the set of TimerFamilyIds that the stage can
expect to receive as inputs.
pcolls_with_da: represent the set of stage input PCollections that had
delayed applications.
transforms_w_splits: represent the set of transforms in the stage that had
input splits.
watermarks_by_transform_and_timer_family: represent the set of watermark
holds to be added for each timer family.
"""
updates = {
} # type: Dict[Union[str, translations.TimerFamilyId], timestamp.Timestamp]
def get_pcoll_id(transform_id):
buffer_id = runner_execution_context.input_transform_to_buffer_id[
transform_id]
# For IMPULSE-reading transforms, we use the transform name as buffer id.
if buffer_id == translations.IMPULSE_BUFFER:
pcollection_id = transform_id
else:
_, pcollection_id = translations.split_buffer_id(buffer_id)
return pcollection_id
# Any PCollections that have deferred applications should have their
# watermark held back.
for pcoll in pcolls_with_da:
updates[pcoll] = timestamp.MIN_TIMESTAMP
# Also any transforms with splits should have their input PCollection's
# watermark held back.
for tr in transforms_w_splits:
pcoll_id = get_pcoll_id(tr)
updates[pcoll_id] = timestamp.MIN_TIMESTAMP
# For all expected stage timers, we have two possible outcomes:
# 1) If the stage set a firing time for the timer, then we hold the
# watermark at that time
# 2) If the stage did not set a firing time for the timer, then we
# advance the watermark for that timer to MAX_TIMESTAMP.
for timer_pcoll_id in expected_timers:
updates[timer_pcoll_id] = watermarks_by_transform_and_timer_family.get(
timer_pcoll_id, timestamp.MAX_TIMESTAMP)
# For any PCollection in the set of stage inputs, if its watermark was not
# held back (i.e. there weren't splits in its consumer PTransform, and there
# weren't delayed applications of the PCollection's elements), then the
# watermark should be advanced to MAX_TIMESTAMP.
for transform_id in stage_inputs:
pcoll_id = get_pcoll_id(transform_id)
if pcoll_id not in updates:
updates[pcoll_id] = timestamp.MAX_TIMESTAMP
return updates
def _run_bundle(
self,
runner_execution_context, # type: execution.FnApiRunnerExecutionContext
bundle_context_manager, # type: execution.BundleContextManager
bundle_input: DataInput,
data_output: DataOutput,
expected_timer_output: OutputTimers,
bundle_manager # type: BundleManager
) -> Tuple[beam_fn_api_pb2.InstructionResponse,
Dict[str, execution.PartitionableBuffer],
OutputTimerData,
Dict[Union[str, translations.TimerFamilyId], timestamp.Timestamp]]:
"""Execute a bundle, and return a result object, and deferred inputs."""
data_input = bundle_input.data
input_timers = bundle_input.timers
self._run_bundle_multiple_times_for_testing(
runner_execution_context,
bundle_manager,
data_input,
data_output,
input_timers,
expected_timer_output)
result, splits = bundle_manager.process_bundle(
data_input, data_output, input_timers, expected_timer_output)