def has_source_to_cache(user_pipeline): """Determines if a user-defined pipeline contains any source that need to be cached. If so, also immediately wrap current cache manager held by current interactive environment into a streaming cache if this has not been done. The wrapping doesn't invalidate existing cache in any way. This can help determining if a background source recording job is needed to write cache for sources and if a test stream service is needed to serve the cache. Throughout the check, if source-to-cache has changed from the last check, it also cleans up the invalidated cache early on. """ # TODO(BEAM-8335): we temporarily only cache replaceable unbounded sources. # Add logic for other cacheable sources here when they are available. has_cache = utils.has_unbounded_sources(user_pipeline) if has_cache: if not isinstance( ie.current_env().get_cache_manager(user_pipeline, create_if_absent=True), streaming_cache.StreamingCache): file_based_cm = ie.current_env().get_cache_manager(user_pipeline) ie.current_env().set_cache_manager( streaming_cache.StreamingCache( file_based_cm._cache_dir, is_cache_complete=is_cache_complete, sample_resolution_sec=1.0, saved_pcoders=file_based_cm._saved_pcoders), user_pipeline) return has_cache
def has_source_to_cache(user_pipeline): """Determines if a user-defined pipeline contains any source that need to be cached. If so, also immediately wrap current cache manager held by current interactive environment into a streaming cache if this has not been done. The wrapping doesn't invalidate existing cache in any way. This can help determining if a background caching job is needed to write cache for sources and if a test stream service is needed to serve the cache. Throughout the check, if source-to-cache has changed from the last check, it also cleans up the invalidated cache early on. """ from apache_beam.runners.interactive import pipeline_instrument as instr # TODO(BEAM-8335): we temporarily only cache replaceable unbounded sources. # Add logic for other cacheable sources here when they are available. has_cache = instr.has_unbounded_sources(user_pipeline) if has_cache: if not isinstance(ie.current_env().cache_manager(), streaming_cache.StreamingCache): # Wrap the cache manager into a streaming cache manager. Note this # does not invalidate the current cache manager. def is_cache_complete(): job = ie.current_env().get_background_caching_job(user_pipeline) is_done = job and job.is_done() cache_changed = is_source_to_cache_changed( user_pipeline, update_cached_source_signature=False) return is_done and not cache_changed ie.current_env().set_cache_manager( streaming_cache.StreamingCache( ie.current_env().cache_manager()._cache_dir, is_cache_complete=is_cache_complete, sample_resolution_sec=1.0)) return has_cache
def test_background_caching_pipeline_proto(self): ie.new_env(cache_manager=streaming_cache.StreamingCache(cache_dir=None)) p = beam.Pipeline(interactive_runner.InteractiveRunner()) # Test that the two ReadFromPubSub are correctly cut out. a = p | 'ReadUnboundedSourceA' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') b = p | 'ReadUnboundedSourceB' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') # Add some extra PTransform afterwards to make sure that only the unbounded # sources remain. c = (a, b) | beam.CoGroupByKey() _ = c | beam.Map(lambda x: x) ib.watch(locals()) instrumenter = instr.build_pipeline_instrument(p) actual_pipeline = instrumenter.background_caching_pipeline_proto() # Now recreate the expected pipeline, which should only have the unbounded # sources. p = beam.Pipeline(interactive_runner.InteractiveRunner()) a = p | 'ReadUnboundedSourceA' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') _ = ( a | 'reify a' >> beam.Map(lambda _: _) | 'a' >> cache.WriteCache(ie.current_env().cache_manager(), '')) b = p | 'ReadUnboundedSourceB' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') _ = ( b | 'reify b' >> beam.Map(lambda _: _) | 'b' >> cache.WriteCache(ie.current_env().cache_manager(), '')) expected_pipeline = p.to_runner_api( return_context=False, use_fake_coders=True) assert_pipeline_proto_equal(self, expected_pipeline, actual_pipeline)
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=streaming_cache.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()) def cache_key_of(name, pcoll): return name + '_' + str(id(pcoll)) + '_' + str(id(pcoll.producer)) for name, pcoll in locals().items(): if not isinstance(pcoll, beam.pvalue.PCollection): continue cache_key = cache_key_of(name, pcoll) self._mock_write_cache([TestStreamPayload()], 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 = cache_key_of('source_1', source_1) source_2_cache_key = cache_key_of('source_2', source_2) p_expected = beam.Pipeline() test_stream = (p_expected | TestStream(output_tags=[ cache_key_of('source_1', source_1), 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_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=streaming_cache.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()) def cache_key_of(name, pcoll): return name + '_' + str(id(pcoll)) + '_' + str(id(pcoll.producer)) # 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 = cache_key_of('source_1', source_1) p_expected = beam.Pipeline() test_stream = ( p_expected | TestStream(output_tags=[cache_key_of('source_1', source_1)])) # pylint: disable=expression-not-assigned (test_stream | '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_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=streaming_cache.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()) def cache_key_of(name, pcoll): return name + '_' + str(id(pcoll)) + '_' + str(id(pcoll.producer)) # 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 = \ 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 | '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_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 a new interactive environment to make the test idempotent. ie.new_env(cache_manager=streaming_cache.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') 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([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() 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().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, 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())