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

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

                Author: ASF GitHub Bot
            Created on: 14/Jan/20 02:37
            Start Date: 14/Jan/20 02:37
    Worklog Time Spent: 10m 
      Work Description: boyuanzz commented on pull request #10375: [BEAM-8537] 
Provide WatermarkEstimator to track watermark
URL: https://github.com/apache/beam/pull/10375#discussion_r366128556
 
 

 ##########
 File path: sdks/python/apache_beam/io/watermark_estimators.py
 ##########
 @@ -0,0 +1,101 @@
+#
+# 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."""
+
+from __future__ import absolute_import
+
+from apache_beam.io.iobase import WatermarkEstimator
+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 tracking <element, restriction> pair, the initial value
+    should be None. When resuming from residual, the initial timestamp should
+    be the same timestamp as the estimated watermark from primary.
+    """
+    self._watermark = timestamp
+
+  def observe_timestamp(self, timestamp):
+    if self._watermark is None:
+      self._watermark = timestamp
+    else:
+      if timestamp < self._watermark:
+        raise ValueError('A MonotonicWatermarkEstimator expects output '
+                         'timestamp to be increasing monotonically.')
 
 Review comment:
   The `MonotonicWatermarkEstimator.observe_timestamp()` expects the timestamp 
of output_record. For me, conceptually it's a timestamp.
 
----------------------------------------------------------------
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: 371301)
    Time Spent: 5h 10m  (was: 5h)

> 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: 5h 10m
>  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