def test_read_cache_expansion(self): p = beam.Pipeline(runner=self.runner) # The cold run. pcoll = (p | 'Create' >> beam.Create([1, 2, 3]) | 'Double' >> beam.Map(lambda x: x * 2) | 'Square' >> beam.Map(lambda x: x**2)) pipeline_proto = to_stable_runner_api(p) pipeline_info = pipeline_analyzer.PipelineInfo( pipeline_proto.components) pcoll_id = 'ref_PCollection_PCollection_3' # Output PCollection of Square cache_label1 = pipeline_info.cache_label(pcoll_id) analyzer = pipeline_analyzer.PipelineAnalyzer(self.cache_manager, pipeline_proto, self.runner) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( analyzer.pipeline_proto_to_execute(), p.runner, p._options) pipeline_to_execute.run().wait_until_finish() # The second run. _ = (pcoll | 'Triple' >> beam.Map(lambda x: x * 3) | 'Cube' >> beam.Map(lambda x: x**3)) analyzer = pipeline_analyzer.PipelineAnalyzer(self.cache_manager, to_stable_runner_api(p), self.runner) expected_pipeline = beam.Pipeline(runner=self.runner) pcoll1 = (expected_pipeline | 'Load%s' % cache_label1 >> cache.ReadCache( self.cache_manager, cache_label1)) pcoll2 = pcoll1 | 'Triple' >> beam.Map(lambda x: x * 3) pcoll3 = pcoll2 | 'Cube' >> beam.Map(lambda x: x**3) cache_label2 = 'PColl-7654321' cache_label3 = 'PColl-3141593' # pylint: disable=expression-not-assigned pcoll2 | 'CacheSample%s' % cache_label2 >> cache.WriteCache( self.cache_manager, cache_label2, sample=True, sample_size=10) pcoll3 | 'CacheSample%s' % cache_label3 >> cache.WriteCache( self.cache_manager, cache_label3, sample=True, sample_size=10) pcoll3 | 'CacheFull%s' % cache_label3 >> cache.WriteCache( self.cache_manager, cache_label3) # Since ReadCache & WriteCache expansion leads to more than 50 PTransform # protos in the pipeline, a simple check of proto map size is enough. self.assertPipelineEqual(analyzer.pipeline_proto_to_execute(), to_stable_runner_api(expected_pipeline))
def test_write_cache_expansion(self): p = beam.Pipeline(runner=self.runner) pcoll1 = p | 'Create' >> beam.Create([1, 2, 3]) pcoll2 = pcoll1 | 'Double' >> beam.Map(lambda x: x * 2) pcoll3 = pcoll2 | 'Square' >> beam.Map(lambda x: x**2) analyzer = pipeline_analyzer.PipelineAnalyzer(self.cache_manager, to_stable_runner_api(p), self.runner) cache_label1 = 'PColl-1234567' cache_label2 = 'PColl-7654321' cache_label3 = 'PColl-3141593' # pylint: disable=expression-not-assigned pcoll1 | 'CacheSample%s' % cache_label1 >> cache.WriteCache( self.cache_manager, cache_label1, sample=True, sample_size=10) pcoll2 | 'CacheSample%s' % cache_label2 >> cache.WriteCache( self.cache_manager, cache_label2, sample=True, sample_size=10) pcoll3 | 'CacheSample%s' % cache_label3 >> cache.WriteCache( self.cache_manager, cache_label3, sample=True, sample_size=10) pcoll3 | 'CacheFull%s' % cache_label3 >> cache.WriteCache( self.cache_manager, cache_label3) expected_pipeline_proto = to_stable_runner_api(p) # Since WriteCache expansion leads to more than 50 PTransform protos in the # pipeline, a simple check of proto map size is enough. self.assertPipelineEqual(analyzer.pipeline_proto_to_execute(), expected_pipeline_proto)
def test_basic(self): p = beam.Pipeline(runner=self.runner) # The cold run. pcoll = (p | 'Create' >> beam.Create([1, 2, 3]) | 'Double' >> beam.Map(lambda x: x * 2) | 'Square' >> beam.Map(lambda x: x**2)) analyzer = pipeline_analyzer.PipelineAnalyzer(self.cache_manager, to_stable_runner_api(p), self.runner) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( analyzer.pipeline_proto_to_execute(), p.runner, p._options) pipeline_to_execute.run().wait_until_finish() self.assertEqual( len(analyzer.tl_required_trans_ids()), 7 # Create, Double, Square, CacheSample * 3, CacheFull ) self.assertEqual(len(analyzer.tl_referenced_pcoll_ids()), 3) self.assertEqual(len(analyzer.read_cache_ids()), 0) self.assertEqual(len(analyzer.write_cache_ids()), 4) # The second run. _ = (pcoll | 'Triple' >> beam.Map(lambda x: x * 3) | 'Cube' >> beam.Map(lambda x: x**3)) analyzer = pipeline_analyzer.PipelineAnalyzer(self.cache_manager, to_stable_runner_api(p), self.runner) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( analyzer.pipeline_proto_to_execute(), p.runner, p._options) self.assertEqual( len(analyzer.tl_required_trans_ids()), 6 # Read, Triple, Cube, CacheSample * 2, CacheFull ) self.assertEqual(len(analyzer.tl_referenced_pcoll_ids()), 3) self.assertEqual(len(analyzer.read_cache_ids()), 1) self.assertEqual(len(analyzer.write_cache_ids()), 3)
def test_word_count(self): p = beam.Pipeline(runner=self.runner) class WordExtractingDoFn(beam.DoFn): def process(self, element): text_line = element.strip() words = text_line.split() return words # Count the occurrences of each word. pcoll1 = p | beam.Create(['to be or not to be that is the question']) pcoll2 = pcoll1 | 'Split' >> beam.ParDo(WordExtractingDoFn()) pcoll3 = pcoll2 | 'Pair with One' >> beam.Map(lambda x: (x, 1)) pcoll4 = pcoll3 | 'Group' >> beam.GroupByKey() pcoll5 = pcoll4 | 'Count' >> beam.Map(lambda item: (item[0], sum(item[1]))) analyzer = pipeline_analyzer.PipelineAnalyzer(self.cache_manager, to_stable_runner_api(p), self.runner) cache_label1 = 'PColl-1111111' cache_label2 = 'PColl-2222222' cache_label3 = 'PColl-3333333' cache_label4 = 'PColl-4444444' cache_label5 = 'PColl-5555555' # pylint: disable=expression-not-assigned pcoll1 | 'CacheSample%s' % cache_label1 >> cache.WriteCache( self.cache_manager, cache_label1, sample=True, sample_size=10) pcoll2 | 'CacheSample%s' % cache_label2 >> cache.WriteCache( self.cache_manager, cache_label2, sample=True, sample_size=10) pcoll3 | 'CacheSample%s' % cache_label3 >> cache.WriteCache( self.cache_manager, cache_label3, sample=True, sample_size=10) pcoll4 | 'CacheSample%s' % cache_label4 >> cache.WriteCache( self.cache_manager, cache_label3, sample=True, sample_size=10) pcoll5 | 'CacheSample%s' % cache_label5 >> cache.WriteCache( self.cache_manager, cache_label3, sample=True, sample_size=10) pcoll5 | 'CacheFull%s' % cache_label5 >> cache.WriteCache( self.cache_manager, cache_label3) expected_pipeline_proto = to_stable_runner_api(p) self.assertPipelineEqual(analyzer.pipeline_proto_to_execute(), expected_pipeline_proto) pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( analyzer.pipeline_proto_to_execute(), p.runner, p._options) pipeline_to_execute.run().wait_until_finish()
def run_pipeline(self, pipeline): if not hasattr(self, '_desired_cache_labels'): self._desired_cache_labels = set() # Invoke a round trip through the runner API. This makes sure the Pipeline # proto is stable. pipeline = beam.pipeline.Pipeline.from_runner_api( pipeline.to_runner_api(), pipeline.runner, pipeline._options) # Snapshot the pipeline in a portable proto before mutating it. pipeline_proto, original_context = pipeline.to_runner_api( return_context=True) pcolls_to_pcoll_id = self._pcolls_to_pcoll_id(pipeline, original_context) analyzer = pipeline_analyzer.PipelineAnalyzer(self._cache_manager, pipeline_proto, self._underlying_runner, pipeline._options, self._desired_cache_labels) # Should be only accessed for debugging purpose. self._analyzer = analyzer pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( analyzer.pipeline_proto_to_execute(), self._underlying_runner, pipeline._options) pipeline_info = pipeline_analyzer.PipelineInfo(pipeline_proto.components) display = display_manager.DisplayManager( pipeline_info=pipeline_info, pipeline_proto=pipeline_proto, caches_used=analyzer.caches_used(), cache_manager=self._cache_manager, referenced_pcollections=analyzer.top_level_referenced_pcollection_ids(), required_transforms=analyzer.top_level_required_transforms()) display.start_periodic_update() result = pipeline_to_execute.run() result.wait_until_finish() display.stop_periodic_update() return PipelineResult(result, self, pipeline_info, self._cache_manager, pcolls_to_pcoll_id)
def run_pipeline(self, pipeline, options): if not hasattr(self, '_desired_cache_labels'): self._desired_cache_labels = set() # Invoke a round trip through the runner API. This makes sure the Pipeline # proto is stable. pipeline = beam.pipeline.Pipeline.from_runner_api( pipeline.to_runner_api(use_fake_coders=True), pipeline.runner, options) # Snapshot the pipeline in a portable proto before mutating it. pipeline_proto, original_context = pipeline.to_runner_api( return_context=True, use_fake_coders=True) pcolls_to_pcoll_id = self._pcolls_to_pcoll_id(pipeline, original_context) analyzer = pipeline_analyzer.PipelineAnalyzer( self._cache_manager, pipeline_proto, self._underlying_runner, options, self._desired_cache_labels) # Should be only accessed for debugging purpose. self._analyzer = analyzer pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api( analyzer.pipeline_proto_to_execute(), self._underlying_runner, options) if not self._skip_display: display = display_manager.DisplayManager( pipeline_proto=pipeline_proto, pipeline_analyzer=analyzer, cache_manager=self._cache_manager, pipeline_graph_renderer=self._renderer) display.start_periodic_update() result = pipeline_to_execute.run() result.wait_until_finish() if not self._skip_display: display.stop_periodic_update() return PipelineResult(result, self, self._analyzer.pipeline_info(), self._cache_manager, pcolls_to_pcoll_id)