You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/03/02 01:13:51 UTC

[GitHub] [beam] robertwb commented on a change in pull request #11296: [BEAM-9640] Sketching watermark tracking on FnApiRunner

robertwb commented on a change in pull request #11296:
URL: https://github.com/apache/beam/pull/11296#discussion_r585156189



##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -391,16 +419,22 @@ def _run_bundle_multiple_times_for_testing(
       finally:
         runner_execution_context.state_servicer.restore()
 
-  def _collect_written_timers_and_add_to_fired_timers(
-      self,
-      bundle_context_manager,  # type: execution.BundleContextManager
-      fired_timers  # type: Dict[Tuple[str, str], ListBuffer]
-  ):
-    # type: (...) -> None
-
+  @staticmethod
+  def _collect_written_timers(
+      bundle_context_manager: execution.BundleContextManager,
+      newly_set_timers: Dict[Tuple[str, str], ListBuffer],

Review comment:
       Should this be a return value as well? 

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/visualization_tools.py
##########
@@ -0,0 +1,115 @@
+#
+# 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.
+#
+
+"""Set of utilities to visualize a pipeline to be executed by FnApiRunner."""
+from typing import Set
+from typing import Tuple
+
+from apache_beam.runners.portability.fn_api_runner.translations import Stage
+from apache_beam.runners.portability.fn_api_runner.watermark_manager import WatermarkManager
+
+
+def show_stage(stage: Stage):
+  try:
+    import graphviz
+  except ImportError:
+    import warnings
+    warnings.warn('Unable to draw pipeline. graphviz library missing.')

Review comment:
       Any reason not to make this a dependency? (E.g. is it fairly large?)

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -391,16 +419,22 @@ def _run_bundle_multiple_times_for_testing(
       finally:
         runner_execution_context.state_servicer.restore()
 
-  def _collect_written_timers_and_add_to_fired_timers(
-      self,
-      bundle_context_manager,  # type: execution.BundleContextManager
-      fired_timers  # type: Dict[Tuple[str, str], ListBuffer]
-  ):
-    # type: (...) -> None
-
+  @staticmethod
+  def _collect_written_timers(
+      bundle_context_manager: execution.BundleContextManager,
+      newly_set_timers: Dict[Tuple[str, str], ListBuffer],
+  ) -> Dict[Tuple[str, str], timestamp.Timestamp]:
+    """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.

Review comment:
       What does it return?

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -570,18 +661,76 @@ def merge_results(last_result):
 
     return final_result
 
+  @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
+      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
+
+    for pcoll in pcolls_with_da:
+      updates[pcoll] = timestamp.MIN_TIMESTAMP
+
+    for tr in transforms_w_splits:
+      pcoll_id = get_pcoll_id(tr)
+      updates[pcoll_id] = timestamp.MIN_TIMESTAMP
+
+    for timer_pcoll_id, ts in watermarks_by_transform_and_timer_family.items():

Review comment:
       It might be simpler to do something like
   
   ```
   for timer_pcoll_id in expected_timers:
     updates[timer_pcoll_id] = watermarks_by_transform_and_timer_family.get(
         timestamp.MAX_TIMESTAMP)
   ```
   
   than these two loops here. Or could timer_pcoll_id be in pcolls_with_da and/or transforms_w_splits?

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/watermark_manager.py
##########
@@ -0,0 +1,199 @@
+#
+# 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.
+#
+
+"""Utilities for managing watermarks for a pipeline execution by FnApiRunner."""
+
+from __future__ import absolute_import
+
+from typing import Any
+from typing import Dict
+from typing import List
+from typing import Set
+from typing import Union
+
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.fn_api_runner import translations
+from apache_beam.runners.portability.fn_api_runner.translations import split_buffer_id
+from apache_beam.runners.worker import bundle_processor
+from apache_beam.utils import proto_utils
+from apache_beam.utils import timestamp
+
+
+class WatermarkManager(object):
+  """Manages the watermarks of a pipeline's stages.
+    It works by constructing an internal graph representation of the pipeline,
+    and keeping track of dependencies."""
+  class WatermarkNode(object):
+    def __init__(self, name):
+      self.name = name
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no output watermark %s' % self)
+
+    def input_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no input watermark %s' % self)
+
+    def watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no own watermark %s' % self)
+
+    def upstream_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no upstream watermark %s' % self)
+
+  class PCollectionNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.PCollectionNode, self).__init__(name)
+      self._watermark = timestamp.MIN_TIMESTAMP
+      self.producers: Set[WatermarkManager.StageNode] = set()
+
+    def __str__(self):
+      return 'PCollectionNode<producers=%s>' % list(self.producers)
+
+    def set_watermark(self, wm: timestamp.Timestamp):
+      self._watermark = min(self.upstream_watermark(), wm)
+
+    def upstream_watermark(self):
+      if self.producers:
+        return min(p.input_watermark() for p in self.producers)
+      else:
+        return timestamp.MAX_TIMESTAMP
+
+    def watermark(self):
+      return self._watermark
+
+  class StageNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.StageNode, self).__init__(name)
+      # We keep separate inputs and side inputs because side inputs
+      # should hold back a stage's input watermark, to hold back execution
+      # for that stage; but they should not be considered when calculating
+      # the output watermark of the stage, because only the main input
+      # can actually advance that watermark.
+      self.inputs: Set[WatermarkManager.PCollectionNode] = set()
+      self.side_inputs: Set[WatermarkManager.PCollectionNode] = set()
+      self.outputs: Set[WatermarkManager.PCollectionNode] = set()
+
+    def __str__(self):
+      return 'StageNode<inputs=%s,side_inputs=%s' % (
+          [i.name for i in self.inputs], [i.name for i in self.side_inputs])
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self):
+      if not self.outputs:
+        return self.input_watermark()
+      else:
+        return min(o.watermark() for o in self.outputs)
+
+    def input_watermark(self):
+      if not self.inputs:
+        return timestamp.MAX_TIMESTAMP
+      w = min(i.upstream_watermark() for i in self.inputs)
+
+      if self.side_inputs:
+        w = min(w, min(i.upstream_watermark() for i in self.side_inputs))
+      return w
+
+  def __init__(self, stages):
+    # type: (List[translations.Stage]) -> None
+    self._watermarks_by_name: Dict[Any,
+                                   Union[
+                                       WatermarkManager.StageNode,
+                                       WatermarkManager.PCollectionNode]] = {}
+    for s in stages:
+      stage_name = s.name
+      stage_node = WatermarkManager.StageNode(stage_name)
+      self._watermarks_by_name[stage_name] = stage_node
+
+      def add_pcollection(
+          pcname: str, snode: WatermarkManager.StageNode
+      ) -> WatermarkManager.PCollectionNode:
+        if pcname not in self._watermarks_by_name:
+          self._watermarks_by_name[pcname] = WatermarkManager.PCollectionNode(
+              pcname)
+        pcnode = self._watermarks_by_name[pcname]
+        assert isinstance(pcnode, WatermarkManager.PCollectionNode)
+        snode.inputs.add(pcnode)
+        node = self._watermarks_by_name[pcname]
+        assert isinstance(node, WatermarkManager.PCollectionNode)
+        return node
+
+      # 1. Get stage inputs, create nodes for them, add to _watermarks_by_name,
+      #    and add as inputs to stage node.
+      for transform in s.transforms:
+        if transform.spec.urn == bundle_processor.DATA_INPUT_URN:
+          buffer_id = transform.spec.payload
+          if buffer_id == translations.IMPULSE_BUFFER:
+            pcoll_name = transform.unique_name
+            add_pcollection(pcoll_name, stage_node)
+            continue
+          else:
+            _, pcoll_name = split_buffer_id(buffer_id)
+          add_pcollection(pcoll_name, stage_node)
+
+      # 2. Get stage timers, and add them as inputs to the stage.
+      for transform in s.transforms:
+        if transform.spec.urn in translations.PAR_DO_URNS:
+          payload = proto_utils.parse_Bytes(
+              transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
+          for timer_family_id in payload.timer_family_specs.keys():
+            timer_pcoll_name = (transform.unique_name, timer_family_id)
+            self._watermarks_by_name[
+                timer_pcoll_name] = WatermarkManager.PCollectionNode(
+                    timer_pcoll_name)
+            timer_pcoll_node = self._watermarks_by_name[timer_pcoll_name]
+            assert isinstance(
+                timer_pcoll_node, WatermarkManager.PCollectionNode)
+            stage_node.inputs.add(timer_pcoll_node)
+
+      # 3. Get stage outputs, create nodes for them, add to _watermarks_by_name,
+      #    and add stage as their producer
+      for transform in s.transforms:
+        if transform.spec.urn == bundle_processor.DATA_OUTPUT_URN:
+          buffer_id = transform.spec.payload
+          _, pcoll_name = split_buffer_id(buffer_id)
+          if pcoll_name not in self._watermarks_by_name:
+            self._watermarks_by_name[
+                pcoll_name] = WatermarkManager.PCollectionNode(pcoll_name)
+          pcoll_node = self._watermarks_by_name[pcoll_name]
+          assert isinstance(pcoll_node, WatermarkManager.PCollectionNode)
+          pcoll_node.producers.add(stage_node)
+          stage_node.outputs.add(pcoll_node)
+
+      # 4. Get stage side inputs, create nodes for them, add to
+      #    _watermarks_by_name, and add them as side inputs of the stage.
+      for pcoll_name in s.side_inputs():
+        if pcoll_name not in self._watermarks_by_name:
+          self._watermarks_by_name[
+              pcoll_name] = WatermarkManager.PCollectionNode(pcoll_name)
+        pcoll_node = self._watermarks_by_name[pcoll_name]
+        assert isinstance(pcoll_node, WatermarkManager.PCollectionNode)
+        stage_node.side_inputs.add(pcoll_node)
+
+  def get_node(self, name):
+    # type: (str) -> Union[PCollectionNode, StageNode]
+    return self._watermarks_by_name[name]
+
+  def get_watermark(self, name) -> timestamp.Timestamp:

Review comment:
       get/set watermark is never used on stage nodes, right? Does it make sense to keep them in the same dictionary? 

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/watermark_manager.py
##########
@@ -0,0 +1,206 @@
+#
+# 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.
+#
+
+"""Utilities for managing watermarks for a pipeline execution by FnApiRunner."""
+
+from __future__ import absolute_import
+
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.fn_api_runner import translations
+from apache_beam.runners.portability.fn_api_runner.translations import split_buffer_id
+from apache_beam.runners.worker import bundle_processor
+from apache_beam.utils import proto_utils
+from apache_beam.utils import timestamp
+
+
+class WatermarkManager(object):
+  """Manages the watermarks of a pipeline's stages.
+    It works by constructing an internal graph representation of the pipeline,
+    and keeping track of dependencies."""
+  class WatermarkNode(object):
+    def __init__(self, name):
+      self.name = name
+
+  class PCollectionNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.PCollectionNode, self).__init__(name)
+      self._watermark = timestamp.MIN_TIMESTAMP
+      self.producers = set()
+
+    def __str__(self):
+      return 'PCollectionNode<producers=[%s]' % ([i for i in self.producers])
+
+    def set_watermark(self, wm):
+      # print('setting watermark for %s to %s' % (self, wm))
+      self._watermark = min(self.upstream_watermark(), wm)
+
+    def upstream_watermark(self):
+      if self.producers:
+        return min(p.output_watermark() for p in self.producers)
+      else:
+        return timestamp.MAX_TIMESTAMP
+
+    def watermark(self):
+      if self._watermark:
+        return self._watermark
+      else:
+        return self.upstream_watermark()
+
+  class StageNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.StageNode, self).__init__(name)
+      # We keep separate inputs and side inputs because side inputs
+      # should hold back a stage's input watermark, to hold back execution
+      # for that stage; but they should not be considered when calculating
+      # the output watermark of the stage, because only the main input
+      # can actually advance that watermark.
+      self.inputs = set()
+      self.side_inputs = set()
+
+    def __str__(self):
+      return 'StageNode<inputs=[%s],side_inputs=[%s]' % (
+          [i.name for i in self.inputs], [i.name for i in self.side_inputs])
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self):
+      w = min(i.watermark() for i in self.inputs)
+      return w
+
+    def input_watermark(self):
+      w = min(i.upstream_watermark() for i in self.inputs)
+
+      if self.side_inputs:
+        w = min(w, min(i.upstream_watermark() for i in self.side_inputs))
+      return w
+
+  def __init__(self, stages):
+    # type: (List[translations.Stage]) -> None
+    self._watermarks_by_name = {}
+    for s in stages:
+      stage_name = s.name
+      stage_node = WatermarkManager.StageNode(stage_name)
+      self._watermarks_by_name[stage_name] = stage_node
+
+      # 1. Get stage inputs, create nodes for them, add to _watermarks_by_name,
+      #    and add as inputs to stage node.
+      for transform in s.transforms:
+        if transform.spec.urn == bundle_processor.DATA_INPUT_URN:
+          buffer_id = transform.spec.payload
+          if buffer_id == translations.IMPULSE_BUFFER:

Review comment:
       OK, well let's at least unify this logic, i.e.
   
   ```
   if buffer_id == translations.IMPULSE_BUFFER
     pcoll_name = transform.unique_name
   else:
     _, pcoll_name = split_buffer_id(buffer_id)
   add_pcollection(pcoll_name, stage_node)
   ```
   
   rather than adding a continue. (We could also inline add_pcollection, for consistency, as it's not used elsewhere.)

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/watermark_manager.py
##########
@@ -0,0 +1,199 @@
+#
+# 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.
+#
+
+"""Utilities for managing watermarks for a pipeline execution by FnApiRunner."""
+
+from __future__ import absolute_import
+
+from typing import Any
+from typing import Dict
+from typing import List
+from typing import Set
+from typing import Union
+
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.fn_api_runner import translations
+from apache_beam.runners.portability.fn_api_runner.translations import split_buffer_id
+from apache_beam.runners.worker import bundle_processor
+from apache_beam.utils import proto_utils
+from apache_beam.utils import timestamp
+
+
+class WatermarkManager(object):
+  """Manages the watermarks of a pipeline's stages.
+    It works by constructing an internal graph representation of the pipeline,
+    and keeping track of dependencies."""
+  class WatermarkNode(object):
+    def __init__(self, name):
+      self.name = name
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no output watermark %s' % self)
+
+    def input_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no input watermark %s' % self)
+
+    def watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no own watermark %s' % self)
+
+    def upstream_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no upstream watermark %s' % self)
+
+  class PCollectionNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.PCollectionNode, self).__init__(name)
+      self._watermark = timestamp.MIN_TIMESTAMP
+      self.producers: Set[WatermarkManager.StageNode] = set()
+
+    def __str__(self):
+      return 'PCollectionNode<producers=%s>' % list(self.producers)
+
+    def set_watermark(self, wm: timestamp.Timestamp):
+      self._watermark = min(self.upstream_watermark(), wm)
+
+    def upstream_watermark(self):
+      if self.producers:
+        return min(p.input_watermark() for p in self.producers)
+      else:
+        return timestamp.MAX_TIMESTAMP
+
+    def watermark(self):
+      return self._watermark
+
+  class StageNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.StageNode, self).__init__(name)
+      # We keep separate inputs and side inputs because side inputs
+      # should hold back a stage's input watermark, to hold back execution
+      # for that stage; but they should not be considered when calculating
+      # the output watermark of the stage, because only the main input
+      # can actually advance that watermark.
+      self.inputs: Set[WatermarkManager.PCollectionNode] = set()
+      self.side_inputs: Set[WatermarkManager.PCollectionNode] = set()
+      self.outputs: Set[WatermarkManager.PCollectionNode] = set()
+
+    def __str__(self):
+      return 'StageNode<inputs=%s,side_inputs=%s' % (
+          [i.name for i in self.inputs], [i.name for i in self.side_inputs])
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self):
+      if not self.outputs:
+        return self.input_watermark()
+      else:
+        return min(o.watermark() for o in self.outputs)
+
+    def input_watermark(self):
+      if not self.inputs:
+        return timestamp.MAX_TIMESTAMP
+      w = min(i.upstream_watermark() for i in self.inputs)
+
+      if self.side_inputs:
+        w = min(w, min(i.upstream_watermark() for i in self.side_inputs))
+      return w
+
+  def __init__(self, stages):
+    # type: (List[translations.Stage]) -> None
+    self._watermarks_by_name: Dict[Any,
+                                   Union[
+                                       WatermarkManager.StageNode,
+                                       WatermarkManager.PCollectionNode]] = {}
+    for s in stages:
+      stage_name = s.name
+      stage_node = WatermarkManager.StageNode(stage_name)
+      self._watermarks_by_name[stage_name] = stage_node
+
+      def add_pcollection(
+          pcname: str, snode: WatermarkManager.StageNode
+      ) -> WatermarkManager.PCollectionNode:
+        if pcname not in self._watermarks_by_name:
+          self._watermarks_by_name[pcname] = WatermarkManager.PCollectionNode(
+              pcname)
+        pcnode = self._watermarks_by_name[pcname]
+        assert isinstance(pcnode, WatermarkManager.PCollectionNode)
+        snode.inputs.add(pcnode)
+        node = self._watermarks_by_name[pcname]
+        assert isinstance(node, WatermarkManager.PCollectionNode)

Review comment:
       Didn't we just assert that above?

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/watermark_manager.py
##########
@@ -0,0 +1,199 @@
+#
+# 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.
+#
+
+"""Utilities for managing watermarks for a pipeline execution by FnApiRunner."""
+
+from __future__ import absolute_import
+
+from typing import Any
+from typing import Dict
+from typing import List
+from typing import Set
+from typing import Union
+
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.fn_api_runner import translations
+from apache_beam.runners.portability.fn_api_runner.translations import split_buffer_id
+from apache_beam.runners.worker import bundle_processor
+from apache_beam.utils import proto_utils
+from apache_beam.utils import timestamp
+
+
+class WatermarkManager(object):
+  """Manages the watermarks of a pipeline's stages.
+    It works by constructing an internal graph representation of the pipeline,
+    and keeping track of dependencies."""
+  class WatermarkNode(object):
+    def __init__(self, name):
+      self.name = name
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no output watermark %s' % self)
+
+    def input_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no input watermark %s' % self)
+
+    def watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no own watermark %s' % self)
+
+    def upstream_watermark(self) -> timestamp.Timestamp:
+      raise NotImplementedError('Node has no upstream watermark %s' % self)
+
+  class PCollectionNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.PCollectionNode, self).__init__(name)
+      self._watermark = timestamp.MIN_TIMESTAMP
+      self.producers: Set[WatermarkManager.StageNode] = set()
+
+    def __str__(self):
+      return 'PCollectionNode<producers=%s>' % list(self.producers)
+
+    def set_watermark(self, wm: timestamp.Timestamp):
+      self._watermark = min(self.upstream_watermark(), wm)
+
+    def upstream_watermark(self):
+      if self.producers:
+        return min(p.input_watermark() for p in self.producers)
+      else:
+        return timestamp.MAX_TIMESTAMP
+
+    def watermark(self):
+      return self._watermark
+
+  class StageNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.StageNode, self).__init__(name)
+      # We keep separate inputs and side inputs because side inputs
+      # should hold back a stage's input watermark, to hold back execution
+      # for that stage; but they should not be considered when calculating
+      # the output watermark of the stage, because only the main input
+      # can actually advance that watermark.
+      self.inputs: Set[WatermarkManager.PCollectionNode] = set()
+      self.side_inputs: Set[WatermarkManager.PCollectionNode] = set()
+      self.outputs: Set[WatermarkManager.PCollectionNode] = set()
+
+    def __str__(self):
+      return 'StageNode<inputs=%s,side_inputs=%s' % (
+          [i.name for i in self.inputs], [i.name for i in self.side_inputs])
+
+    def set_watermark(self, wm):
+      raise NotImplementedError('Stages do not have a watermark')
+
+    def output_watermark(self):
+      if not self.outputs:
+        return self.input_watermark()
+      else:
+        return min(o.watermark() for o in self.outputs)
+
+    def input_watermark(self):
+      if not self.inputs:
+        return timestamp.MAX_TIMESTAMP
+      w = min(i.upstream_watermark() for i in self.inputs)
+
+      if self.side_inputs:
+        w = min(w, min(i.upstream_watermark() for i in self.side_inputs))

Review comment:
       I'm trying to understand when node.watermark() vs. node.upstream_watermark() would be used. It seems we'd always want the output watermark of our predecessors as our input watermark. 

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -414,26 +448,50 @@ def _collect_written_timers_and_add_to_fired_timers(
           # Only add not cleared timer to fired timers.
           if not decoded_timer.clear_bit:
             timer_coder_impl.encode_to_stream(decoded_timer, out, True)
-        fired_timers[(transform_id, timer_family_id)] = ListBuffer(
+            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.fire_timestamp)
+        newly_set_timers[(transform_id, timer_family_id)] = ListBuffer(
             coder_impl=timer_coder_impl)
-        fired_timers[(transform_id, timer_family_id)].append(out.get())
+        newly_set_timers[(transform_id, timer_family_id)].append(out.get())
         written_timers.clear()
 
+    return timer_watermark_data
+
   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: (...) -> None
+    # 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:
-      name = bundle_context_manager.input_for(
+      producer_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)
+      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(transform.outputs['out'])

Review comment:
       More flexibly, you could do `only_element(transform.values())`

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/watermark_manager.py
##########
@@ -0,0 +1,206 @@
+#
+# 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.
+#
+
+"""Utilities for managing watermarks for a pipeline execution by FnApiRunner."""
+
+from __future__ import absolute_import
+
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.fn_api_runner import translations
+from apache_beam.runners.portability.fn_api_runner.translations import split_buffer_id
+from apache_beam.runners.worker import bundle_processor
+from apache_beam.utils import proto_utils
+from apache_beam.utils import timestamp
+
+
+class WatermarkManager(object):
+  """Manages the watermarks of a pipeline's stages.
+    It works by constructing an internal graph representation of the pipeline,
+    and keeping track of dependencies."""
+  class WatermarkNode(object):
+    def __init__(self, name):
+      self.name = name
+
+  class PCollectionNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.PCollectionNode, self).__init__(name)
+      self._watermark = timestamp.MIN_TIMESTAMP
+      self.producers = set()
+
+    def __str__(self):
+      return 'PCollectionNode<producers=[%s]' % ([i for i in self.producers])
+
+    def set_watermark(self, wm):
+      # print('setting watermark for %s to %s' % (self, wm))
+      self._watermark = min(self.upstream_watermark(), wm)
+
+    def upstream_watermark(self):
+      if self.producers:
+        return min(p.output_watermark() for p in self.producers)
+      else:
+        return timestamp.MAX_TIMESTAMP
+
+    def watermark(self):
+      if self._watermark:
+        return self._watermark
+      else:
+        return self.upstream_watermark()
+
+  class StageNode(WatermarkNode):
+    def __init__(self, name):
+      super(WatermarkManager.StageNode, self).__init__(name)
+      # We keep separate inputs and side inputs because side inputs

Review comment:
       OK.

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -570,18 +661,76 @@ def merge_results(last_result):
 
     return final_result
 
+  @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
+      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
+
+    for pcoll in pcolls_with_da:
+      updates[pcoll] = timestamp.MIN_TIMESTAMP
+
+    for tr in transforms_w_splits:
+      pcoll_id = get_pcoll_id(tr)
+      updates[pcoll_id] = timestamp.MIN_TIMESTAMP
+
+    for timer_pcoll_id, ts in watermarks_by_transform_and_timer_family.items():
+      if timer_pcoll_id not in updates:
+        updates[timer_pcoll_id] = timestamp.MAX_TIMESTAMP
+      updates[timer_pcoll_id] = min(ts, updates[timer_pcoll_id])
+
+    for timer_pcoll_id in expected_timers:
+      if timer_pcoll_id not in updates:
+        updates[timer_pcoll_id] = timestamp.MAX_TIMESTAMP
+
+    for input in stage_inputs:
+      pcoll_id = get_pcoll_id(input)
+      if pcoll_id not in updates:

Review comment:
       I'll admit I have a hard time keeping the exact ordering here in my head. E.g. is expected_timers in this set? In which of the loops above could updates[pcoll_id] have been set? 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org