[ 
https://issues.apache.org/jira/browse/BEAM-8823?focusedWorklogId=750673&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-750673
 ]

ASF GitHub Bot logged work on BEAM-8823:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 31/Mar/22 01:38
            Start Date: 31/Mar/22 01:38
    Worklog Time Spent: 10m 
      Work Description: pabloem commented on a change in pull request #16841:
URL: https://github.com/apache/beam/pull/16841#discussion_r839098985



##########
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:
+          _LOGGER.debug(
+              "Remaining ready bundles: %s\n"
+              "\tWatermark pending bunbles: %s\n"
+              "\tTime pending bunbles: %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
+
+          if '' not in monitoring_infos_by_stage:

Review comment:
       done

##########
File path: 
sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py
##########
@@ -187,6 +222,9 @@ def __init__(self,
     self._windowing = windowing
     self._grouped_output = None  # type: Optional[List[List[bytes]]]
 
+  def copy(self) -> 'GroupingBuffer':
+    return self

Review comment:
       added. thanks!




-- 
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]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 750673)
    Time Spent: 18h 10m  (was: 18h)

> Make FnApiRunner work by executing ready elements instead of stages
> -------------------------------------------------------------------
>
>                 Key: BEAM-8823
>                 URL: https://issues.apache.org/jira/browse/BEAM-8823
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-py-core
>            Reporter: Pablo Estrada
>            Priority: P3
>          Time Spent: 18h 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to