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.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 | '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_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()) pipeline_instrument = instr.build_pipeline_instrument(p) self.assertEqual( pipeline_instrument.cacheables, { pipeline_instrument._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 }, pipeline_instrument._cacheable_key(squares): { 'var': 'squares', 'version': str(id(squares)), 'pcoll_id': 'ref_PCollection_PCollection_11', 'producer_version': str(id(squares.producer)), 'pcoll': squares }, pipeline_instrument._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_pcoll_id_with_user_pipeline(self): p_id_user = beam.Pipeline(interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), p_id_user) init_pcoll = p_id_user | 'Init Create' >> beam.Create([1, 2, 3]) instrumentation = instr.build_pipeline_instrument(p_id_user) self.assertEqual(instrumentation.pcoll_id(init_pcoll), 'ref_PCollection_PCollection_8')
def test_instrument_example_unbounded_pipeline_to_read_cache(self): """Tests that the instrumenter works for a single unbounded source. """ # Create the pipeline that will be instrumented. p_original = beam.Pipeline(interactive_runner.InteractiveRunner()) 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 pcoll_1 = source_1 | 'square1' >> 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(p_original, [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) 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) # 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 attempt_to_run_background_caching_job(runner, user_pipeline, options=None): """Attempts to run a background source recording job for a user-defined pipeline. Returns True if a job was started, False otherwise. 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) # Cancel the gRPC server serving the test stream if there is one. attempt_to_stop_test_stream_service(user_pipeline) # TODO(BEAM-8335): refactor background source recording 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.build_pipeline_instrument( runner_pipeline).background_caching_pipeline_proto(), runner, options).run() limiters = ie.current_env().options.capture_control.limiters() ie.current_env().set_background_caching_job( user_pipeline, BackgroundCachingJob(background_caching_job_result, limiters=limiters)) return True return False
def test_cacheables(self): p_cacheables = beam.Pipeline(interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), p_cacheables) # pylint: disable=bad-option-value init_pcoll = p_cacheables | '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()) pipeline_instrument = instr.build_pipeline_instrument(p_cacheables) self.assertEqual( pipeline_instrument._cacheables, { pipeline_instrument.pcoll_id(init_pcoll): Cacheable(var='init_pcoll', version=str(id(init_pcoll)), producer_version=str(id(init_pcoll.producer)), pcoll=init_pcoll), pipeline_instrument.pcoll_id(squares): Cacheable(var='squares', version=str(id(squares)), producer_version=str(id(squares.producer)), pcoll=squares), pipeline_instrument.pcoll_id(cubes): Cacheable(var='cubes', version=str(id(cubes)), producer_version=str(id(cubes.producer)), pcoll=cubes) })
def test_cacheables(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), p) # 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()) pipeline_instrument = instr.build_pipeline_instrument(p) self.assertEqual( pipeline_instrument.cacheables, { pipeline_instrument._cacheable_key(init_pcoll): instr.Cacheable( var='init_pcoll', version=str(id(init_pcoll)), pcoll_id='ref_PCollection_PCollection_8', producer_version=str(id(init_pcoll.producer)), pcoll=init_pcoll), pipeline_instrument._cacheable_key(squares): instr.Cacheable( var='squares', version=str(id(squares)), pcoll_id='ref_PCollection_PCollection_9', producer_version=str(id(squares.producer)), pcoll=squares), pipeline_instrument._cacheable_key(cubes): instr.Cacheable( var='cubes', version=str(id(cubes)), pcoll_id='ref_PCollection_PCollection_10', producer_version=str(id(cubes.producer)), pcoll=cubes) })
def test_pipeline_pruned_when_input_pcoll_is_cached(self): user_pipeline, init_pcoll, _ = self._example_pipeline() runner_pipeline = beam.Pipeline.from_runner_api( user_pipeline.to_runner_api(), user_pipeline.runner, None) ie.current_env().add_derived_pipeline(user_pipeline, runner_pipeline) # Mock as if init_pcoll is cached. init_pcoll_cache_key = self.cache_key_of('init_pcoll', init_pcoll) self._mock_write_cache(user_pipeline, [b'1', b'2', b'3'], init_pcoll_cache_key) ie.current_env().mark_pcollection_computed([init_pcoll]) # Build an instrument from the runner pipeline. pipeline_instrument = instr.build_pipeline_instrument(runner_pipeline) pruned_proto = pipeline_instrument.instrumented_pipeline_proto() # Skip the prune step for comparison, it should contain the sub-graph that # produces init_pcoll but not useful anymore. full_proto = pipeline_instrument._pipeline.to_runner_api() self.assertEqual( len(pruned_proto.components.transforms[ 'ref_AppliedPTransform_AppliedPTransform_1'].subtransforms), 5) assert_pipeline_proto_not_contain_top_level_transform( self, pruned_proto, 'Init Source') self.assertEqual( len(full_proto.components.transforms[ 'ref_AppliedPTransform_AppliedPTransform_1'].subtransforms), 6) assert_pipeline_proto_contain_top_level_transform( self, full_proto, 'Init-Source')
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 = self.cache_key_of('init_pcoll', init_pcoll) self._mock_write_cache(p_origin, [b'1', b'2', b'3'], init_pcoll_cache_key) second_pcoll_cache_key = self.cache_key_of('second_pcoll', second_pcoll) self._mock_write_cache(p_origin, [b'1', b'4', b'9'], second_pcoll_cache_key) # Mark the completeness of PCollections from the original(user) pipeline. ie.current_env().mark_pcollection_computed((init_pcoll, second_pcoll)) ie.current_env().add_derived_pipeline(p_origin, p_copy) instr.build_pipeline_instrument(p_copy) cached_init_pcoll = ( p_origin | '_ReadCache_' + init_pcoll_cache_key >> cache.ReadCache( ie.current_env().get_cache_manager(p_origin), init_pcoll_cache_key) | 'unreify' >> beam.Map(lambda _: _)) # 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: main_inputs = dict(transform_node.main_inputs) for tag, main_input in main_inputs.items(): if main_input == init_pcoll: main_inputs[tag] = cached_init_pcoll transform_node.main_inputs = main_inputs v = TestReadCacheWireVisitor() p_origin.visit(v) assert_pipeline_equal(self, p_origin, p_copy)
def test_side_effect_pcoll_is_included(self): pipeline_with_side_effect = beam.Pipeline( interactive_runner.InteractiveRunner()) # Deliberately not assign the result to a variable to make it a # "side effect" transform. Note we never watch anything from # the pipeline defined locally either. # pylint: disable=range-builtin-not-iterating,expression-not-assigned pipeline_with_side_effect | 'Init Create' >> beam.Create(range(10)) pipeline_instrument = instr.build_pipeline_instrument( pipeline_with_side_effect) self.assertTrue(pipeline_instrument._extended_targets)
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([b'1', b'2', b'3'], init_pcoll_cache_key) second_pcoll_cache_key = 'second_pcoll_' + str( id(second_pcoll)) + '_' + str(id(second_pcoll.producer)) self._mock_write_cache([b'1', b'4', b'9'], second_pcoll_cache_key) # Mark the completeness of PCollections from the original(user) pipeline. ie.current_env().mark_pcollection_computed( (p_origin, init_pcoll, second_pcoll)) instr.build_pipeline_instrument(p_copy) cached_init_pcoll = ( p_origin | '_ReadCache_' + init_pcoll_cache_key >> cache.ReadCache( ie.current_env().cache_manager(), init_pcoll_cache_key) | 'unreify' >> beam.Map(lambda _: _)) # 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_side_effect_pcoll_is_included(self): pipeline_with_side_effect = beam.Pipeline( interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), pipeline_with_side_effect) # Deliberately not assign the result to a variable to make it a # "side effect" transform. Note we never watch anything from # the pipeline defined locally either. # pylint: disable=bad-option-value,expression-not-assigned pipeline_with_side_effect | 'Init Create' >> beam.Create(range(10)) pipeline_instrument = instr.build_pipeline_instrument( pipeline_with_side_effect) self.assertTrue(pipeline_instrument._extended_targets)
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(), user_pipeline.runner, options=None) ie.current_env().add_derived_pipeline(user_pipeline, runner_pipeline) # 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.build_pipeline_instrument(runner_pipeline) self.assertIs(pipeline_instrument.user_pipeline, user_pipeline)
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()) pipeline_instrument = instr.build_pipeline_instrument(p) self.assertEqual(pipeline_instrument.cache_key(init_pcoll), self.cache_key_of('init_pcoll', init_pcoll)) self.assertEqual(pipeline_instrument.cache_key(squares), self.cache_key_of('squares', squares)) self.assertEqual(pipeline_instrument.cache_key(cubes), self.cache_key_of('cubes', cubes))
def run_pipeline(self, pipeline, options): if not ie.current_env().options.enable_capture_replay: capture_control.evict_captured_data() if self._force_compute: ie.current_env().evict_computed_pcollections() pipeline_instrument = inst.build_pipeline_instrument(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) if self._blocking: 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 test_cache_key(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), p) # pylint: disable=bad-option-value 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()) pipeline_instrument = instr.build_pipeline_instrument(p) self.assertEqual(pipeline_instrument.cache_key(init_pcoll), self.cache_key_of('init_pcoll', init_pcoll)) self.assertEqual(pipeline_instrument.cache_key(squares), self.cache_key_of('squares', squares)) self.assertEqual(pipeline_instrument.cache_key(cubes), self.cache_key_of('cubes', cubes))
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. pipeline_instrument = instr.build_pipeline_instrument(p_copy) # Manually instrument original pipeline with expected pipeline transforms. init_pcoll_cache_key = pipeline_instrument.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 = pipeline_instrument.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. assert_pipeline_equal(self, p_copy, p_origin)
def test_cacheables(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), p) # 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()) pipeline_instrument = instr.build_pipeline_instrument(p) # TODO(BEAM-7760): The PipelineInstrument cacheables maintains a global list # of cacheable PCollections across all pipelines. Here we take the subset of # cacheables that only pertain to this test's pipeline. cacheables = { k: c for k, c in pipeline_instrument.cacheables.items() if c.pcoll.pipeline is p } self.assertEqual( cacheables, { pipeline_instrument._cacheable_key(init_pcoll): instr.Cacheable( var='init_pcoll', version=str(id(init_pcoll)), pcoll_id='ref_PCollection_PCollection_8', producer_version=str(id(init_pcoll.producer)), pcoll=init_pcoll), pipeline_instrument._cacheable_key(squares): instr.Cacheable( var='squares', version=str(id(squares)), pcoll_id='ref_PCollection_PCollection_9', producer_version=str(id(squares.producer)), pcoll=squares), pipeline_instrument._cacheable_key(cubes): instr.Cacheable( var='cubes', version=str(id(cubes)), pcoll_id='ref_PCollection_PCollection_10', producer_version=str(id(cubes.producer)), pcoll=cubes) })
def _build_query_components( query: str, found: Dict[str, beam.PCollection] ) -> Tuple[str, Union[Dict[str, beam.PCollection], beam.PCollection, beam.Pipeline]]: """Builds necessary components needed to apply the SqlTransform. Args: query: The SQL query to be executed by the magic. found: The PCollections with variable names found to be used by the query. Returns: The processed query to be executed by the magic and a source to apply the SqlTransform to: a dictionary of tagged PCollections, or a single PCollection, or the pipeline to execute the query. """ if found: user_pipeline = next(iter(found.values())).pipeline cache_manager = ie.current_env().get_cache_manager(user_pipeline) instrumentation = inst.build_pipeline_instrument(user_pipeline) sql_pipeline = beam.Pipeline(options=user_pipeline._options) ie.current_env().add_derived_pipeline(user_pipeline, sql_pipeline) sql_source = {} if instrumentation.has_unbounded_sources: sql_source = pcolls_from_streaming_cache(user_pipeline, sql_pipeline, found, instrumentation, cache_manager) else: for pcoll_name, pcoll in found.items(): cache_key = instrumentation.cache_key(pcoll) sql_source[pcoll_name] = pcoll_from_file_cache( sql_pipeline, pcoll, cache_manager, cache_key) if len(sql_source) == 1: query = replace_single_pcoll_token(query, next(iter(sql_source.keys()))) sql_source = next(iter(sql_source.values())) else: sql_source = beam.Pipeline() return query, sql_source
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) # Evict all caches if there is any. ie.current_env().cleanup() # 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.build_pipeline_instrument( 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)
def test_pcoll_id_with_runner_pipeline(self): p_id_runner = beam.Pipeline(interactive_runner.InteractiveRunner()) ie.current_env().set_cache_manager(InMemoryCache(), p_id_runner) # pylint: disable=possibly-unused-variable init_pcoll = p_id_runner | 'Init Create' >> beam.Create([1, 2, 3]) ib.watch(locals()) # It's normal that when executing, the pipeline object is a different # but equivalent instance from what user has built. The pipeline instrument # should be able to identify if the original instance has changed in an # interactive env while mutating the other instance for execution. The # version map can be used to figure out what the PCollection instances are # in the original instance and if the evaluation has changed since last # execution. p2_id_runner = beam.Pipeline(interactive_runner.InteractiveRunner()) # pylint: disable=bad-option-value init_pcoll_2 = p2_id_runner | 'Init Create' >> beam.Create(range(10)) ie.current_env().add_derived_pipeline(p_id_runner, p2_id_runner) instrumentation = instr.build_pipeline_instrument(p2_id_runner) # The cache_key should use id(init_pcoll) as prefix even when # init_pcoll_2 is supplied as long as the version map is given. self.assertEqual(instrumentation.pcoll_id(init_pcoll_2), 'ref_PCollection_PCollection_8')
def test_instrument_example_unbounded_pipeline_direct_from_source(self): """Tests that the it caches PCollections from a source. """ # 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 # 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() # pylint: disable=unused-variable test_stream = ( p_expected | TestStream(output_tags=[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(use_fake_coders=True), instrumenter.instrumented_pipeline_proto())
def run_pipeline(self, pipeline, options): if not ie.current_env().options.enable_recording_replay: capture_control.evict_captured_data() if self._force_compute: ie.current_env().evict_computed_pcollections() # Make sure that sources without a user reference are still cached. watch_sources(pipeline) user_pipeline = ie.current_env().user_pipeline(pipeline) pipeline_instrument = inst.build_pipeline_instrument(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. 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) if (background_caching_job.has_source_to_cache(user_pipeline) and not background_caching_job. is_a_test_stream_service_running(user_pipeline)): streaming_cache_manager = ie.current_env().get_cache_manager( user_pipeline) # Only make the server if it doesn't exist already. if (streaming_cache_manager and not ie.current_env(). get_test_stream_service_controller(user_pipeline)): def exception_handler(e): _LOGGER.error(str(e)) return True test_stream_service = TestStreamServiceController( streaming_cache_manager, exception_handler=exception_handler) test_stream_service.start() ie.current_env().set_test_stream_service_controller( user_pipeline, test_stream_service) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( pipeline_instrument.instrumented_pipeline_proto(), self._underlying_runner, options) if ie.current_env().get_test_stream_service_controller(user_pipeline): endpoint = ie.current_env().get_test_stream_service_controller( user_pipeline).endpoint # TODO: make the StreamingCacheManager and TestStreamServiceController # constructed when the InteractiveEnvironment is imported. class TestStreamVisitor(PipelineVisitor): def visit_transform(self, transform_node): from apache_beam.testing.test_stream import TestStream if (isinstance(transform_node.transform, TestStream) and not transform_node.transform._events): transform_node.transform._endpoint = endpoint pipeline_to_execute.visit(TestStreamVisitor()) if not self._skip_display: a_pipeline_graph = pipeline_graph.PipelineGraph( pipeline_instrument.original_pipeline_proto, 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) if self._blocking: 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.cached_pcolls) return main_job_result
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 the pipeline that will be instrumented. from apache_beam.options.pipeline_options import StandardOptions options = StandardOptions(streaming=True) streaming_cache_manager = StreamingCache(cache_dir=None) p_original_cache_source = beam.Pipeline( interactive_runner.InteractiveRunner(), options) ie.current_env().set_cache_manager(streaming_cache_manager, p_original_cache_source) # pylint: disable=possibly-unused-variable source_1 = ( p_original_cache_source | '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. utils.watch_sources(p_original_cache_source) 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_cache_source.to_runner_api(), runner=interactive_runner.InteractiveRunner(), options=options) ie.current_env().add_derived_pipeline(p_original_cache_source, p_copy) 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) ie.current_env().add_derived_pipeline(p_original_cache_source, actual_pipeline) # 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() ie.current_env().set_cache_manager(streaming_cache_manager, p_expected) 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().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([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(), 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 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()) # This should be noop. utils.watch_sources(p_original) self._mock_write_cache(p_original, [], 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) ie.current_env().add_derived_pipeline(p_original, p_copy) 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(), 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 the pipeline that will be instrumented. from apache_beam.options.pipeline_options import StandardOptions options = StandardOptions(streaming=True) p_original_read_cache = beam.Pipeline( interactive_runner.InteractiveRunner(), options) ie.current_env().set_cache_manager(StreamingCache(cache_dir=None), p_original_read_cache) source_1 = p_original_read_cache | '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()) # This should be noop. utils.watch_sources(p_original_read_cache) # Instrument the original pipeline to create the pipeline the user will see. p_copy = beam.Pipeline.from_runner_api( p_original_read_cache.to_runner_api(), runner=interactive_runner.InteractiveRunner(), options=options) ie.current_env().add_derived_pipeline(p_original_read_cache, p_copy) 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() ie.current_env().set_cache_manager(StreamingCache(cache_dir=None), p_expected) 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().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]) 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())