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

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

                Author: ASF GitHub Bot
            Created on: 21/Feb/20 18:10
            Start Date: 21/Feb/20 18:10
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on pull request #10375: [BEAM-8537] 
Provide WatermarkEstimator to track watermark
URL: https://github.com/apache/beam/pull/10375#discussion_r382723627
 
 

 ##########
 File path: sdks/python/apache_beam/io/watermark_estimators.py
 ##########
 @@ -0,0 +1,150 @@
+#
+# 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.
+#
+
+"""A collection of WatermarkEstimator implementations that SplittableDoFns
+can use."""
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+from apache_beam.io.iobase import WatermarkEstimator
+from apache_beam.transforms.core import WatermarkEstimatorProvider
+from apache_beam.utils.timestamp import Timestamp
+
+
+class MonotonicWatermarkEstimator(WatermarkEstimator):
+  """A WatermarkEstimator which assumes that timestamps of all ouput records
+  are increasing monotonically.
+  """
+  def __init__(self, timestamp):
+    """For a new <element, restriction> pair, the initial value is None. When
+    resuming processing, the initial timestamp will be the last reported
+    watermark.
+    """
+    self._watermark = timestamp
+
+  def observe_timestamp(self, timestamp):
+    if self._watermark is None:
+      self._watermark = timestamp
+    else:
+      # TODO(BEAM-9312): Consider making it configurable to deal with late
+      # timestamp.
+      if timestamp < self._watermark:
+        raise ValueError(
+            'A MonotonicWatermarkEstimator expects output '
+            'timestamp to be increasing monotonically.')
+      self._watermark = timestamp
+
+  def current_watermark(self):
+    return self._watermark
+
+  def get_estimator_state(self):
+    return self._watermark
+
+  @staticmethod
+  def default_provider():
+    """Provide a default WatermarkEstimatorProvider for
+    MonotonicWatermarkEstimator.
+    """
+    class DefaultMonotonicWatermarkEstimator(WatermarkEstimatorProvider):
+      def initial_estimator_state(self, element, restriction):
+        return None
+
+      def create_watermark_estimator(self, estimator_state):
+        return MonotonicWatermarkEstimator(estimator_state)
+
+    return DefaultMonotonicWatermarkEstimator()
+
+
+class WalltimeWatermarkEstimator(WatermarkEstimator):
+  """A WatermarkEstimator which uses processing time as the estimated 
watermark.
+  """
+  def __init__(self, timestamp=None):
+    self._timestamp = timestamp or Timestamp.now()
+
+  def observe_timestamp(self, timestamp):
+    pass
+
+  def current_watermark(self):
+    self._timestamp = max(self._timestamp, Timestamp.now())
+    return self._timestamp
+
+  def get_estimator_state(self):
+    return self._timestamp
+
+  @staticmethod
+  def default_provider():
+    """Provide a default WatermarkEstimatorProvider for
+    WalltimeWatermarkEstimator.
+    """
+    class DefaultWalltimeWatermarkEstimator(WatermarkEstimatorProvider):
+      def initial_estimator_state(self, element, restriction):
+        return None
+
+      def create_watermark_estimator(self, estimator_state):
+        return WalltimeWatermarkEstimator(estimator_state)
+
+    return DefaultWalltimeWatermarkEstimator()
+
+
+class ManualWatermarkEstimator(WatermarkEstimator):
+  """A WatermarkEstimator which is controlled manually from within a DoFn.
+
+  The DoFn must invoke set_watermark to advance the watermark.
+  """
+  def __init__(self, watermark):
+    self._watermark = watermark
+
+  def observe_timestamp(self, timestamp):
+    pass
+
+  def current_watermark(self):
+    return self._watermark
+
+  def get_estimator_state(self):
+    return self._watermark
+
+  def set_watermark(self, timestamp):
+    # Please call set_watermark after calling restriction_tracker.try_claim() 
to
+    # prevent advancing watermark early.
+    # TODO(BEAM-7473): It's possible that getting a slightly stale watermark
 
 Review comment:
   I would drop the TODO and close the JIRA since we can't reliably resolve it 
with how our APIs are setup since try_claim needs to advance the watermark at 
the same time.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 390820)
    Time Spent: 17h  (was: 16h 50m)

> Provide WatermarkEstimatorProvider for different types of WatermarkEstimator
> ----------------------------------------------------------------------------
>
>                 Key: BEAM-8537
>                 URL: https://issues.apache.org/jira/browse/BEAM-8537
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-core, sdk-py-harness
>            Reporter: Boyuan Zhang
>            Assignee: Boyuan Zhang
>            Priority: Major
>          Time Spent: 17h
>  Remaining Estimate: 0h
>
> This is a follow up for in-progress PR:  
> https://github.com/apache/beam/pull/9794.
> Current implementation in PR9794 provides a default implementation of 
> WatermarkEstimator. For further work, we want to let WatermarkEstimator to be 
> a pure Interface. We'll provide a WatermarkEstimatorProvider to be able to 
> create a custom WatermarkEstimator per windowed value. It should be similar 
> to how we track restriction for SDF: 
> WatermarkEstimator <---> RestrictionTracker 
> WatermarkEstimatorProvider <---> RestrictionTrackerProvider
> WatermarkEstimatorParam <---> RestrictionDoFnParam



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

Reply via email to