Skip to content

Commit 755082a

Browse files
authored
Fix flakiness in fuzzy tests and support TimestampTypes in PeriodicImpulse. (#35470)
* Support TimestampTypes in PeriodicImpulse. Fix flakiness in fuzzy tests. * Fix lints. * Add tests for different timestamp input types. * Use an identity function in map after ImpulseSeqGenDoFn
1 parent 431cb56 commit 755082a

File tree

2 files changed

+97
-49
lines changed

2 files changed

+97
-49
lines changed

sdks/python/apache_beam/transforms/periodicsequence.py

Lines changed: 32 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -33,28 +33,27 @@
3333
from apache_beam.transforms.window import TimestampedValue
3434
from apache_beam.utils import timestamp
3535
from apache_beam.utils.timestamp import MAX_TIMESTAMP
36+
from apache_beam.utils.timestamp import Duration
3637
from apache_beam.utils.timestamp import Timestamp
38+
from apache_beam.utils.timestamp import TimestampTypes
3739

3840

3941
class ImpulseSeqGenRestrictionProvider(core.RestrictionProvider):
4042
def initial_restriction(self, element):
4143
start, end, interval = element
42-
if isinstance(start, Timestamp):
43-
start_micros = start.micros
44-
else:
45-
start_micros = round(start * 1000000)
44+
if not isinstance(start, Timestamp):
45+
start = Timestamp.of(start)
4646

47-
if isinstance(end, Timestamp):
48-
end_micros = end.micros
49-
else:
50-
end_micros = round(end * 1000000)
47+
if not isinstance(end, Timestamp):
48+
end = Timestamp.of(end)
5149

52-
interval_micros = round(interval * 1000000)
50+
interval_duration = Duration(interval)
5351

54-
assert start_micros <= end_micros
52+
assert start <= end
5553
assert interval > 0
56-
delta_micros: int = end_micros - start_micros
57-
total_outputs = math.ceil(delta_micros / interval_micros)
54+
total_duration = end - start
55+
total_outputs = math.ceil(total_duration.micros / interval_duration.micros)
56+
5857
return OffsetRange(0, total_outputs)
5958

6059
def create_tracker(self, restriction):
@@ -230,38 +229,31 @@ def _validate_and_adjust_duration(self):
230229
assert self.data
231230

232231
# The total time we need to impulse all the data.
233-
data_duration = (len(self.data) - 1) * self.interval
232+
data_duration = (len(self.data) - 1) * Duration(self.interval)
234233

235234
is_pre_timestamped = isinstance(self.data[0], tuple) and \
236235
isinstance(self.data[0][0], timestamp.Timestamp)
237236

238-
if isinstance(self.start_ts, Timestamp):
239-
start = self.start_ts.micros / 1000000
240-
else:
241-
start = self.start_ts
242-
243-
if isinstance(self.stop_ts, Timestamp):
244-
if self.stop_ts == MAX_TIMESTAMP:
245-
# When the stop timestamp is unbounded (MAX_TIMESTAMP), set it to the
246-
# data's actual end time plus an extra fire interval, because the
247-
# impulse duration's upper bound is exclusive.
248-
end = start + data_duration + self.interval
249-
self.stop_ts = Timestamp(micros=end * 1000000)
250-
else:
251-
end = self.stop_ts.micros / 1000000
252-
else:
253-
end = self.stop_ts
237+
start_ts = Timestamp.of(self.start_ts)
238+
stop_ts = Timestamp.of(self.stop_ts)
239+
240+
if stop_ts == MAX_TIMESTAMP:
241+
# When the stop timestamp is unbounded (MAX_TIMESTAMP), set it to the
242+
# data's actual end time plus an extra fire interval, because the
243+
# impulse duration's upper bound is exclusive.
244+
self.stop_ts = start_ts + data_duration + Duration(self.interval)
245+
stop_ts = self.stop_ts
254246

255247
# The total time for the impulse signal which occurs in [start, end).
256-
impulse_duration = end - start
257-
if round(data_duration + self.interval, 6) < round(impulse_duration, 6):
248+
impulse_duration = stop_ts - start_ts
249+
if data_duration + Duration(self.interval) < impulse_duration:
258250
# We don't have enough data for the impulse.
259251
# If we can fit at least one more data point in the impulse duration,
260252
# then we will be in the repeat mode.
261253
message = 'The number of elements in the provided pre-timestamped ' \
262254
'data sequence is not enough to span the full impulse duration. ' \
263-
f'Expected duration: {impulse_duration:.6f}, ' \
264-
f'actual data duration: {data_duration:.6f}.'
255+
f'Expected duration: {impulse_duration}, ' \
256+
f'actual data duration: {data_duration}.'
265257

266258
if is_pre_timestamped:
267259
raise ValueError(
@@ -274,8 +266,8 @@ def _validate_and_adjust_duration(self):
274266

275267
def __init__(
276268
self,
277-
start_timestamp: Timestamp = Timestamp.now(),
278-
stop_timestamp: Timestamp = MAX_TIMESTAMP,
269+
start_timestamp: TimestampTypes = Timestamp.now(),
270+
stop_timestamp: TimestampTypes = MAX_TIMESTAMP,
279271
fire_interval: float = 360.0,
280272
apply_windowing: bool = False,
281273
data: Optional[Sequence[Any]] = None):
@@ -327,11 +319,11 @@ def expand(self, pbegin):
327319
| 'GenSequence' >> beam.ParDo(ImpulseSeqGenDoFn(self.data)))
328320

329321
if not self.data:
330-
# This step is only to ensure the current PTransform expansion is
331-
# compatible with the previous Beam versions.
332-
result = (
333-
result
334-
| 'MapToTimestamped' >> beam.Map(lambda tt: TimestampedValue(tt, tt)))
322+
# This step is actually an identity transform, because the Timestamped
323+
# values have already been generated in `ImpulseSeqGenDoFn`.
324+
# We keep this step here to prevent the current PeriodicImpulse from
325+
# breaking the compatibility.
326+
result = (result | 'MapToTimestamped' >> beam.Map(lambda tt: tt))
335327

336328
if self.apply_windowing:
337329
result = result | 'ApplyWindowing' >> beam.WindowInto(

sdks/python/apache_beam/transforms/periodicsequence_test.py

Lines changed: 65 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -261,22 +261,78 @@ def test_not_enough_timestamped_value(self):
261261
data=data,
262262
fire_interval=0.5))
263263

