[ https://issues.apache.org/jira/browse/BEAM-9650?focusedWorklogId=417223&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-417223 ]
ASF GitHub Bot logged work on BEAM-9650: ---------------------------------------- Author: ASF GitHub Bot Created on: 06/Apr/20 22:14 Start Date: 06/Apr/20 22:14 Worklog Time Spent: 10m Work Description: Ardagan commented on pull request #11182: [BEAM-9650] Add Heartbeat Transform and slowly changing side input documentation URL: https://github.com/apache/beam/pull/11182#discussion_r404420904 ########## File path: sdks/python/apache_beam/transforms/periodicsequence.py ########## @@ -0,0 +1,160 @@ +# +# 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. +# + +from __future__ import absolute_import + +import time + +import apache_beam as beam +import apache_beam.runners.sdf_utils as sdf_utils +from apache_beam.io.restriction_trackers import OffsetRange +from apache_beam.io.restriction_trackers import OffsetRestrictionTracker +from apache_beam.transforms import core +from apache_beam.transforms import window +from apache_beam.transforms.ptransform import PTransform +from apache_beam.transforms.window import TimestampedValue +from apache_beam.utils import timestamp +from apache_beam.utils.timestamp import MAX_TIMESTAMP +from apache_beam.utils.timestamp import Timestamp + + +class ImpulseSeqGenRestrictionProvider(core.RestrictionProvider): + def initial_restriction(self, element): + start, end, interval = element + return OffsetRange(start - interval, end) + + def create_tracker(self, restriction): + return ImpulseSeqGenRestrictionTracker(restriction) + + def restriction_size(self, unused_element, restriction): + return restriction.size() + + +class ImpulseSeqGenRestrictionTracker(OffsetRestrictionTracker): + def try_split(self, fraction_of_remainder): + if not self._checkpointed: + if fraction_of_remainder != 0: + return None + + if self._current_position is None: + cur = self._range.start + else: + cur = self._current_position + split_point = cur + + if split_point < self._range.stop: + self._checkpointed = True + self._range, residual_range = self._range.split_at(split_point) + return self._range, residual_range + + def cur_pos(self): + return self._current_position + + def try_claim(self, pos): + if ((self._last_claim_attempt is None) or + (pos > self._last_claim_attempt and pos == self._range.stop)): + self._last_claim_attempt = pos + return True + else: + return super(ImpulseSeqGenRestrictionTracker, self).try_claim(pos) + + +class ImpulseSeqGenDoFn(beam.DoFn): + def process( + self, + element, + restriction_tracker=beam.DoFn.RestrictionParam( + ImpulseSeqGenRestrictionProvider())): + + _, _, interval = element + + assert isinstance(restriction_tracker, sdf_utils.RestrictionTrackerView) + + t = time.time() + cr = restriction_tracker.current_restriction() + current_timestamp = cr.start + + restriction_tracker.try_claim(current_timestamp) Review comment: This guarantees that left part of deferring is treated as complete. ---------------------------------------------------------------- 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: 417223) Time Spent: 1h (was: 50m) > Add consistent slowly changing side inputs support > -------------------------------------------------- > > Key: BEAM-9650 > URL: https://issues.apache.org/jira/browse/BEAM-9650 > Project: Beam > Issue Type: Bug > Components: io-ideas > Reporter: Mikhail Gryzykhin > Assignee: Mikhail Gryzykhin > Priority: Major > Time Spent: 1h > Remaining Estimate: 0h > > Add implementation for slowly changing dimentions based on [design > doc](https://docs.google.com/document/d/1LDY_CtsOJ8Y_zNv1QtkP6AGFrtzkj1q5EW_gSChOIvg/edit] -- This message was sent by Atlassian Jira (v8.3.4#803005)