You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/05/05 21:15:00 UTC

[jira] [Work logged] (BEAM-9640) Track PCollection watermark across bundle executions

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

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

                Author: ASF GitHub Bot
            Created on: 05/May/20 21:14
            Start Date: 05/May/20 21:14
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on a change in pull request #11296:
URL: https://github.com/apache/beam/pull/11296#discussion_r420388089



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

Review comment:
       Are these fired_timers, or timers_to_fire? 

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py
##########
@@ -296,6 +297,12 @@ def __init__(self,
     self.safe_coders = safe_coders
     self.data_channel_coders = data_channel_coders
 
+    self.transform_id_to_buffer_id = {

Review comment:
       Different transforms may have different input/output buffers associated with them. Perhaps name this `input_transform_to_buffer_id` or `buffer_id_by_consumer` or similar. 

##########
File path: sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
##########
@@ -384,13 +409,25 @@ def _collect_written_timers_and_add_to_fired_timers(
         out = create_OutputStream()
         for decoded_timer in timers_by_key_and_window.values():
           timer_coder_impl.encode_to_stream(decoded_timer, out, True)
+          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)
         fired_timers[(transform_id, timer_family_id)] = ListBuffer(
             coder_impl=timer_coder_impl)
         fired_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, bundle_result, deferred_inputs):
+    # type: (...) -> Set[str]
+
+    """Returns a set of PCollections with delayed applications."""

Review comment:
       Set of PCollection ids? Buffer ids? 

##########
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))

Review comment:
       Leftover debugging?

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

Review comment:
       `list(self.producers)` (Also, won't this give double brackets?) Also, '<' is unmatched and not typically for string representations. 

##########
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)

Review comment:
       I suppose timers are considered an input? 

##########
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)

Review comment:
       Nit: `return min(...)`

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

Review comment:
       Does initializing this hear mean that we never use the watermark of the producers. (I have to admit, I'm not quite understanding why the watermark of a PCollection would ever be different than that of its producers.)

##########
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:
+            pcoll_name = transform.unique_name
+          else:
+            _, 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)
+          stage_node.inputs.add(self._watermarks_by_name[pcoll_name])
+
+      # 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)
+            stage_node.inputs.add(self._watermarks_by_name[timer_pcoll_name])
+
+      # 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)
+          self._watermarks_by_name[pcoll_name].producers.add(stage_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)
+        stage_node.side_inputs.add(self._watermarks_by_name[pcoll_name])
+
+  def get_node(self, name):
+    # type: (str) -> WatermarkNode
+    return self._watermarks_by_name[name]
+
+  def get_watermark(self, name):
+    element = self._watermarks_by_name[name]

Review comment:
       Nit: (here and below): elide unneeded intermediate assignments and just chain. 

##########
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:

Review comment:
       What about self._watermark = 0? 

##########
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:
       side inputs and timers all have this effect, right? 

##########
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):

Review comment:
       This doesn't seem to take into account data that's "in flight." E.g. all the input watermarks could be at max-timestamp, but that doesn't mean that all the inputs' data has been consumed. 

##########
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):

Review comment:
       This doesn't seem right, the input watermarks should always be an upper bound on the output watermark. 

##########
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:
       Should we change `IMPULSE_BUFFER` to conform to the spec of all other buffer ids rather than branching on it? (Or at least push this branch into split_buffer_id)?




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


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

    Worklog Id:     (was: 430902)
    Time Spent: 1h 10m  (was: 1h)

> Track PCollection watermark across bundle executions
> ----------------------------------------------------
>
>                 Key: BEAM-9640
>                 URL: https://issues.apache.org/jira/browse/BEAM-9640
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-py-core
>            Reporter: Pablo Estrada
>            Assignee: Pablo Estrada
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> This can be done without relying on the watermark manager for execution.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)