264-
def test_fuzzy_interval(self):
265-
seed = int(time.time() * 1000)
264+
def test_fuzzy_length_and_interval(self):
266265
times = 30
267-
logging.warning("random seed=%d", seed)
268-
random.seed(seed)
269266
for _ in range(times):
267+
seed = int(time.time() * 1000)
268+
random.seed(seed)
270269
n = int(random.randint(1, 100))
271270
data = list(range(n))
272271
m = random.randint(1, 1000)
273272
interval = m / 1e6
274273
now = Timestamp.now()
275-
with TestPipeline() as p:
276-
ret = (
277-
p | PeriodicImpulse(
278-
start_timestamp=now, data=data, fire_interval=interval))
279-
assert_that(ret, equal_to(data))
274+
try:
275+
with TestPipeline() as p:
276+
ret = (
277+
p | PeriodicImpulse(
278+
start_timestamp=now, data=data, fire_interval=interval))
279+
assert_that(ret, equal_to(data))
280+
except Exception as e: # pylint: disable=broad-except
281+
logging.error("Error occurred at random seed=%d", seed)
282+
raise e
283+
284+
def test_fuzzy_length_at_minimal_interval(self):
285+
times = 30
286+
for _ in range(times):
287+
seed = int(time.time() * 1000)
288+
seed = 1751135957975
289+
random.seed(seed)
290+
n = int(random.randint(1, 100))
291+
data = list(range(n))
292+
interval = 1e-6
293+
now = Timestamp.now()
294+
try:
295+
with TestPipeline() as p:
296+
ret = (
297+
p | PeriodicImpulse(
298+
start_timestamp=now, data=data, fire_interval=interval))
299+
assert_that(ret, equal_to(data))
300+
except Exception as e: # pylint: disable=broad-except
301+
logging.error("Error occurred at random seed=%d", seed)
302+
raise e
303+
304+
def test_int_type_input(self):
305+
# This test is to verify that if input timestamps and interval are integers,
306+
# the generated timestamped values are also integers.
307+
# This is necessary for the following test to pass:
308+
# apache_beam.examples.snippets.snippets_test.SlowlyChangingSideInputsTest
309+
with TestPipeline() as p:
310+
ret = (
311+
p | PeriodicImpulse(
312+
start_timestamp=1, stop_timestamp=5, fire_interval=1))
313+
expected = [1, 2, 3, 4]
314+
assert_that(
315+
ret, equal_to(expected, lambda x, y: type(x) is type(y) and x == y))
316+
317+
def test_float_type_input(self):
318+
with TestPipeline() as p:
319+
ret = (
320+
p | PeriodicImpulse(
321+
start_timestamp=1.0, stop_timestamp=5.0, fire_interval=1))
322+
expected = [1.0, 2.0, 3.0, 4.0]
323+
assert_that(
324+
ret, equal_to(expected, lambda x, y: type(x) is type(y) and x == y))
325+
326+
def test_timestamp_type_input(self):
327+
with TestPipeline() as p:
328+
ret = (
329+
p | PeriodicImpulse(
330+
start_timestamp=Timestamp.of(1),
331+
stop_timestamp=Timestamp.of(5),
332+
fire_interval=1))
333+
expected = [1.0, 2.0, 3.0, 4.0]
334+
assert_that(
335+
ret, equal_to(expected, lambda x, y: type(x) is type(y) and x == y))
280336

281337

282338
if __name__ == '__main__':

0 commit comments

Comments
 (0)