def _setup_test_streaming_cache(): cache_manager = StreamingCache(cache_dir=None) ie.current_env().set_cache_manager(cache_manager) builder = FileRecordsBuilder(tag=_TEST_CACHE_KEY) (builder .advance_watermark(watermark_secs=0) .advance_processing_time(5) .add_element(element='a', event_time_secs=1) .advance_watermark(watermark_secs=100) .advance_processing_time(10)) # yapf: disable cache_manager.write(builder.build(), _TEST_CACHE_KEY)
def test_read_duration(self): """Test that the stream only reads a 'duration' of elements.""" values = (FileRecordsBuilder(tag=self.cache_key) .advance_processing_time(1) .add_element(element=0, event_time_secs=0) .advance_processing_time(1) .add_element(element=1, event_time_secs=1) .advance_processing_time(1) .add_element(element=2, event_time_secs=3) .advance_processing_time(1) .add_element(element=3, event_time_secs=4) .advance_processing_time(1) .add_element(element=4, event_time_secs=5) .build()) # yapf: disable self.mock_result.set_state(PipelineState.DONE) self.cache.write(values, 'full', self.cache_key) self.cache.save_pcoder(None, 'full', self.cache_key) # The elements read from the cache are TestStreamFileRecord instances and # have the underlying elements encoded. This method decodes the elements # from the TestStreamFileRecord. def get_elements(events): coder = coders.FastPrimitivesCoder() elements = [] for e in events: if not isinstance(e, TestStreamFileRecord): continue if e.recorded_event.element_event: elements += ([ coder.decode(el.encoded_element) for el in e.recorded_event.element_event.elements ]) return elements # The following tests a progression of reading different durations from the # cache. stream = ElementStream(self.pcoll, '', self.cache_key, max_n=100, max_duration_secs=1) self.assertSequenceEqual(get_elements(stream.read()), [0]) stream = ElementStream(self.pcoll, '', self.cache_key, max_n=100, max_duration_secs=2) self.assertSequenceEqual(get_elements(stream.read()), [0, 1]) stream = ElementStream(self.pcoll, '', self.cache_key, max_n=100, max_duration_secs=10) self.assertSequenceEqual(get_elements(stream.read()), [0, 1, 2, 3, 4])
def test_read_duration(self): """Test that the stream only reads a 'duration' of elements.""" def as_windowed_value(element): return WindowedValueHolder(WindowedValue(element, 0, [])) values = (FileRecordsBuilder(tag=self.cache_key) .advance_processing_time(1) .add_element(element=as_windowed_value(0), event_time_secs=0) .advance_processing_time(1) .add_element(element=as_windowed_value(1), event_time_secs=1) .advance_processing_time(1) .add_element(element=as_windowed_value(2), event_time_secs=3) .advance_processing_time(1) .add_element(element=as_windowed_value(3), event_time_secs=4) .advance_processing_time(1) .add_element(element=as_windowed_value(4), event_time_secs=5) .build()) # yapf: disable values = [ v.recorded_event for v in values if isinstance(v, beam_interactive_api_pb2.TestStreamFileRecord) ] self.mock_result.set_state(PipelineState.DONE) self.cache.write(values, 'full', self.cache_key) self.cache.save_pcoder(coders.FastPrimitivesCoder(), 'full', self.cache_key) # The following tests a progression of reading different durations from the # cache. stream = ElementStream(self.pcoll, '', self.cache_key, max_n=100, max_duration_secs=1) self.assertSequenceEqual([e.value for e in stream.read()], [0]) stream = ElementStream(self.pcoll, '', self.cache_key, max_n=100, max_duration_secs=2) self.assertSequenceEqual([e.value for e in stream.read()], [0, 1]) stream = ElementStream(self.pcoll, '', self.cache_key, max_n=100, max_duration_secs=10) self.assertSequenceEqual([e.value for e in stream.read()], [0, 1, 2, 3, 4])
def test_single_reader_with_processing_time_limiter(self): """Tests that we expect to see all the correctly emitted TestStreamPayloads. """ CACHED_PCOLLECTION_KEY = repr(CacheKey('arbitrary_key', '', '', '')) values = (FileRecordsBuilder(tag=CACHED_PCOLLECTION_KEY) .advance_processing_time(1e-6) .add_element(element=0, event_time_secs=0) .advance_processing_time(1) .add_element(element=1, event_time_secs=1) .advance_processing_time(1) .add_element(element=2, event_time_secs=2) .advance_processing_time(1) .add_element(element=3, event_time_secs=2) .advance_processing_time(1) .add_element(element=4, event_time_secs=2) .build()) # yapf: disable cache = StreamingCache(cache_dir=None) cache.write(values, CACHED_PCOLLECTION_KEY) reader, _ = cache.read(CACHED_PCOLLECTION_KEY, limiters=[ProcessingTimeLimiter(2)]) coder = coders.FastPrimitivesCoder() events = list(reader) # Units here are in microseconds. # Expects that the elements are a slice of the original values where all # processing time is less than the duration. expected = [ TestStreamPayload.Event(processing_time_event=TestStreamPayload. Event.AdvanceProcessingTime( advance_duration=1)), TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(0), timestamp=0) ], tag=CACHED_PCOLLECTION_KEY)), 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=1 * 10**6) ], tag=CACHED_PCOLLECTION_KEY)), ] self.assertSequenceEqual(events, expected)
def test_single_reader(self): """Tests that we expect to see all the correctly emitted TestStreamPayloads. """ CACHED_PCOLLECTION_KEY = 'arbitrary_key' values = (FileRecordsBuilder(tag=CACHED_PCOLLECTION_KEY) .add_element(element=0, event_time_secs=0) .advance_processing_time(1) .add_element(element=1, event_time_secs=1) .advance_processing_time(1) .add_element(element=2, event_time_secs=2) .build()) # yapf: disable cache = StreamingCache(cache_dir=None) cache.write(values, CACHED_PCOLLECTION_KEY) reader, _ = cache.read(CACHED_PCOLLECTION_KEY) coder = coders.FastPrimitivesCoder() events = list(reader) # Units here are in microseconds. expected = [ TestStreamPayload.Event( element_event=TestStreamPayload.Event.AddElements( elements=[ TestStreamPayload.TimestampedElement( encoded_element=coder.encode(0), timestamp=0) ], tag=CACHED_PCOLLECTION_KEY)), 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=1 * 10**6) ], tag=CACHED_PCOLLECTION_KEY)), 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=2 * 10**6) ], tag=CACHED_PCOLLECTION_KEY)), ] self.assertSequenceEqual(events, expected)
def test_always_default_coder_for_test_stream_records(self): CACHED_NUMBERS = repr(CacheKey('numbers', '', '', '')) 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 cache = StreamingCache(cache_dir=None) cache.write(numbers, CACHED_NUMBERS) self.assertIs(type(cache.load_pcoder(CACHED_NUMBERS)), type(cache._default_pcoder))
def test_streaming_cache_uses_local_ib_cache_root(self): """ Checks that StreamingCache._cache_dir is set to the cache_root set under Interactive Beam for a local directory and that the cached values are the same as the values of a cache using default settings. """ CACHED_PCOLLECTION_KEY = repr(CacheKey('arbitrary_key', '', '', '')) values = (FileRecordsBuilder(CACHED_PCOLLECTION_KEY) .advance_processing_time(1) .advance_watermark(watermark_secs=0) .add_element(element=1, event_time_secs=0) .build()) # yapf: disable local_cache = StreamingCache(cache_dir=None) local_cache.write(values, CACHED_PCOLLECTION_KEY) reader_one, _ = local_cache.read(CACHED_PCOLLECTION_KEY) pcoll_list_one = list(reader_one) # Set Interactive Beam specified cache dir to cloud storage ib.options.cache_root = '/tmp/it-test/' cache_manager_with_ib_option = StreamingCache( cache_dir=ib.options.cache_root) self.assertEqual(ib.options.cache_root, cache_manager_with_ib_option._cache_dir) cache_manager_with_ib_option.write(values, CACHED_PCOLLECTION_KEY) reader_two, _ = cache_manager_with_ib_option.read( CACHED_PCOLLECTION_KEY) pcoll_list_two = list(reader_two) self.assertEqual(pcoll_list_one, pcoll_list_two) # Reset Interactive Beam setting ib.options.cache_root = None
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)