def process_bundle_progress( self, request, # type: beam_fn_api_pb2.ProcessBundleProgressRequest instruction_id # type: str ): # type: (...) -> beam_fn_api_pb2.InstructionResponse # It is an error to get progress for a not-in-flight bundle. processor = self.bundle_processor_cache.lookup(request.instruction_id) if processor: self._log_lull_in_bundle_processor(processor) monitoring_infos = processor.monitoring_infos() if processor else [] return beam_fn_api_pb2.InstructionResponse( instruction_id=instruction_id, process_bundle_progress=beam_fn_api_pb2. ProcessBundleProgressResponse(monitoring_infos=monitoring_infos, monitoring_data={ SHORT_ID_CACHE.getShortId(info): info.payload for info in monitoring_infos }))
def process_bundle(self, request, instruction_id): bundle_processor = self.bundle_processor_cache.get( instruction_id, request.process_bundle_descriptor_reference) try: with bundle_processor.state_handler.process_instruction_id( instruction_id): with self.maybe_profile(instruction_id): delayed_applications = bundle_processor.process_bundle( instruction_id) response = beam_fn_api_pb2.InstructionResponse( instruction_id=instruction_id, process_bundle=beam_fn_api_pb2.ProcessBundleResponse( residual_roots=delayed_applications, metrics=bundle_processor.metrics(), monitoring_infos=bundle_processor.monitoring_infos( ))) # TODO(boyuanz): Don't release here if finalize is needed. self.bundle_processor_cache.release(instruction_id) return response except: # pylint: disable=broad-except # Don't re-use bundle processors on failure. self.bundle_processor_cache.discard(instruction_id) raise
def process_bundle_progress(self, request, instruction_id): return beam_fn_api_pb2.InstructionResponse( instruction_id=instruction_id, error='Not Supported')
def register(self, request, instruction_id): for process_bundle_descriptor in request.process_bundle_descriptor: self.fns[process_bundle_descriptor.id] = process_bundle_descriptor return beam_fn_api_pb2.InstructionResponse( instruction_id=instruction_id, register=beam_fn_api_pb2.RegisterResponse())
def register(self, request, instruction_id): for process_bundle_descriptor in request.process_bundle_descriptor: self.bundle_processor_cache.register(process_bundle_descriptor) return beam_fn_api_pb2.InstructionResponse( instruction_id=instruction_id, register=beam_fn_api_pb2.RegisterResponse())
def _run_stage(self, runner_execution_context, # type: execution.FnApiRunnerExecutionContext bundle_context_manager, # type: execution.BundleContextManager ): # type: (...) -> beam_fn_api_pb2.InstructionResponse """Run an individual stage. Args: runner_execution_context (execution.FnApiRunnerExecutionContext): An object containing execution information for the pipeline. stage (translations.Stage): A description of the stage to execute. """ worker_handler_list = bundle_context_manager.worker_handlers worker_handler_manager = runner_execution_context.worker_handler_manager _LOGGER.info('Running %s', bundle_context_manager.stage.name) (data_input, data_side_input, data_output, expected_timer_output) = self._extract_endpoints( bundle_context_manager, runner_execution_context) worker_handler_manager.register_process_bundle_descriptor( bundle_context_manager.process_bundle_descriptor) # 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(runner_execution_context, data_side_input) # Change cache token across bundle repeats cache_token_generator = FnApiRunner.get_cache_token_generator(static=False) self._run_bundle_multiple_times_for_testing( runner_execution_context, bundle_context_manager, data_input, data_output, {}, expected_timer_output, cache_token_generator=cache_token_generator) bundle_manager = ParallelBundleManager( worker_handler_list, bundle_context_manager.get_buffer, bundle_context_manager.get_input_coder_impl, bundle_context_manager.process_bundle_descriptor, self._progress_frequency, num_workers=self._num_workers, cache_token_generator=cache_token_generator) # For the first time of processing, we don't have fired timers as inputs. result, splits = bundle_manager.process_bundle(data_input, data_output, {}, expected_timer_output) last_result = result last_sent = data_input # 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. # We also don't partition fired timer inputs for the same reason. bundle_manager._num_workers = 1 while True: deferred_inputs = {} # type: Dict[str, PartitionableBuffer] fired_timers = {} self._collect_written_timers_and_add_to_fired_timers( bundle_context_manager, fired_timers) # Queue any process-initiated delayed bundle applications. for delayed_application in last_result.process_bundle.residual_roots: name = bundle_context_manager.input_for( delayed_application.application.transform_id, delayed_application.application.input_id) if name not in deferred_inputs: deferred_inputs[name] = ListBuffer( coder_impl=bundle_context_manager.get_input_coder_impl(name)) deferred_inputs[name].append(delayed_application.application.element) # Queue any runner-initiated delayed bundle applications. self._add_residuals_and_channel_splits_to_deferred_inputs( splits, bundle_context_manager, last_sent, deferred_inputs) if deferred_inputs or fired_timers: # 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( coder_impl=bundle_context_manager.get_input_coder_impl( other_input)) # TODO(robertwb): merge results last_result, splits = bundle_manager.process_bundle( deferred_inputs, data_output, fired_timers, expected_timer_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