def test_instrument_example_unbounded_pipeline_to_multiple_read_cache( self): """Tests that the instrumenter works for multiple unbounded sources. """ # Create a new interactive environment to make the test idempotent. ie.new_env(cache_manager=StreamingCache(cache_dir=None)) # Create the pipeline that will be instrumented. p_original = beam.Pipeline(interactive_runner.InteractiveRunner()) source_1 = p_original | 'source1' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') source_2 = p_original | 'source2' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') # pylint: disable=possibly-unused-variable pcoll_1 = source_1 | 'square1' >> beam.Map(lambda x: x * x) # pylint: disable=possibly-unused-variable pcoll_2 = source_2 | 'square2' >> beam.Map(lambda x: x * x) # Mock as if cacheable PCollections are cached. ib.watch(locals()) for name, pcoll in locals().items(): if not isinstance(pcoll, beam.pvalue.PCollection): continue cache_key = self.cache_key_of(name, pcoll) self._mock_write_cache([b''], cache_key) # Instrument the original pipeline to create the pipeline the user will see. instrumenter = instr.build_pipeline_instrument(p_original) actual_pipeline = beam.Pipeline.from_runner_api( proto=instrumenter.instrumented_pipeline_proto(), runner=interactive_runner.InteractiveRunner(), options=None) # Now, build the expected pipeline which replaces the unbounded source with # a TestStream. source_1_cache_key = self.cache_key_of('source_1', source_1) source_2_cache_key = self.cache_key_of('source_2', source_2) p_expected = beam.Pipeline() test_stream = (p_expected | TestStream(output_tags=[ self.cache_key_of('source_1', source_1), self.cache_key_of('source_2', source_2) ])) # pylint: disable=expression-not-assigned test_stream[source_1_cache_key] | 'square1' >> beam.Map( lambda x: x * x) # pylint: disable=expression-not-assigned test_stream[source_2_cache_key] | 'square2' >> beam.Map( lambda x: x * x) # Test that the TestStream is outputting to the correct PCollection. class TestStreamVisitor(PipelineVisitor): def __init__(self): self.output_tags = set() def enter_composite_transform(self, transform_node): self.visit_transform(transform_node) def visit_transform(self, transform_node): transform = transform_node.transform if isinstance(transform, TestStream): self.output_tags = transform.output_tags v = TestStreamVisitor() actual_pipeline.visit(v) expected_output_tags = set([source_1_cache_key, source_2_cache_key]) actual_output_tags = v.output_tags self.assertSetEqual(expected_output_tags, actual_output_tags) # Test that the pipeline is as expected. assert_pipeline_proto_equal(self, p_expected.to_runner_api(), instrumenter.instrumented_pipeline_proto())
def test_able_to_cache_intermediate_unbounded_source_pcollection(self): """Tests being able to cache an intermediate source PCollection. In the following pipeline, the source doesn't have a reference and so is not automatically cached in the watch() command. This tests that this case is taken care of. """ # Create a new interactive environment to make the test idempotent. ie.new_env(cache_manager=StreamingCache(cache_dir=None)) # Create the pipeline that will be instrumented. from apache_beam.options.pipeline_options import StandardOptions options = StandardOptions(streaming=True) p_original = beam.Pipeline(interactive_runner.InteractiveRunner(), options) # pylint: disable=possibly-unused-variable source_1 = ( p_original | 'source1' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') | beam.Map(lambda e: e)) # Watch but do not cache the PCollections. ib.watch(locals()) # Make sure that sources without a user reference are still cached. instr.watch_sources(p_original) intermediate_source_pcoll = None for watching in ie.current_env().watching(): watching = list(watching) for var, watchable in watching: if 'synthetic' in var: intermediate_source_pcoll = watchable break # Instrument the original pipeline to create the pipeline the user will see. p_copy = beam.Pipeline.from_runner_api( p_original.to_runner_api(), runner=interactive_runner.InteractiveRunner(), options=options) instrumenter = instr.build_pipeline_instrument(p_copy) actual_pipeline = beam.Pipeline.from_runner_api( proto=instrumenter.instrumented_pipeline_proto(), runner=interactive_runner.InteractiveRunner(), options=options) # Now, build the expected pipeline which replaces the unbounded source with # a TestStream. intermediate_source_pcoll_cache_key = \ self.cache_key_of('synthetic_var_' + str(id(intermediate_source_pcoll)), intermediate_source_pcoll) p_expected = beam.Pipeline() test_stream = ( p_expected | TestStream(output_tags=[intermediate_source_pcoll_cache_key])) # pylint: disable=expression-not-assigned (test_stream[intermediate_source_pcoll_cache_key] | 'square1' >> beam.Map(lambda e: e) | 'reify' >> beam.Map(lambda _: _) | cache.WriteCache(ie.current_env().cache_manager(), 'unused')) # Test that the TestStream is outputting to the correct PCollection. class TestStreamVisitor(PipelineVisitor): def __init__(self): self.output_tags = set() def enter_composite_transform(self, transform_node): self.visit_transform(transform_node) def visit_transform(self, transform_node): transform = transform_node.transform if isinstance(transform, TestStream): self.output_tags = transform.output_tags v = TestStreamVisitor() actual_pipeline.visit(v) expected_output_tags = set([intermediate_source_pcoll_cache_key]) actual_output_tags = v.output_tags self.assertSetEqual(expected_output_tags, actual_output_tags) # Test that the pipeline is as expected. assert_pipeline_proto_equal( self, p_expected.to_runner_api(use_fake_coders=True), instrumenter.instrumented_pipeline_proto())
def test_instrument_example_unbounded_pipeline_to_read_cache_not_cached( self): """Tests that the instrumenter works when the PCollection is not cached. """ # Create a new interactive environment to make the test idempotent. ie.new_env(cache_manager=StreamingCache(cache_dir=None)) # Create the pipeline that will be instrumented. from apache_beam.options.pipeline_options import StandardOptions options = StandardOptions(streaming=True) p_original = beam.Pipeline(interactive_runner.InteractiveRunner(), options) source_1 = p_original | 'source1' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') # pylint: disable=possibly-unused-variable pcoll_1 = source_1 | 'square1' >> beam.Map(lambda x: x * x) # Watch but do not cache the PCollections. ib.watch(locals()) # Instrument the original pipeline to create the pipeline the user will see. p_copy = beam.Pipeline.from_runner_api( p_original.to_runner_api(), runner=interactive_runner.InteractiveRunner(), options=options) instrumenter = instr.build_pipeline_instrument(p_copy) actual_pipeline = beam.Pipeline.from_runner_api( proto=instrumenter.instrumented_pipeline_proto(), runner=interactive_runner.InteractiveRunner(), options=options) # Now, build the expected pipeline which replaces the unbounded source with # a TestStream. source_1_cache_key = self.cache_key_of('source_1', source_1) p_expected = beam.Pipeline() test_stream = (p_expected | TestStream(output_tags=[source_1_cache_key])) # pylint: disable=expression-not-assigned (test_stream[source_1_cache_key] | 'square1' >> beam.Map(lambda x: x * x) | 'reify' >> beam.Map(lambda _: _) | cache.WriteCache(ie.current_env().cache_manager(), 'unused')) # Test that the TestStream is outputting to the correct PCollection. class TestStreamVisitor(PipelineVisitor): def __init__(self): self.output_tags = set() def enter_composite_transform(self, transform_node): self.visit_transform(transform_node) def visit_transform(self, transform_node): transform = transform_node.transform if isinstance(transform, TestStream): self.output_tags = transform.output_tags v = TestStreamVisitor() actual_pipeline.visit(v) expected_output_tags = set([source_1_cache_key]) actual_output_tags = v.output_tags self.assertSetEqual(expected_output_tags, actual_output_tags) # Test that the pipeline is as expected. assert_pipeline_proto_equal( self, p_expected.to_runner_api(use_fake_coders=True), instrumenter.instrumented_pipeline_proto())
def test_streaming_cache_does_not_write_non_record_or_header_types(self): cache = StreamingCache(cache_dir=None) self.assertRaises(TypeError, cache.write, 'some value', 'a key')
def test_instrument_mixed_streaming_batch(self): """Tests caching for both batch and streaming sources in the same pipeline. This ensures that cached bounded and unbounded sources are read from the TestStream. """ # Create the pipeline that will be instrumented. from apache_beam.options.pipeline_options import StandardOptions options = StandardOptions(streaming=True) p_original = beam.Pipeline(interactive_runner.InteractiveRunner(), options) streaming_cache_manager = StreamingCache(cache_dir=None) ie.current_env().set_cache_manager(streaming_cache_manager, p_original) source_1 = p_original | 'source1' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') source_2 = p_original | 'source2' >> beam.Create([1, 2, 3, 4, 5]) # pylint: disable=possibly-unused-variable pcoll_1 = ((source_1, source_2) | beam.Flatten() | 'square1' >> beam.Map(lambda x: x * x)) # Watch but do not cache the PCollections. ib.watch(locals()) self._mock_write_cache(p_original, [b''], self.cache_key_of('source_2', source_2)) ie.current_env().mark_pcollection_computed([source_2]) # Instrument the original pipeline to create the pipeline the user will see. p_copy = beam.Pipeline.from_runner_api( p_original.to_runner_api(), runner=interactive_runner.InteractiveRunner(), options=options) instrumenter = instr.build_pipeline_instrument(p_copy) actual_pipeline = beam.Pipeline.from_runner_api( proto=instrumenter.instrumented_pipeline_proto(), runner=interactive_runner.InteractiveRunner(), options=options) # Now, build the expected pipeline which replaces the unbounded source with # a TestStream. source_1_cache_key = self.cache_key_of('source_1', source_1) source_2_cache_key = self.cache_key_of('source_2', source_2) p_expected = beam.Pipeline() ie.current_env().set_cache_manager(streaming_cache_manager, p_expected) test_stream = ( p_expected | TestStream(output_tags=[source_1_cache_key, source_2_cache_key])) # pylint: disable=expression-not-assigned ((test_stream[self.cache_key_of('source_1', source_1)], test_stream[self.cache_key_of('source_2', source_2)]) | beam.Flatten() | 'square1' >> beam.Map(lambda x: x * x) | 'reify' >> beam.Map(lambda _: _) | cache.WriteCache(ie.current_env().get_cache_manager(p_expected), 'unused')) # Test that the TestStream is outputting to the correct PCollection. class TestStreamVisitor(PipelineVisitor): def __init__(self): self.output_tags = set() def enter_composite_transform(self, transform_node): self.visit_transform(transform_node) def visit_transform(self, transform_node): transform = transform_node.transform if isinstance(transform, TestStream): self.output_tags = transform.output_tags v = TestStreamVisitor() actual_pipeline.visit(v) expected_output_tags = set([source_1_cache_key, source_2_cache_key]) actual_output_tags = v.output_tags self.assertSetEqual(expected_output_tags, actual_output_tags) # Test that the pipeline is as expected. assert_pipeline_proto_equal( self, p_expected.to_runner_api(use_fake_coders=True), instrumenter.instrumented_pipeline_proto())
def test_read_and_write(self): """An integration test between the Sink and Source. This ensures that the sink and source speak the same language in terms of coders, protos, order, and units. """ CACHED_RECORDS = repr(CacheKey('records', '', '', '')) # Units here are in seconds. test_stream = ( TestStream(output_tags=(CACHED_RECORDS)) .advance_watermark_to(0, tag=CACHED_RECORDS) .advance_processing_time(5) .add_elements(['a', 'b', 'c'], tag=CACHED_RECORDS) .advance_watermark_to(10, tag=CACHED_RECORDS) .advance_processing_time(1) .add_elements( [ TimestampedValue('1', 15), TimestampedValue('2', 15), TimestampedValue('3', 15) ], tag=CACHED_RECORDS)) # yapf: disable coder = SafeFastPrimitivesCoder() cache = StreamingCache(cache_dir=None, sample_resolution_sec=1.0) # Assert that there are no capture keys at first. self.assertEqual(cache.capture_keys, set()) options = StandardOptions(streaming=True) with TestPipeline(options=options) as p: records = (p | test_stream)[CACHED_RECORDS] # pylint: disable=expression-not-assigned records | cache.sink([CACHED_RECORDS], is_capture=True) reader, _ = cache.read(CACHED_RECORDS) actual_events = list(reader) # Assert that the capture keys are forwarded correctly. self.assertEqual(cache.capture_keys, set([CACHED_RECORDS])) # Units here are in microseconds. expected_events = [ TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=5 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=0, tag=CACHED_RECORDS)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('a'), timestamp=0), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('b'), timestamp=0), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('c'), timestamp=0), ], tag=CACHED_RECORDS)), TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=10 * 10**6, tag=CACHED_RECORDS)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('1'), timestamp=15 * 10**6), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('2'), timestamp=15 * 10**6), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('3'), timestamp=15 * 10**6), ], tag=CACHED_RECORDS)), ] self.assertEqual(actual_events, expected_events)
def test_read_and_write_multiple_outputs(self): """An integration test between the Sink and Source with multiple outputs. This tests the funcionatlity that the StreamingCache reads from multiple files and combines them into a single sorted output. """ LETTERS_TAG = repr(CacheKey('letters', '', '', '')) NUMBERS_TAG = repr(CacheKey('numbers', '', '', '')) # Units here are in seconds. test_stream = (TestStream() .advance_watermark_to(0, tag=LETTERS_TAG) .advance_processing_time(5) .add_elements(['a', 'b', 'c'], tag=LETTERS_TAG) .advance_watermark_to(10, tag=NUMBERS_TAG) .advance_processing_time(1) .add_elements( [ TimestampedValue('1', 15), TimestampedValue('2', 15), TimestampedValue('3', 15) ], tag=NUMBERS_TAG)) # yapf: disable cache = StreamingCache(cache_dir=None, sample_resolution_sec=1.0) coder = SafeFastPrimitivesCoder() options = StandardOptions(streaming=True) with TestPipeline(options=options) as p: # pylint: disable=expression-not-assigned events = p | test_stream events[LETTERS_TAG] | 'Letters sink' >> cache.sink([LETTERS_TAG]) events[NUMBERS_TAG] | 'Numbers sink' >> cache.sink([NUMBERS_TAG]) reader = cache.read_multiple([[LETTERS_TAG], [NUMBERS_TAG]]) actual_events = list(reader) # Units here are in microseconds. expected_events = [ TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=5 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=0, tag=LETTERS_TAG)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('a'), timestamp=0), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('b'), timestamp=0), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('c'), timestamp=0), ], tag=LETTERS_TAG)), TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=10 * 10**6, tag=NUMBERS_TAG)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=0, tag=LETTERS_TAG)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('1'), timestamp=15 * 10**6), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('2'), timestamp=15 * 10**6), TestStreamPayload.TimestampedElement( encoded_element=coder.encode('3'), timestamp=15 * 10**6), ], tag=NUMBERS_TAG)), ] self.assertListEqual(actual_events, expected_events)
def test_multiple_readers(self): """Tests that the service advances the clock with multiple outputs. """ CACHED_LETTERS = repr(CacheKey('letters', '', '', '')) CACHED_NUMBERS = repr(CacheKey('numbers', '', '', '')) CACHED_LATE = repr(CacheKey('late', '', '', '')) letters = (FileRecordsBuilder(CACHED_LETTERS) .advance_processing_time(1) .advance_watermark(watermark_secs=0) .add_element(element='a', event_time_secs=0) .advance_processing_time(10) .advance_watermark(watermark_secs=10) .add_element(element='b', event_time_secs=10) .build()) # yapf: disable numbers = (FileRecordsBuilder(CACHED_NUMBERS) .advance_processing_time(2) .add_element(element=1, event_time_secs=0) .advance_processing_time(1) .add_element(element=2, event_time_secs=0) .advance_processing_time(1) .add_element(element=2, event_time_secs=0) .build()) # yapf: disable late = (FileRecordsBuilder(CACHED_LATE) .advance_processing_time(101) .add_element(element='late', event_time_secs=0) .build()) # yapf: disable cache = StreamingCache(cache_dir=None) cache.write(letters, CACHED_LETTERS) cache.write(numbers, CACHED_NUMBERS) cache.write(late, CACHED_LATE) reader = cache.read_multiple([[CACHED_LETTERS], [CACHED_NUMBERS], [CACHED_LATE]]) coder = coders.FastPrimitivesCoder() events = list(reader) # Units here are in microseconds. expected = [ # Advances clock from 0 to 1 TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=0, tag=CACHED_LETTERS)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('a'), timestamp=0) ], tag=CACHED_LETTERS)), # Advances clock from 1 to 2 TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(1), timestamp=0) ], tag=CACHED_NUMBERS)), # Advances clock from 2 to 3 TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(2), timestamp=0) ], tag=CACHED_NUMBERS)), # Advances clock from 3 to 4 TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(2), timestamp=0) ], tag=CACHED_NUMBERS)), # Advances clock from 4 to 11 TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=7 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=10 * 10**6, tag=CACHED_LETTERS)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('b'), timestamp=10 * 10**6) ], tag=CACHED_LETTERS)), # Advances clock from 11 to 101 TestStreamPayload.Event( processing_time_event=TestStreamPayload.Event.AdvanceProcessingTime( advance_duration=90 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('late'), timestamp=0) ], tag=CACHED_LATE)), ] self.assertSequenceEqual(events, expected)
def test_capture_size_limit_not_reached_when_no_file(self): cache = StreamingCache(cache_dir=None) self.assertFalse(cache.exists('my_label')) ie.current_env().set_cache_manager(cache) self.assertFalse(ie.current_env().options.capture_control. is_capture_size_limit_reached())
def test_instrument_example_unbounded_pipeline_direct_from_source(self): """Tests that the it caches PCollections from a source. """ # Create the pipeline that will be instrumented. from apache_beam.options.pipeline_options import StandardOptions options = StandardOptions(streaming=True) p_original = beam.Pipeline(interactive_runner.InteractiveRunner(), options) ie.current_env().set_cache_manager(StreamingCache(cache_dir=None), p_original) source_1 = p_original | 'source1' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') # pylint: disable=possibly-unused-variable # Watch but do not cache the PCollections. ib.watch(locals()) # Instrument the original pipeline to create the pipeline the user will see. p_copy = beam.Pipeline.from_runner_api( p_original.to_runner_api(), runner=interactive_runner.InteractiveRunner(), options=options) instrumenter = instr.build_pipeline_instrument(p_copy) actual_pipeline = beam.Pipeline.from_runner_api( proto=instrumenter.instrumented_pipeline_proto(), runner=interactive_runner.InteractiveRunner(), options=options) # Now, build the expected pipeline which replaces the unbounded source with # a TestStream. source_1_cache_key = self.cache_key_of('source_1', source_1) p_expected = beam.Pipeline() # pylint: disable=unused-variable test_stream = ( p_expected | TestStream(output_tags=[self.cache_key_of('source_1', source_1)])) # Test that the TestStream is outputting to the correct PCollection. class TestStreamVisitor(PipelineVisitor): def __init__(self): self.output_tags = set() def enter_composite_transform(self, transform_node): self.visit_transform(transform_node) def visit_transform(self, transform_node): transform = transform_node.transform if isinstance(transform, TestStream): self.output_tags = transform.output_tags v = TestStreamVisitor() actual_pipeline.visit(v) expected_output_tags = set([source_1_cache_key]) actual_output_tags = v.output_tags self.assertSetEqual(expected_output_tags, actual_output_tags) # Test that the pipeline is as expected. assert_pipeline_proto_equal(self, p_expected.to_runner_api(), instrumenter.instrumented_pipeline_proto())
def test_exists(self): cache = StreamingCache(cache_dir=None) self.assertFalse(cache.exists('my_label')) cache.write([TestStreamFileRecord()], 'my_label') self.assertTrue(cache.exists('my_label'))
def test_multiple_readers(self): """Tests that the service advances the clock with multiple outputs.""" letters = InMemoryReader('letters') letters.advance_processing_time(1) letters.advance_watermark(0) letters.add_element(element='a', event_time=0) letters.advance_processing_time(10) letters.advance_watermark(10) letters.add_element(element='b', event_time=10) numbers = InMemoryReader('numbers') numbers.advance_processing_time(2) numbers.add_element(element=1, event_time=0) numbers.advance_processing_time(1) numbers.add_element(element=2, event_time=0) numbers.advance_processing_time(1) numbers.add_element(element=2, event_time=0) late = InMemoryReader('late') late.advance_processing_time(101) late.add_element(element='late', event_time=0) cache = StreamingCache([letters, numbers, late]) reader = cache.reader() coder = coders.FastPrimitivesCoder() events = all_events(reader) expected = [ # Advances clock from 0 to 1 TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=0, tag='letters')), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('a'), timestamp=0) ], tag='letters')), # Advances clock from 1 to 2 TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(1), timestamp=0) ], tag='numbers')), # Advances clock from 2 to 3 TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(2), timestamp=0) ], tag='numbers')), # Advances clock from 3 to 4 TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=1 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(2), timestamp=0) ], tag='numbers')), # Advances clock from 4 to 11 TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=7 * 10**6)), TestStreamPayload.Event( watermark_event=TestStreamPayload.Event.AdvanceWatermark( new_watermark=10 * 10**6, tag='letters')), TestStreamPayload.Event(element_event=TestStreamPayload.Event. AddElements(elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('b'), timestamp=10 * 10**6) ], tag='letters')), # Advances clock from 11 to 101 TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=90 * 10**6)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements(elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode('late'), timestamp=0) ], tag='late')), ] self.assertSequenceEqual(events, expected)