Thanks to visit codestin.com
Credit goes to github.com

Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Consider small periodic impulses as bounded.
  • Loading branch information
robertwb committed Apr 28, 2025
commit 9cc1da4b2322f71dc9a373c84f79519ebb8f2a71
1 change: 1 addition & 0 deletions sdks/python/apache_beam/transforms/combiners_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -991,6 +991,7 @@ def test_combine_globally_for_unbounded_source_with_default(self):
stop_timestamp=time.time() + 4,
fire_interval=1,
apply_windowing=False,
is_bounded=False,
)
| beam.Map(lambda x: ('c', 1))
| beam.WindowInto(
Expand Down
16 changes: 13 additions & 3 deletions sdks/python/apache_beam/transforms/periodicsequence.py
Original file line number Diff line number Diff line change
Expand Up @@ -178,25 +178,35 @@ def __init__(
start_timestamp=Timestamp.now(),
stop_timestamp=MAX_TIMESTAMP,
fire_interval=360.0,
apply_windowing=False):
apply_windowing=False,
is_bounded=None):
'''
:param start_timestamp: Timestamp for first element.
:param stop_timestamp: Timestamp after which no elements will be output.
:param fire_interval: Interval in seconds at which to output elements.
:param apply_windowing: Whether each element should be assigned to
individual window. If false, all elements will reside in global window.
:param is_bounded: whether to treat the output PCollection as bounded.
Defaults to True for small timestamp ranges and False for large ones.
'''
self.start_ts = start_timestamp
self.stop_ts = stop_timestamp
self.interval = fire_interval
self.apply_windowing = apply_windowing
self.is_bounded = (
stop_timestamp - start_timestamp < 60
if is_bounded is None else is_bounded)

def expand(self, pbegin):
result = (
sequence = (
pbegin
| 'ImpulseElement' >> beam.Create(
[(self.start_ts, self.stop_ts, self.interval)])
| 'GenSequence' >> beam.ParDo(ImpulseSeqGenDoFn())
| 'GenSequence' >> beam.ParDo(ImpulseSeqGenDoFn()))
if self.is_bounded:
sequence.is_bounded = True
result = (
sequence
| 'MapToTimestamped' >> beam.Map(lambda tt: TimestampedValue(tt, tt)))
if self.apply_windowing:
result = result | 'ApplyWindowing' >> beam.WindowInto(
Expand Down
3 changes: 2 additions & 1 deletion sdks/python/apache_beam/transforms/periodicsequence_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,8 @@ def test_periodicimpulse_default_start(self):
assert is_same_type, error

with TestPipeline() as p:
result = p | 'PeriodicImpulse' >> PeriodicImpulse(it, et, interval)
result = p | 'PeriodicImpulse' >> PeriodicImpulse(
it, et, interval, is_bounded=False)

k = [it + x * interval for x in range(0, int(duration / interval))]
self.assertEqual(result.is_bounded, False)
Expand Down
Loading