-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathexecutor.py
More file actions
722 lines (611 loc) · 25.4 KB
/
executor.py
File metadata and controls
722 lines (611 loc) · 25.4 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
#
# 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.
#
"""An executor that schedules and executes applied ptransforms."""
# pytype: skip-file
import collections
import itertools
import logging
import queue
import threading
import traceback
from typing import TYPE_CHECKING
from typing import Any
from typing import Dict
from typing import FrozenSet
from typing import Optional
from typing import Set
from weakref import WeakValueDictionary
from apache_beam.metrics.execution import MetricsContainer
from apache_beam.runners.worker import statesampler
from apache_beam.transforms import sideinputs
from apache_beam.utils import counters
if TYPE_CHECKING:
from apache_beam import pvalue
from apache_beam.runners.direct.bundle_factory import _Bundle
from apache_beam.runners.direct.evaluation_context import EvaluationContext
from apache_beam.runners.direct.transform_evaluator import TransformEvaluatorRegistry
_LOGGER = logging.getLogger(__name__)
class _ExecutorService(object):
"""Thread pool for executing tasks in parallel."""
class CallableTask(object):
def call(self, state_sampler):
pass
@property
def name(self):
return None
class _ExecutorServiceWorker(threading.Thread):
"""Worker thread for executing a single task at a time."""
# Amount to block waiting for getting an item from the queue in seconds.
TIMEOUT = 5
def __init__(
self, queue: 'queue.Queue[_ExecutorService.CallableTask]', index):
super().__init__()
self.queue = queue
self._index = index
self._default_name = 'ExecutorServiceWorker-' + str(index)
self._update_name()
self.shutdown_requested = False
# Stop worker thread when main thread exits.
self.daemon = True
self.start()
def _update_name(self, task=None):
if task and task.name:
name = task.name
else:
name = self._default_name
self.name = 'Thread: %d, %s (%s)' % (
self._index, name, 'executing' if task else 'idle')
def _get_task_or_none(self) -> Optional['_ExecutorService.CallableTask']:
try:
# Do not block indefinitely, otherwise we may not act for a requested
# shutdown.
return self.queue.get(
timeout=_ExecutorService._ExecutorServiceWorker.TIMEOUT)
except queue.Empty:
return None
def run(self):
state_sampler = statesampler.StateSampler('', counters.CounterFactory())
statesampler.set_current_tracker(state_sampler)
while not self.shutdown_requested:
task = self._get_task_or_none()
if task:
try:
if not self.shutdown_requested:
self._update_name(task)
task.call(state_sampler)
self._update_name()
finally:
self.queue.task_done()
def shutdown(self):
self.shutdown_requested = True
def __init__(self, num_workers):
self.queue: queue.Queue[_ExecutorService.CallableTask] = queue.Queue()
self.workers = [
_ExecutorService._ExecutorServiceWorker(self.queue, i)
for i in range(num_workers)
]
self.shutdown_requested = False
def submit(self, task: '_ExecutorService.CallableTask') -> None:
assert isinstance(task, _ExecutorService.CallableTask)
if not self.shutdown_requested:
self.queue.put(task)
def await_completion(self):
for worker in self.workers:
worker.join()
def shutdown(self):
self.shutdown_requested = True
for worker in self.workers:
worker.shutdown()
# Consume all the remaining items in the queue
while not self.queue.empty():
try:
self.queue.get_nowait()
self.queue.task_done()
except queue.Empty:
continue
# All existing threads will eventually terminate (after they complete their
# last task).
class _TransformEvaluationState(object):
def __init__(self, executor_service, scheduled: Set['TransformExecutor']):
self.executor_service = executor_service
self.scheduled = scheduled
def schedule(self, work):
self.scheduled.add(work)
self.executor_service.submit(work)
def complete(self, completed_work):
self.scheduled.remove(completed_work)
class _ParallelEvaluationState(_TransformEvaluationState):
"""A TransformEvaluationState with unlimited parallelism.
Any TransformExecutor scheduled will be immediately submitted to the
ExecutorService.
A principal use of this is for evaluators that can generate output bundles
only using the input bundle (e.g. ParDo).
"""
pass
class _SerialEvaluationState(_TransformEvaluationState):
"""A TransformEvaluationState with a single work queue.
Any TransformExecutor scheduled will be placed on the work queue. Only one
item of work will be submitted to the ExecutorService at any time.
A principal use of this is for evaluators that keeps a global state such as
_GroupByKeyOnly.
"""
def __init__(self, executor_service, scheduled):
super().__init__(executor_service, scheduled)
self.serial_queue = collections.deque()
self.currently_evaluating = None
self._lock = threading.Lock()
def complete(self, completed_work):
self._update_currently_evaluating(None, completed_work)
super().complete(completed_work)
def schedule(self, new_work):
self._update_currently_evaluating(new_work, None)
def _update_currently_evaluating(self, new_work, completed_work):
with self._lock:
if new_work:
self.serial_queue.append(new_work)
if completed_work:
assert self.currently_evaluating == completed_work
self.currently_evaluating = None
if self.serial_queue and not self.currently_evaluating:
next_work = self.serial_queue.pop()
self.currently_evaluating = next_work
super().schedule(next_work)
class _TransformExecutorServices(object):
"""Schedules and completes TransformExecutors.
Controls the concurrency as appropriate for the applied transform the executor
exists for.
"""
def __init__(self, executor_service: _ExecutorService) -> None:
self._executor_service = executor_service
self._scheduled: Set[TransformExecutor] = set()
self._parallel = _ParallelEvaluationState(
self._executor_service, self._scheduled)
self._serial_cache: WeakValueDictionary[
Any, _SerialEvaluationState] = WeakValueDictionary()
def parallel(self) -> _ParallelEvaluationState:
return self._parallel
def serial(self, step: Any) -> _SerialEvaluationState:
cached = self._serial_cache.get(step)
if not cached:
cached = _SerialEvaluationState(self._executor_service, self._scheduled)
self._serial_cache[step] = cached
return cached
@property
def executors(self) -> FrozenSet['TransformExecutor']:
return frozenset(self._scheduled)
class _CompletionCallback(object):
"""The default completion callback.
The default completion callback is used to complete transform evaluations
that are triggered due to the arrival of elements from an upstream transform,
or for a source transform.
"""
def __init__(
self,
evaluation_context: 'EvaluationContext',
all_updates,
timer_firings=None):
self._evaluation_context = evaluation_context
self._all_updates = all_updates
self._timer_firings = timer_firings or []
def handle_result(
self, transform_executor, input_committed_bundle, transform_result):
output_committed_bundles = self._evaluation_context.handle_result(
input_committed_bundle, self._timer_firings, transform_result)
for output_committed_bundle in output_committed_bundles:
self._all_updates.offer(
_ExecutorServiceParallelExecutor._ExecutorUpdate(
transform_executor, committed_bundle=output_committed_bundle))
for unprocessed_bundle in transform_result.unprocessed_bundles:
self._all_updates.offer(
_ExecutorServiceParallelExecutor._ExecutorUpdate(
transform_executor, unprocessed_bundle=unprocessed_bundle))
return output_committed_bundles
def handle_exception(self, transform_executor, exception):
self._all_updates.offer(
_ExecutorServiceParallelExecutor._ExecutorUpdate(
transform_executor, exception=exception))
class TransformExecutor(_ExecutorService.CallableTask):
"""For internal use only; no backwards-compatibility guarantees.
TransformExecutor will evaluate a bundle using an applied ptransform.
A CallableTask responsible for constructing a TransformEvaluator and
evaluating it on some bundle of input, and registering the result using the
completion callback.
"""
_MAX_RETRY_PER_BUNDLE = 4
def __init__(
self,
transform_evaluator_registry: 'TransformEvaluatorRegistry',
evaluation_context: 'EvaluationContext',
input_bundle: '_Bundle',
fired_timers,
applied_ptransform,
completion_callback,
transform_evaluation_state: _TransformEvaluationState):
self._transform_evaluator_registry = transform_evaluator_registry
self._evaluation_context = evaluation_context
self._input_bundle = input_bundle
# For non-empty bundles, store the window of the max EOW.
# TODO(mariagh): Move to class _Bundle's inner _StackedWindowedValues
self._latest_main_input_window = None
if input_bundle.has_elements():
self._latest_main_input_window = input_bundle._elements[0].windows[0]
for elem in input_bundle.get_elements_iterable():
if elem.windows[0].end > self._latest_main_input_window.end:
self._latest_main_input_window = elem.windows[0]
self._fired_timers = fired_timers
self._applied_ptransform = applied_ptransform
self._completion_callback = completion_callback
self._transform_evaluation_state = transform_evaluation_state
self._side_input_values: Dict[pvalue.AsSideInput, Any] = {}
self.blocked = False
self._call_count = 0
self._retry_count = 0
self._max_retries_per_bundle = TransformExecutor._MAX_RETRY_PER_BUNDLE
def call(self, state_sampler):
self._call_count += 1
assert self._call_count <= (1 + len(self._applied_ptransform.side_inputs))
metrics_container = MetricsContainer(self._applied_ptransform.full_label)
start_state = state_sampler.scoped_state(
self._applied_ptransform.full_label,
'start',
metrics_container=metrics_container)
process_state = state_sampler.scoped_state(
self._applied_ptransform.full_label,
'process',
metrics_container=metrics_container)
finish_state = state_sampler.scoped_state(
self._applied_ptransform.full_label,
'finish',
metrics_container=metrics_container)
with start_state:
# Side input initialization should be accounted for in start_state.
for side_input in self._applied_ptransform.side_inputs:
# Find the projection of main's window onto the side input's window.
window_mapping_fn = side_input._view_options().get(
'window_mapping_fn', sideinputs._global_window_mapping_fn)
main_onto_side_window = window_mapping_fn(
self._latest_main_input_window)
block_until = main_onto_side_window.end
if side_input not in self._side_input_values:
value = self._evaluation_context.get_value_or_block_until_ready(
side_input, self, block_until)
if not value:
# Monitor task will reschedule this executor once the side input is
# available.
return
self._side_input_values[side_input] = value
side_input_values = [
self._side_input_values[side_input]
for side_input in self._applied_ptransform.side_inputs
]
while self._retry_count < self._max_retries_per_bundle:
try:
self.attempt_call(
metrics_container,
side_input_values,
start_state,
process_state,
finish_state)
break
except Exception as e:
self._retry_count += 1
_LOGGER.error(
'Exception at bundle %r, due to an exception.\n %s',
self._input_bundle,
traceback.format_exc())
if self._retry_count == self._max_retries_per_bundle:
_LOGGER.error(
'Giving up after %s attempts.', self._max_retries_per_bundle)
self._completion_callback.handle_exception(self, e)
self._evaluation_context.metrics().commit_physical(
self._input_bundle, metrics_container.get_cumulative())
self._transform_evaluation_state.complete(self)
def attempt_call(
self,
metrics_container,
side_input_values,
start_state,
process_state,
finish_state):
"""Attempts to run a bundle."""
evaluator = self._transform_evaluator_registry.get_evaluator(
self._applied_ptransform, self._input_bundle, side_input_values)
with start_state:
evaluator.start_bundle()
with process_state:
if self._fired_timers:
for timer_firing in self._fired_timers:
evaluator.process_timer_wrapper(timer_firing)
if self._input_bundle:
for value in self._input_bundle.get_elements_iterable():
evaluator.process_element(value)
with finish_state:
result = evaluator.finish_bundle()
result.logical_metric_updates = metrics_container.get_cumulative()
self._completion_callback.handle_result(self, self._input_bundle, result)
return result
class Executor(object):
"""For internal use only; no backwards-compatibility guarantees."""
def __init__(self, *args, **kwargs):
self._executor = _ExecutorServiceParallelExecutor(*args, **kwargs)
def start(self, roots):
self._executor.start(roots)
def await_completion(self):
self._executor.await_completion()
def shutdown(self):
self._executor.request_shutdown()
class _ExecutorServiceParallelExecutor(object):
"""An internal implementation for Executor."""
NUM_WORKERS = 1
def __init__(
self,
value_to_consumers,
transform_evaluator_registry,
evaluation_context: 'EvaluationContext'):
self.executor_service = _ExecutorService(
_ExecutorServiceParallelExecutor.NUM_WORKERS)
self.transform_executor_services = _TransformExecutorServices(
self.executor_service)
self.value_to_consumers = value_to_consumers
self.transform_evaluator_registry = transform_evaluator_registry
self.evaluation_context = evaluation_context
self.all_updates = _ExecutorServiceParallelExecutor._TypedUpdateQueue(
_ExecutorServiceParallelExecutor._ExecutorUpdate)
self.visible_updates = _ExecutorServiceParallelExecutor._TypedUpdateQueue(
_ExecutorServiceParallelExecutor._VisibleExecutorUpdate)
self.default_completion_callback = _CompletionCallback(
evaluation_context, self.all_updates)
def start(self, roots):
self.root_nodes = frozenset(roots)
self.all_nodes = frozenset(
itertools.chain(
roots, *itertools.chain(self.value_to_consumers.values())))
self.node_to_pending_bundles = {}
for root_node in self.root_nodes:
provider = (
self.transform_evaluator_registry.get_root_bundle_provider(root_node))
self.node_to_pending_bundles[root_node] = provider.get_root_bundles()
self.executor_service.submit(
_ExecutorServiceParallelExecutor._MonitorTask(self))
def await_completion(self):
update = self.visible_updates.take()
try:
if update.exception:
raise update.exception
finally:
self.executor_service.shutdown()
self.executor_service.await_completion()
def request_shutdown(self):
self.executor_service.shutdown()
self.executor_service.await_completion()
self.evaluation_context.shutdown()
def schedule_consumers(self, committed_bundle: '_Bundle') -> None:
if committed_bundle.pcollection in self.value_to_consumers:
consumers = self.value_to_consumers[committed_bundle.pcollection]
for applied_ptransform in consumers:
self.schedule_consumption(
applied_ptransform,
committed_bundle, [],
self.default_completion_callback)
def schedule_unprocessed_bundle(self, applied_ptransform, unprocessed_bundle):
self.node_to_pending_bundles[applied_ptransform].append(unprocessed_bundle)
def schedule_consumption(
self,
consumer_applied_ptransform,
committed_bundle: '_Bundle',
fired_timers,
on_complete):
"""Schedules evaluation of the given bundle with the transform."""
assert consumer_applied_ptransform
assert committed_bundle
assert on_complete
if self.transform_evaluator_registry.should_execute_serially(
consumer_applied_ptransform):
transform_executor_service: _TransformEvaluationState = (
self.transform_executor_services.serial(consumer_applied_ptransform))
else:
transform_executor_service = self.transform_executor_services.parallel()
transform_executor = TransformExecutor(
self.transform_evaluator_registry,
self.evaluation_context,
committed_bundle,
fired_timers,
consumer_applied_ptransform,
on_complete,
transform_executor_service)
transform_executor_service.schedule(transform_executor)
class _TypedUpdateQueue(object):
"""Type checking update queue with blocking and non-blocking operations."""
def __init__(self, item_type):
self._item_type = item_type
self._queue = queue.Queue()
def poll(self):
try:
item = self._queue.get_nowait()
self._queue.task_done()
return item
except queue.Empty:
return None
def take(self):
# The implementation of Queue.Queue.get() does not propagate
# KeyboardInterrupts when a timeout is not used. We therefore use a
# one-second timeout in the following loop to allow KeyboardInterrupts
# to be correctly propagated.
while True:
try:
item = self._queue.get(timeout=1)
self._queue.task_done()
return item
except queue.Empty:
pass
def offer(self, item):
assert isinstance(item, self._item_type)
self._queue.put_nowait(item)
class _ExecutorUpdate(object):
"""An internal status update on the state of the executor."""
def __init__(
self,
transform_executor,
committed_bundle=None,
unprocessed_bundle=None,
exception=None):
self.transform_executor = transform_executor
# Exactly one of them should be not-None
assert sum(
[bool(committed_bundle), bool(unprocessed_bundle),
bool(exception)]) == 1
self.committed_bundle = committed_bundle
self.unprocessed_bundle = unprocessed_bundle
self.exception = exception
class _VisibleExecutorUpdate(object):
"""An update of interest to the user.
Used for awaiting the completion to decide whether to return normally or
raise an exception.
"""
def __init__(self, exception=None):
self.finished = exception is not None
self.exception = exception
class _MonitorTask(_ExecutorService.CallableTask):
"""MonitorTask continuously runs to ensure that pipeline makes progress."""
def __init__(self, executor: '_ExecutorServiceParallelExecutor') -> None:
self._executor = executor
@property
def name(self):
return 'monitor'
def call(self, state_sampler):
try:
update = self._executor.all_updates.poll()
while update:
if update.committed_bundle:
self._executor.schedule_consumers(update.committed_bundle)
elif update.unprocessed_bundle:
self._executor.schedule_unprocessed_bundle(
update.transform_executor._applied_ptransform,
update.unprocessed_bundle)
else:
assert update.exception
_LOGGER.warning(
'A task failed with exception: %s', update.exception)
self._executor.visible_updates.offer(
_ExecutorServiceParallelExecutor._VisibleExecutorUpdate(
update.exception))
update = self._executor.all_updates.poll()
self._executor.evaluation_context.schedule_pending_unblocked_tasks(
self._executor.executor_service)
self._add_work_if_necessary(self._fire_timers())
except Exception as e: # pylint: disable=broad-except
_LOGGER.error('Monitor task died due to exception.\n %s', e)
self._executor.visible_updates.offer(
_ExecutorServiceParallelExecutor._VisibleExecutorUpdate(e))
finally:
if not self._should_shutdown():
self._executor.executor_service.submit(self)
def _should_shutdown(self) -> bool:
"""Checks whether the pipeline is completed and should be shut down.
If there is anything in the queue of tasks to do or
if there are any realtime timers set, do not shut down.
Otherwise, check if all the transforms' watermarks are complete.
If they are not, the pipeline is not progressing (stall detected).
Whether the pipeline has stalled or not, the executor should shut
down the pipeline.
Returns:
True only if the pipeline has reached a terminal state and should
be shut down.
"""
if self._is_executing():
# There are some bundles still in progress.
return False
watermark_manager = self._executor.evaluation_context._watermark_manager
_, any_unfired_realtime_timers = watermark_manager.extract_all_timers()
if any_unfired_realtime_timers:
return False
else:
if self._executor.evaluation_context.is_done():
self._executor.visible_updates.offer(
_ExecutorServiceParallelExecutor._VisibleExecutorUpdate())
else:
# Nothing is scheduled for execution, but watermarks incomplete.
self._executor.visible_updates.offer(
_ExecutorServiceParallelExecutor._VisibleExecutorUpdate((
Exception('Monitor task detected a pipeline stall.'),
None,
None)))
self._executor.executor_service.shutdown()
return True
def _fire_timers(self):
"""Schedules triggered consumers if any timers fired.
Returns:
True if timers fired.
"""
transform_fired_timers, _ = (
self._executor.evaluation_context.extract_all_timers())
for applied_ptransform, fired_timers in transform_fired_timers:
# Use an empty committed bundle. just to trigger.
empty_bundle = (
self._executor.evaluation_context.create_empty_committed_bundle(
applied_ptransform.inputs[0]))
timer_completion_callback = _CompletionCallback(
self._executor.evaluation_context,
self._executor.all_updates,
timer_firings=fired_timers)
self._executor.schedule_consumption(
applied_ptransform,
empty_bundle,
fired_timers,
timer_completion_callback)
return bool(transform_fired_timers)
def _is_executing(self) -> bool:
"""Checks whether the job is still executing.
Returns:
True if there is at least one non-blocked TransformExecutor active."""
executors = self._executor.transform_executor_services.executors
if not executors:
# Nothing is executing.
return False
# Ensure that at least one of those executors is not blocked.
for transform_executor in executors:
if not transform_executor.blocked:
return True
return False
def _add_work_if_necessary(self, timers_fired):
"""Adds more work from the roots if pipeline requires more input.
If all active TransformExecutors are in a blocked state, add more work
from root nodes that may have additional work. This ensures that if a
pipeline has elements available from the root nodes it will add those
elements when necessary.
Args:
timers_fired: True if any timers fired prior to this call.
"""
# If any timers have fired, they will add more work; No need to add more.
if timers_fired:
return
if self._is_executing():
# We have at least one executor that can proceed without adding
# additional work.
return
# All current TransformExecutors are blocked; add more work from any
# pending bundles.
for applied_ptransform in self._executor.all_nodes:
if not self._executor.evaluation_context.is_done(applied_ptransform):
pending_bundles = self._executor.node_to_pending_bundles.get(
applied_ptransform, [])
for bundle in pending_bundles:
self._executor.schedule_consumption(
applied_ptransform,
bundle, [],
self._executor.default_completion_callback)
self._executor.node_to_pending_bundles[applied_ptransform] = []