def test_cacheables(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) # pylint: disable=range-builtin-not-iterating init_pcoll = p | 'Init Create' >> beam.Create(range(10)) squares = init_pcoll | 'Square' >> beam.Map(lambda x: x * x) cubes = init_pcoll | 'Cube' >> beam.Map(lambda x: x**3) ib.watch(locals()) pin = instr.pin(p) self.assertEqual( pin.cacheables, { pin._cacheable_key(init_pcoll): { 'var': 'init_pcoll', 'version': str(id(init_pcoll)), 'pcoll_id': 'ref_PCollection_PCollection_10', 'producer_version': str(id(init_pcoll.producer)), 'pcoll': init_pcoll }, pin._cacheable_key(squares): { 'var': 'squares', 'version': str(id(squares)), 'pcoll_id': 'ref_PCollection_PCollection_11', 'producer_version': str(id(squares.producer)), 'pcoll': squares }, pin._cacheable_key(cubes): { 'var': 'cubes', 'version': str(id(cubes)), 'pcoll_id': 'ref_PCollection_PCollection_12', 'producer_version': str(id(cubes.producer)), 'pcoll': cubes } })
def test_background_caching_pipeline_proto(self): 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.pin(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 | 'a' >> cache.WriteCache(ie.current_env().cache_manager(), '') b = p | 'ReadUnboundedSourceB' >> beam.io.ReadFromPubSub( subscription='projects/fake-project/subscriptions/fake_sub') _ = b | '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_pipeline_to_read_cache(self): p_origin, init_pcoll, second_pcoll = self._example_pipeline() p_copy, _, _ = self._example_pipeline(False) # Mock as if cacheable PCollections are cached. init_pcoll_cache_key = 'init_pcoll_' + str(id(init_pcoll)) + '_' + str( id(init_pcoll.producer)) self._mock_write_cache(init_pcoll, init_pcoll_cache_key) second_pcoll_cache_key = 'second_pcoll_' + str( id(second_pcoll)) + '_' + str(id(second_pcoll.producer)) self._mock_write_cache(second_pcoll, second_pcoll_cache_key) ie.current_env().cache_manager().exists = MagicMock(return_value=True) # Mark the completeness of PCollections from the original(user) pipeline. ie.current_env().mark_pcollection_computed( (p_origin, init_pcoll, second_pcoll)) instr.pin(p_copy) cached_init_pcoll = p_origin | ( '_ReadCache_' + init_pcoll_cache_key) >> cache.ReadCache( ie.current_env().cache_manager(), init_pcoll_cache_key) # second_pcoll is never used as input and there is no need to read cache. class TestReadCacheWireVisitor(PipelineVisitor): """Replace init_pcoll with cached_init_pcoll for all occuring inputs.""" def enter_composite_transform(self, transform_node): self.visit_transform(transform_node) def visit_transform(self, transform_node): if transform_node.inputs: input_list = list(transform_node.inputs) for i in range(len(input_list)): if input_list[i] == init_pcoll: input_list[i] = cached_init_pcoll transform_node.inputs = tuple(input_list) v = TestReadCacheWireVisitor() p_origin.visit(v) assert_pipeline_equal(self, p_origin, p_copy)
def test_find_out_correct_user_pipeline(self): # This is the user pipeline instance we care in the watched scope. user_pipeline, _, _ = self._example_pipeline() # This is a new runner pipeline instance with the same pipeline graph to # what the user_pipeline represents. runner_pipeline = beam.pipeline.Pipeline.from_runner_api( user_pipeline.to_runner_api(use_fake_coders=True), user_pipeline.runner, options=None) # This is a totally irrelevant user pipeline in the watched scope. irrelevant_user_pipeline = beam.Pipeline( interactive_runner.InteractiveRunner()) ib.watch({'irrelevant_user_pipeline': irrelevant_user_pipeline}) # Build instrument from the runner pipeline. pipeline_instrument = instr.pin(runner_pipeline) self.assertIs(pipeline_instrument.user_pipeline, user_pipeline)
def run_pipeline(self, pipeline, options): if self._force_compute: ie.current_env().evict_computed_pcollections() pipeline_instrument = inst.pin(pipeline, options) # The user_pipeline analyzed might be None if the pipeline given has nothing # to be cached and tracing back to the user defined pipeline is impossible. # When it's None, there is no need to cache including the background # caching job and no result to track since no background caching job is # started at all. user_pipeline = pipeline_instrument.user_pipeline if user_pipeline: # Should use the underlying runner and run asynchronously. background_caching_job.attempt_to_run_background_caching_job( self._underlying_runner, user_pipeline, options) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( pipeline_instrument.instrumented_pipeline_proto(), self._underlying_runner, options) if not self._skip_display: a_pipeline_graph = pipeline_graph.PipelineGraph( pipeline_instrument.original_pipeline, render_option=self._render_option) a_pipeline_graph.display_graph() main_job_result = PipelineResult(pipeline_to_execute.run(), pipeline_instrument) # In addition to this pipeline result setting, redundant result setting from # outer scopes are also recommended since the user_pipeline might not be # available from within this scope. if user_pipeline: ie.current_env().set_pipeline_result( user_pipeline, main_job_result, is_main_job=True) main_job_result.wait_until_finish() if main_job_result.state is beam.runners.runner.PipelineState.DONE: # pylint: disable=dict-values-not-iterating ie.current_env().mark_pcollection_computed( pipeline_instrument.runner_pcoll_to_user_pcoll.values()) return main_job_result
def run_pipeline(self, pipeline, options): pipeline_instrument = inst.pin(pipeline, options) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( pipeline_instrument.instrumented_pipeline_proto(), self._underlying_runner, options) if not self._skip_display: a_pipeline_graph = pipeline_graph.PipelineGraph( pipeline_instrument.original_pipeline, render_option=self._render_option) a_pipeline_graph.display_graph() result = pipeline_to_execute.run() result.wait_until_finish() return PipelineResult(result, pipeline_instrument)
def test_instrument_example_pipeline_to_write_cache(self): # Original instance defined by user code has all variables handlers. p_origin, init_pcoll, second_pcoll = self._example_pipeline() # Copied instance when execution has no user defined variables. p_copy, _, _ = self._example_pipeline(False) # Instrument the copied pipeline. pin = instr.pin(p_copy) # Manually instrument original pipeline with expected pipeline transforms. init_pcoll_cache_key = pin.cache_key(init_pcoll) _ = init_pcoll | ( ('_WriteCache_' + init_pcoll_cache_key) >> cache.WriteCache( ie.current_env().cache_manager(), init_pcoll_cache_key)) second_pcoll_cache_key = pin.cache_key(second_pcoll) _ = second_pcoll | ( ('_WriteCache_' + second_pcoll_cache_key) >> cache.WriteCache( ie.current_env().cache_manager(), second_pcoll_cache_key)) # The 2 pipelines should be the same now. self.assertPipelineEqual(p_copy, p_origin)
def test_cache_key(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) # pylint: disable=range-builtin-not-iterating init_pcoll = p | 'Init Create' >> beam.Create(range(10)) squares = init_pcoll | 'Square' >> beam.Map(lambda x: x * x) cubes = init_pcoll | 'Cube' >> beam.Map(lambda x: x**3) # Watch the local variables, i.e., the Beam pipeline defined. ib.watch(locals()) pin = instr.pin(p) self.assertEqual( pin.cache_key(init_pcoll), 'init_pcoll_' + str(id(init_pcoll)) + '_ref_PCollection_PCollection_10_' + str(id(init_pcoll.producer))) self.assertEqual( pin.cache_key(squares), 'squares_' + str(id(squares)) + '_ref_PCollection_PCollection_11_' + str(id(squares.producer))) self.assertEqual( pin.cache_key(cubes), 'cubes_' + str(id(cubes)) + '_ref_PCollection_PCollection_12_' + str(id(cubes.producer)))
def attempt_to_run_background_caching_job(runner, user_pipeline, options=None): """Attempts to run a background caching job for a user-defined pipeline. The pipeline result is automatically tracked by Interactive Beam in case future cancellation/cleanup is needed. """ if is_background_caching_job_needed(user_pipeline): # Cancel non-terminal jobs if there is any before starting a new one. attempt_to_cancel_background_caching_job(user_pipeline) # TODO(BEAM-8335): refactor background caching job logic from # pipeline_instrument module to this module and aggregate tests. from apache_beam.runners.interactive import pipeline_instrument as instr runner_pipeline = beam.pipeline.Pipeline.from_runner_api( user_pipeline.to_runner_api(use_fake_coders=True), runner, options) background_caching_job_result = beam.pipeline.Pipeline.from_runner_api( instr.pin(runner_pipeline).background_caching_pipeline_proto(), runner, options).run() ie.current_env().set_pipeline_result(user_pipeline, background_caching_job_result, is_main_job=False)