def test_in_streaming_mode(self): timestamp_interval = 1 offset = itertools.count(0) start_time = timestamp.Timestamp(0) window_duration = 6 test_stream = ( TestStream().advance_watermark_to(start_time).add_elements([ TimestampedValue(x, next(offset) * timestamp_interval) for x in GroupIntoBatchesTest._create_test_data() ]).advance_watermark_to(start_time + (window_duration - 1)). advance_watermark_to(start_time + (window_duration + 1)). advance_watermark_to(start_time + GroupIntoBatchesTest.NUM_ELEMENTS ).advance_watermark_to_infinity()) pipeline = TestPipeline(options=StandardOptions(streaming=True)) # window duration is 6 and batch size is 5, so output batch size should be # 5 (flush because of batchSize reached) expected_0 = 5 # there is only one element left in the window so batch size should be 1 # (flush because of end of window reached) expected_1 = 1 # collection is 10 elements, there is only 4 left, so batch size should be # 4 (flush because end of collection reached) expected_2 = 4 collection = pipeline | test_stream \ | WindowInto(FixedWindows(window_duration)) \ | util.GroupIntoBatches(GroupIntoBatchesTest.BATCH_SIZE) num_elements_in_batches = collection | beam.Map(len) result = pipeline.run() result.wait_until_finish() assert_that(num_elements_in_batches, equal_to([expected_0, expected_1, expected_2]))
def test_in_global_window(self): pipeline = TestPipeline() collection = pipeline \ | beam.Create(GroupIntoBatchesTest._create_test_data()) \ | util.GroupIntoBatches(GroupIntoBatchesTest.BATCH_SIZE) num_batches = collection | beam.combiners.Count.Globally() assert_that(num_batches, equal_to([int(math.ceil(GroupIntoBatchesTest.NUM_ELEMENTS / GroupIntoBatchesTest.BATCH_SIZE))])) pipeline.run()
def test_runner_api(self): batch_size = 10 max_buffering_duration_secs = [None, 0, 5] for duration in max_buffering_duration_secs: self._test_runner_api_round_trip( util.GroupIntoBatches(batch_size, duration), common_urns.group_into_batches_components.GROUP_INTO_BATCHES.urn) self._test_runner_api_round_trip( util.GroupIntoBatches(batch_size), common_urns.group_into_batches_components.GROUP_INTO_BATCHES.urn) for duration in max_buffering_duration_secs: self._test_runner_api_round_trip( util.GroupIntoBatches.WithShardedKey(batch_size, duration), common_urns.composites.GROUP_INTO_BATCHES_WITH_SHARDED_KEY.urn) self._test_runner_api_round_trip( util.GroupIntoBatches.WithShardedKey(batch_size), common_urns.composites.GROUP_INTO_BATCHES_WITH_SHARDED_KEY.urn)
def test_buffering_timer_in_fixed_window_streaming(self): window_duration = 6 max_buffering_duration_secs = 100 start_time = timestamp.Timestamp(0) test_stream = ( TestStream().add_elements([ TimestampedValue(value, start_time + i) for i, value in enumerate(GroupIntoBatchesTest._create_test_data()) ]).advance_processing_time(150).advance_watermark_to( start_time + window_duration).advance_watermark_to( start_time + window_duration + 1).advance_watermark_to_infinity()) with TestPipeline(options=StandardOptions(streaming=True)) as pipeline: # To trigger the processing time timer, use a fake clock with start time # being Timestamp(0). fake_clock = FakeClock(now=start_time) num_elements_per_batch = ( pipeline | test_stream | "fixed window" >> WindowInto(FixedWindows(window_duration)) | util.GroupIntoBatches( GroupIntoBatchesTest.BATCH_SIZE, max_buffering_duration_secs, fake_clock) | "count elements in batch" >> Map(lambda x: (None, len(x[1]))) | "global window" >> WindowInto(GlobalWindows()) | GroupByKey() | FlatMapTuple(lambda k, vs: vs)) # Window duration is 6 and batch size is 5, so output batch size # should be 5 (flush because of batch size reached). expected_0 = 5 # There is only one element left in the window so batch size # should be 1 (flush because of max buffering duration reached). expected_1 = 1 # Collection has 10 elements, there are only 4 left, so batch size should # be 4 (flush because of end of window reached). expected_2 = 4 assert_that( num_elements_per_batch, equal_to([expected_0, expected_1, expected_2]), "assert2")
def test_buffering_timer_in_global_window_streaming(self): max_buffering_duration_secs = 42 start_time = timestamp.Timestamp(0) test_stream = TestStream().advance_watermark_to(start_time) for i, value in enumerate(GroupIntoBatchesTest._create_test_data()): test_stream.add_elements( [TimestampedValue(value, start_time + i)]) \ .advance_processing_time(5) test_stream.advance_watermark_to( start_time + GroupIntoBatchesTest.NUM_ELEMENTS + 1) \ .advance_watermark_to_infinity() with TestPipeline(options=StandardOptions(streaming=True)) as pipeline: # Set a batch size larger than the total number of elements. # Since we're in a global window, we would have been waiting # for all the elements to arrive without the buffering time limit. batch_size = GroupIntoBatchesTest.NUM_ELEMENTS * 2 # To trigger the processing time timer, use a fake clock with start time # being Timestamp(0). Since the fake clock never really advances during # the pipeline execution, meaning that the timer is always set to the same # value, the timer will be fired on every element after the first firing. fake_clock = FakeClock(now=start_time) num_elements_per_batch = ( pipeline | test_stream | WindowInto( GlobalWindows(), trigger=Repeatedly(AfterCount(1)), accumulation_mode=trigger.AccumulationMode.DISCARDING) | util.GroupIntoBatches( batch_size, max_buffering_duration_secs, fake_clock) | 'count elements in batch' >> Map(lambda x: (None, len(x[1]))) | GroupByKey() | FlatMapTuple(lambda k, vs: vs)) # We will flush twice when the max buffering duration is reached and when # the global window ends. assert_that(num_elements_per_batch, equal_to([9, 1]))