y1chi commented on a change in pull request #16841:
URL: https://github.com/apache/beam/pull/16841#discussion_r839063266
##########
File path:
sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -363,68 +364,164 @@ def run_stages(self,
self.NUM_FUSED_STAGES_COUNTER,
urn='internal:' + self.NUM_FUSED_STAGES_COUNTER)).update(
len(stages))
- monitoring_infos_by_stage = {}
+ 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)
+ stage_context.data_channel_coders,
+ self._num_workers)
try:
with self.maybe_profile():
- for stage in stages:
- bundle_context_manager = execution.BundleContextManager(
- runner_execution_context, stage, self._num_workers)
-
- assert (
- runner_execution_context.watermark_manager.get_stage_node(
- bundle_context_manager.stage.name
- ).input_watermark() == timestamp.MAX_TIMESTAMP), (
- 'wrong watermark for %s. Expected %s, but got %s.' % (
- runner_execution_context.watermark_manager.get_stage_node(
- bundle_context_manager.stage.name),
- timestamp.MAX_TIMESTAMP,
- runner_execution_context.watermark_manager.get_stage_node(
- bundle_context_manager.stage.name
- ).input_watermark()
- )
- )
-
- stage_results = self._run_stage(
- runner_execution_context, bundle_context_manager)
-
- assert (
- runner_execution_context.watermark_manager.get_stage_node(
- bundle_context_manager.stage.name
- ).input_watermark() == timestamp.MAX_TIMESTAMP), (
- 'wrong input watermark for %s. Expected %s, but got %s.' % (
- runner_execution_context.watermark_manager.get_stage_node(
- bundle_context_manager.stage.name),
- timestamp.MAX_TIMESTAMP,
- runner_execution_context.watermark_manager.get_stage_node(
- bundle_context_manager.stage.name
- ).output_watermark())
- )
-
- monitoring_infos_by_stage[stage.name] = (
- list(stage_results.process_bundle.monitoring_infos))
-
- monitoring_infos_by_stage[''] = list(
- pipeline_metrics.to_runner_api_monitoring_infos('').values())
+ # 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:
Review comment:
I think time-pending is more depend on the processing time rather than
the watermark? eg. if a stage sets a processing time timer to infinity future,
will it be skipped by _schedule_ready_bundles always and break the while loop?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]