def test_equal_to_per_window_succeeds_no_reify_windows(self): start = int(MIN_TIMESTAMP.micros // 1e6) - 5 end = start + 20 expected = { window.IntervalWindow(start, end): [('k', [1])], } with TestPipeline(options=StandardOptions(streaming=True)) as p: assert_that( (p | Create([1]) | beam.WindowInto( FixedWindows(20), trigger=trigger.AfterWatermark(), accumulation_mode=trigger.AccumulationMode.DISCARDING) | beam.Map(lambda x: ('k', x)) | beam.GroupByKey()), equal_to_per_window(expected))
def test_equal_to_per_window_fail_unmatched_window(self): with self.assertRaises(BeamAssertException): expected = { window.IntervalWindow(50, 100): [('k', [1])], } with TestPipeline(options=StandardOptions(streaming=True)) as p: assert_that( (p | Create([1]) | beam.WindowInto( FixedWindows(20), trigger=trigger.AfterWatermark(), accumulation_mode=trigger.AccumulationMode.DISCARDING) | beam.Map(lambda x: ('k', x)) | beam.GroupByKey()), equal_to_per_window(expected), reify_windows=True)
def test_gbk_execution_after_processing_trigger_fired(self): """Advance TestClock to (X + delta) and see the pipeline does finish.""" # TODO(mariagh): Add test_gbk_execution_after_processing_trigger_unfired # Advance TestClock to (X + delta) and see the pipeline does finish # Possibly to the framework trigger_transcripts.yaml test_stream = (TestStream() .advance_watermark_to(10) .add_elements(['a']) .advance_processing_time(5.1)) options = PipelineOptions() options.view_as(StandardOptions).streaming = True p = TestPipeline(options=options) records = (p | test_stream | beam.WindowInto( beam.window.FixedWindows(15), trigger=trigger.AfterProcessingTime(5), accumulation_mode=trigger.AccumulationMode.DISCARDING ) | beam.Map(lambda x: ('k', x)) | beam.GroupByKey()) # TODO(BEAM-2519): timestamp assignment for elements from a GBK should # respect the TimestampCombiner. The test below should also verify the # timestamps of the outputted elements once this is implemented. assert_that(records, equal_to([ ('k', ['a'])])) # assert per window expected_window_to_elements = { window.IntervalWindow(15, 30): [('k', ['a'])], } assert_that( records, equal_to_per_window(expected_window_to_elements), custom_windowing=window.FixedWindows(15), label='assert per window') p.run()
def test_basic_execution_sideinputs(self): options = PipelineOptions() options.view_as(StandardOptions).streaming = True p = TestPipeline(options=options) main_stream = (p | 'main TestStream' >> TestStream() .advance_watermark_to(10) .add_elements(['e'])) side_stream = (p | 'side TestStream' >> TestStream() .add_elements([window.TimestampedValue(2, 2)]) .add_elements([window.TimestampedValue(1, 1)]) .add_elements([window.TimestampedValue(7, 7)]) .add_elements([window.TimestampedValue(4, 4)]) ) class RecordFn(beam.DoFn): def process(self, elm=beam.DoFn.ElementParam, ts=beam.DoFn.TimestampParam, side=beam.DoFn.SideInputParam): yield (elm, ts, side) records = (main_stream # pylint: disable=unused-variable | beam.ParDo(RecordFn(), beam.pvalue.AsList(side_stream))) # assert per window expected_window_to_elements = { window.IntervalWindow(0, 15): [ ('e', Timestamp(10), [2, 1, 7, 4]), ], } assert_that( records, equal_to_per_window(expected_window_to_elements), custom_windowing=window.FixedWindows(15), label='assert per window') assert_that(records, equal_to([('e', Timestamp(10), [2, 1, 7, 4])])) p.run()
def test_gbk_execution_after_watermark_trigger(self): test_stream = (TestStream() .advance_watermark_to(10) .add_elements(['a']) .advance_watermark_to(20) .advance_watermark_to_infinity()) options = PipelineOptions() options.view_as(StandardOptions).streaming = True p = TestPipeline(options=options) records = (p # pylint: disable=unused-variable | test_stream | beam.WindowInto( FixedWindows(15), trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)), accumulation_mode=trigger.AccumulationMode.DISCARDING) | beam.Map(lambda x: ('k', x)) | beam.GroupByKey()) # TODO(BEAM-2519): timestamp assignment for elements from a GBK should # respect the TimestampCombiner. The test below should also verify the # timestamps of the outputted elements once this is implemented. # assert per window expected_window_to_elements = { window.IntervalWindow(15, 30): [ ('k', ['a']), ('k', []), ], } assert_that( records, equal_to_per_window(expected_window_to_elements), use_global_window=False, label='assert per window') p.run()
def test_fixed_windows(self): # Test windows with offset: 2, 7, 12, 17, ... windowfn = window.FixedWindows(size=5, offset=2) self.assertEqual([window.IntervalWindow(7, 12)], windowfn.assign(context('v', 7, []))) self.assertEqual([window.IntervalWindow(7, 12)], windowfn.assign(context('v', 11, []))) self.assertEqual([window.IntervalWindow(12, 17)], windowfn.assign(context('v', 12, []))) # Test windows without offset: 0, 5, 10, 15, ... windowfn = window.FixedWindows(size=5) self.assertEqual([window.IntervalWindow(5, 10)], windowfn.assign(context('v', 5, []))) self.assertEqual([window.IntervalWindow(5, 10)], windowfn.assign(context('v', 9, []))) self.assertEqual([window.IntervalWindow(10, 15)], windowfn.assign(context('v', 10, []))) # Test windows with offset out of range. windowfn = window.FixedWindows(size=5, offset=12) self.assertEqual([window.IntervalWindow(7, 12)], windowfn.assign(context('v', 11, [])))
def test_multiple_outputs_with_watermark_advancement(self): """Tests that the TestStream can independently control output watermarks.""" # Purposely set the watermark of numbers to 20 then letters to 5 to test # that the watermark advancement is per PCollection. # # This creates two PCollections, (a, b, c) and (1, 2, 3). These will be # emitted at different times so that they will have different windows. The # watermark advancement is checked by checking their windows. If the # watermark does not advance, then the windows will be [-inf, -inf). If the # windows do not advance separately, then the PCollections will both # windowed in [15, 30). letters_elements = [ TimestampedValue('a', 6), TimestampedValue('b', 7), TimestampedValue('c', 8), ] numbers_elements = [ TimestampedValue('1', 21), TimestampedValue('2', 22), TimestampedValue('3', 23), ] test_stream = (TestStream().advance_watermark_to( 0, tag='letters').advance_watermark_to( 0, tag='numbers').advance_watermark_to( 20, tag='numbers').advance_watermark_to( 5, tag='letters').add_elements( letters_elements, tag='letters').advance_watermark_to( 10, tag='letters').add_elements( numbers_elements, tag='numbers').advance_watermark_to( 30, tag='numbers')) options = StandardOptions(streaming=True) p = TestPipeline(is_integration_test=True, options=options) main = p | test_stream # Use an AfterWatermark trigger with an early firing to test that the # watermark is advancing properly and that the element is being emitted in # the correct window. letters = ( main['letters'] | 'letter windows' >> beam.WindowInto( FixedWindows(15), trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)), accumulation_mode=trigger.AccumulationMode.DISCARDING) | 'letter with key' >> beam.Map(lambda x: ('k', x)) | 'letter gbk' >> beam.GroupByKey()) numbers = ( main['numbers'] | 'number windows' >> beam.WindowInto( FixedWindows(15), trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)), accumulation_mode=trigger.AccumulationMode.DISCARDING) | 'number with key' >> beam.Map(lambda x: ('k', x)) | 'number gbk' >> beam.GroupByKey()) # The letters were emitted when the watermark was at 5, thus we expect to # see the elements in the [0, 15) window. We used an early trigger to make # sure that the ON_TIME empty pane was also emitted with a TestStream. # This pane has no data because of the early trigger causes the elements to # fire before the end of the window and because the accumulation mode # discards any data after the trigger fired. expected_letters = { window.IntervalWindow(0, 15): [ ('k', ['a', 'b', 'c']), ('k', []), ], } # Same here, except the numbers were emitted at watermark = 20, thus they # are in the [15, 30) window. expected_numbers = { window.IntervalWindow(15, 30): [ ('k', ['1', '2', '3']), ('k', []), ], } assert_that(letters, equal_to_per_window(expected_letters), label='letters assert per window') assert_that(numbers, equal_to_per_window(expected_numbers), label='numbers assert per window') p.run()
def run(argv=None): """Build and run the pipeline.""" parser = argparse.ArgumentParser() parser.add_argument( '--output_topic', required=True, help=('Output PubSub topic of the form ' '"projects/<PROJECT>/topic/<TOPIC>".')) group = parser.add_mutually_exclusive_group(required=True) group.add_argument( '--input_topic', help=('Input PubSub topic of the form ' '"projects/<PROJECT>/topics/<TOPIC>".')) group.add_argument( '--input_subscription', help=('Input PubSub subscription of the form ' '"projects/<PROJECT>/subscriptions/<SUBSCRIPTION>."')) known_args, pipeline_args = parser.parse_known_args(argv) # We use the save_main_session option because one or more DoFn's in this # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True pipeline_options.view_as(StandardOptions).streaming = True with beam.Pipeline(options=pipeline_options) as p: # Read from PubSub into a PCollection. if known_args.input_subscription: lines = p | beam.io.ReadFromPubSub( subscription=known_args.input_subscription) else: lines = p | beam.io.ReadFromPubSub(topic=known_args.input_topic) # Count the occurrences of each word. def count_ones(word_ones): (word, ones) = word_ones return (word, sum(ones)) counts = (lines | 'AddTimestampFn' >> beam.ParDo(AddTimestampFn()) | 'After AddTimestampFn' >> ParDo(PrintFn('After AddTimestampFn')) | 'Split' >> (beam.ParDo(WordExtractingDoFn()) .with_output_types(unicode)) | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) | beam.WindowInto(window.FixedWindows(5, 0)) | 'GroupByKey' >> beam.GroupByKey() | 'CountOnes' >> beam.Map(count_ones)) # Format the counts into a PCollection of strings. def format_result(word_count): (word, count) = word_count return '%s: %d' % (word, count) output = counts | 'format' >> beam.Map(format_result) # Write to PubSub. # pylint: disable=expression-not-assigned output | beam.io.WriteStringsToPubSub(known_args.output_topic) def check_gbk_format(): # A matcher that checks that the output of GBK is of the form word: count. def matcher(elements): # pylint: disable=unused-variable actual_elements_in_window, window = elements for elm in actual_elements_in_window: assert re.match(r'\S+:\s+\d+', elm) is not None return matcher # Check that the format of the output is correct. assert_that( output, check_gbk_format(), use_global_window=False, label='Assert word:count format.') # Check also that elements are ouput in the right window. # This expects exactly 1 occurrence of any subset of the elements # 150, 151, 152, 153, 154 in the window [150, 155) # or exactly 1 occurrence of any subset of the elements # 210, 211, 212, 213, 214 in the window [210, 215). expected_window_to_elements = { window.IntervalWindow(150, 155): [ ('150: 1'), ('151: 1'), ('152: 1'), ('153: 1'), ('154: 1'), ], window.IntervalWindow(210, 215): [ ('210: 1'), ('211: 1'), ('212: 1'), ('213: 1'), ('214: 1'), ], } # To pass, publish numbers in [150-155) or [210-215) with no repeats. # To fail, publish a repeated number in the range above range. # For example: '210 213 151 213' assert_that( output, equal_to_per_window(expected_window_to_elements), use_global_window=False, label='Assert correct streaming windowing.')
def test_interval_window_coder(self): self.check_coder( coders.IntervalWindowCoder(), *[ window.IntervalWindow(x, y) for x in [-2**52, 0, 2**52] for y in range(-100, 100) ])
def test_output(): options = PipelineOptions() options.view_as(StandardOptions).streaming = True test_pipeline = TestPipeline(options=options) events = (test_pipeline | TestStream().add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()). add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()).advance_watermark_to( datetime( 2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()). add_elements(elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()). add_elements(elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()). advance_watermark_to( datetime( 2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()). add_elements(elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC). timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()). add_elements(elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()). advance_watermark_to( datetime( 2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()). add_elements(elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC). timestamp()).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()). add_elements(elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()). advance_watermark_to( datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp() ).add_elements( elements=["event"], event_timestamp=datetime( 2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()).advance_watermark_to( datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). timestamp()).advance_watermark_to_infinity()) results = apply_transform(events) answers = { window.IntervalWindow( datetime(2021, 3, 1, 0, 0, 0, 0, tzinfo=pytz.UTC).timestamp(), datetime(2021, 3, 2, 0, 0, 0, 0, tzinfo=pytz.UTC).timestamp()): [1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0], } assert_that(results, equal_to_per_window(answers), label='count assert per window') test_pipeline.run()
def assign(self, context): return [window.IntervalWindow(context.element[0], MAX_TIMESTAMP)]
def test_multi_triggered_gbk_side_input(self): """Test a GBK sideinput, with multiple triggering.""" # TODO(BEAM-9322): Remove use of this experiment. # This flag is only necessary when using the multi-output TestStream b/c # it relies on using the PCollection output tags as the PCollection output # ids. options = StandardOptions(streaming=True) options.view_as(DebugOptions).add_experiment( 'passthrough_pcollection_output_ids') p = TestPipeline(options=options) test_stream = ( p | 'Mixed TestStream' >> TestStream().advance_watermark_to( 3, tag='main').add_elements( ['a1'], tag='main').advance_watermark_to( 8, tag='main').add_elements(['a2'], tag='main'). add_elements([window.TimestampedValue( ('k', 100), 2)], tag='side').add_elements( [window.TimestampedValue(('k', 400), 7)], tag='side').advance_watermark_to_infinity( tag='main').advance_watermark_to_infinity(tag='side')) main_data = ( test_stream['main'] | 'Main windowInto' >> beam.WindowInto( window.FixedWindows(5), accumulation_mode=trigger.AccumulationMode.DISCARDING)) side_data = ( test_stream['side'] | 'Side windowInto' >> beam.WindowInto( window.FixedWindows(5), trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)), accumulation_mode=trigger.AccumulationMode.DISCARDING) | beam.CombinePerKey(sum) | 'Values' >> Map(lambda k_vs: k_vs[1])) class RecordFn(beam.DoFn): def process(self, elm=beam.DoFn.ElementParam, ts=beam.DoFn.TimestampParam, side=beam.DoFn.SideInputParam): yield (elm, ts, side) records = (main_data | beam.ParDo(RecordFn(), beam.pvalue.AsList(side_data))) expected_window_to_elements = { window.IntervalWindow(0, 5): [ ('a1', Timestamp(3), [100, 0]), ], window.IntervalWindow(5, 10): [('a2', Timestamp(8), [400, 0])], } assert_that(records, equal_to_per_window(expected_window_to_elements), use_global_window=False, label='assert per window') p.run()