[ https://issues.apache.org/jira/browse/BEAM-8537?focusedWorklogId=379680&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-379680 ]
ASF GitHub Bot logged work on BEAM-8537: ---------------------------------------- Author: ASF GitHub Bot Created on: 31/Jan/20 00:18 Start Date: 31/Jan/20 00:18 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_r373262173 ########## File path: sdks/python/apache_beam/runners/common.py ########## @@ -765,41 +978,63 @@ def _invoke_process_per_window(self, # ProcessSizedElementAndRestriction. self.threadsafe_restriction_tracker.check_done() deferred_status = self.threadsafe_restriction_tracker.deferred_status() - output_watermark = None - if self.watermark_estimator: - output_watermark = self.watermark_estimator.current_watermark() if deferred_status: deferred_restriction, deferred_watermark = deferred_status element = windowed_value.value size = self.signature.get_restriction_provider().restriction_size( element, deferred_restriction) - return (( - windowed_value.with_value(((element, deferred_restriction), size)), - output_watermark), deferred_watermark) - return None + if self.threadsafe_watermark_estimator: + output_watermark = ( + self.threadsafe_watermark_estimator.current_watermark()) + estimator_state = ( + self.threadsafe_watermark_estimator.get_estimator_state()) + return (( + windowed_value.with_value( + ((element, (deferred_restriction, estimator_state)), size)), + output_watermark), deferred_watermark) + else: + return (( + windowed_value.with_value( + ((element, deferred_restriction), size)), None), + deferred_watermark) + return None def try_split(self, fraction): # type: (...) -> Optional[Tuple[SplitResultType, SplitResultType]] - if self.threadsafe_restriction_tracker and self.current_windowed_value: - # Temporary workaround for [BEAM-7473]: get current_watermark before - # split, in case watermark gets advanced before getting split results. - # In worst case, current_watermark is always stale, which is ok. - if self.watermark_estimator: - current_watermark = self.watermark_estimator.current_watermark() - else: - current_watermark = None - split = self.threadsafe_restriction_tracker.try_split(fraction) + restriction_tracker = self.threadsafe_restriction_tracker + current_windowed_value = self.current_windowed_value + if restriction_tracker and current_windowed_value: + current_watermark = None + # Make sure that the RestrictionTracker and WatermarkEstimator are locked + # together. + with self._synchronized_lock: + split = restriction_tracker.try_split(fraction) + if self.threadsafe_watermark_estimator: + current_watermark = ( + self.threadsafe_watermark_estimator.current_watermark_with_lock()) + estimator_state = (self.threadsafe_watermark_estimator + .get_estimator_state_with_lock()) if split: primary, residual = split element = self.current_windowed_value.value restriction_provider = self.signature.get_restriction_provider() primary_size = restriction_provider.restriction_size(element, primary) residual_size = restriction_provider.restriction_size(element, residual) - return ( - ((self.current_windowed_value.with_value(( - (element, primary), primary_size)), None), None), - ((self.current_windowed_value.with_value(( - (element, residual), residual_size)), current_watermark), None)) + if self.threadsafe_watermark_estimator: + return ( + ((self.current_windowed_value.with_value(( Review comment: I like the idea of named tuple. I also agree the primary should be different from the residual. If that sounds good, I can make these changes together with moving `ThreadsafeRestrictionTracker` out from `common.py`. ---------------------------------------------------------------- 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: 379680) Time Spent: 9h 10m (was: 9h) > 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: 9h 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)