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

Skip to content

Commit 9cc1da4

Browse files
committed
Consider small periodic impulses as bounded.
1 parent f0f7a42 commit 9cc1da4

3 files changed

Lines changed: 16 additions & 4 deletions

File tree

sdks/python/apache_beam/transforms/combiners_test.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -991,6 +991,7 @@ def test_combine_globally_for_unbounded_source_with_default(self):
991991
stop_timestamp=time.time() + 4,
992992
fire_interval=1,
993993
apply_windowing=False,
994+
is_bounded=False,
994995
)
995996
| beam.Map(lambda x: ('c', 1))
996997
| beam.WindowInto(

sdks/python/apache_beam/transforms/periodicsequence.py

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -178,25 +178,35 @@ def __init__(
178178
start_timestamp=Timestamp.now(),
179179
stop_timestamp=MAX_TIMESTAMP,
180180
fire_interval=360.0,
181-
apply_windowing=False):
181+
apply_windowing=False,
182+
is_bounded=None):
182183
'''
183184
:param start_timestamp: Timestamp for first element.
184185
:param stop_timestamp: Timestamp after which no elements will be output.
185186
:param fire_interval: Interval in seconds at which to output elements.
186187
:param apply_windowing: Whether each element should be assigned to
187188
individual window. If false, all elements will reside in global window.
189+
:param is_bounded: whether to treat the output PCollection as bounded.
190+
Defaults to True for small timestamp ranges and False for large ones.
188191
'''
189192
self.start_ts = start_timestamp
190193
self.stop_ts = stop_timestamp
191194
self.interval = fire_interval
192195
self.apply_windowing = apply_windowing
196+
self.is_bounded = (
197+
stop_timestamp - start_timestamp < 60
198+
if is_bounded is None else is_bounded)
193199

194200
def expand(self, pbegin):
195-
result = (
201+
sequence = (
196202
pbegin
197203
| 'ImpulseElement' >> beam.Create(
198204
[(self.start_ts, self.stop_ts, self.interval)])
199-
| 'GenSequence' >> beam.ParDo(ImpulseSeqGenDoFn())
205+
| 'GenSequence' >> beam.ParDo(ImpulseSeqGenDoFn()))
206+
if self.is_bounded:
207+
sequence.is_bounded = True
208+
result = (
209+
sequence
200210
| 'MapToTimestamped' >> beam.Map(lambda tt: TimestampedValue(tt, tt)))
201211
if self.apply_windowing:
202212
result = result | 'ApplyWindowing' >> beam.WindowInto(

sdks/python/apache_beam/transforms/periodicsequence_test.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -91,7 +91,8 @@ def test_periodicimpulse_default_start(self):
9191
assert is_same_type, error
9292

9393
with TestPipeline() as p:
94-
result = p | 'PeriodicImpulse' >> PeriodicImpulse(it, et, interval)
94+
result = p | 'PeriodicImpulse' >> PeriodicImpulse(
95+
it, et, interval, is_bounded=False)
9596

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

0 commit comments

Comments
 (0)