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 20:52:42 UTC

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

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



##########
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:
       Once we switch to streaming, for timer PCollections, we'll mainly use their watermark directly, and we'll set the watermark directly; while for data PCollections we'll mainly use the upstream watermark.
   
   In this change we're not seeing these changes, but watermark information will be extracted from splits and delayed applications, and PCollection watermarks will not advance to MAX_TIMESTAMP, they will depend on their producers.
   
   - Timers will trigger whenever all inputs to a stage reach their time.
   - Data PCollection watermarks will advance from the top




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