[
https://issues.apache.org/jira/browse/BEAM-8537?focusedWorklogId=379610&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-379610
]
ASF GitHub Bot logged work on BEAM-8537:
----------------------------------------
Author: ASF GitHub Bot
Created on: 30/Jan/20 21:41
Start Date: 30/Jan/20 21:41
Worklog Time Spent: 10m
Work Description: robertwb commented on pull request #10375: [BEAM-8537]
Provide WatermarkEstimator to track watermark
URL: https://github.com/apache/beam/pull/10375#discussion_r373198094
##########
File path: sdks/python/apache_beam/runners/common.py
##########
@@ -527,11 +734,11 @@ def __init__(self,
signature.is_stateful_dofn())
self.user_state_context = user_state_context
self.is_splittable = signature.is_splittable_dofn()
- self.watermark_estimator = self.signature.get_watermark_estimator()
- self.watermark_estimator_param = (
- self.signature.process_method.watermark_estimator_arg_name
- if self.watermark_estimator else None)
- self.threadsafe_restriction_tracker = None # type:
Optional[iobase.ThreadsafeRestrictionTracker]
+ self.threadsafe_restriction_tracker = None
+ self.threadsafe_watermark_estimator = None
+ # The lock which guarantee synchronization for both
+ # ThreadsafeRestrictionTracker and ThreadsafeWatermarkEstimator.
+ self._synchronized_lock = threading.Lock()
Review comment:
IMHO making this lock external to both classes, and having some methods that
do locking (exclusive of holding the lock) and others that require holding the
lock, introduces complexity that is not worth the benefit (presumably making
sure the watermark is never slightly stale).
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 379610)
Time Spent: 7h 20m (was: 7h 10m)
> 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: 7h 20m
> 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)