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. assert_pipeline_equal(self, p_copy, p_origin)
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_build_pipeline_fragment(self, cell): with cell: # Cell 1 p = beam.Pipeline(ir.InteractiveRunner()) p_expected = beam.Pipeline(ir.InteractiveRunner()) # Watch local scope now to allow interactive beam to track the pipelines. ib.watch(locals()) with cell: # Cell 2 # pylint: disable=range-builtin-not-iterating init = p | 'Init' >> beam.Create(range(10)) init_expected = p_expected | 'Init' >> beam.Create(range(10)) with cell: # Cell 3 square = init | 'Square' >> beam.Map(lambda x: x * x) _ = init | 'Cube' >> beam.Map(lambda x: x**3) _ = init_expected | 'Square' >> beam.Map(lambda x: x * x) # Watch every PCollection has been defined so far in local scope. ib.watch(locals()) fragment = pf.PipelineFragment([square]).deduce_fragment() assert_pipeline_equal(self, p_expected, fragment)
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_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(watch=False) ie.current_env().add_derived_pipeline(p_origin, p_copy) # 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 | 'reify init' >> beam.Map(lambda _: _) | '_WriteCache_' + init_pcoll_cache_key >> cache.WriteCache( ie.current_env().get_cache_manager(p_origin), init_pcoll_cache_key)) second_pcoll_cache_key = pipeline_instrument.cache_key(second_pcoll) _ = (second_pcoll | 'reify second' >> beam.Map(lambda _: _) | '_WriteCache_' + second_pcoll_cache_key >> cache.WriteCache( ie.current_env().get_cache_manager(p_origin), second_pcoll_cache_key)) # The 2 pipelines should be the same now. assert_pipeline_equal(self, p_copy, p_origin)
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) 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)