def test_determine_a_test_stream_service_running(self): pipeline = _build_an_empty_stream_pipeline() test_stream_service = TestStreamServiceController(reader=None) test_stream_service.start() ie.current_env().set_test_stream_service_controller( pipeline, test_stream_service) self.assertTrue(bcj.is_a_test_stream_service_running(pipeline))
def test_stop_a_running_test_stream_service(self): pipeline = _build_an_empty_stream_pipeline() test_stream_service = TestStreamServiceController(reader=None) test_stream_service.start() ie.current_env().set_test_stream_service_controller( pipeline, test_stream_service) bcj.attempt_to_stop_test_stream_service(pipeline) self.assertFalse(bcj.is_a_test_stream_service_running(pipeline))
def test_noop_when_no_test_stream_service_running(self, _mocked_stop): pipeline = _build_an_empty_stream_pipeline() self.assertFalse(bcj.is_a_test_stream_service_running(pipeline)) bcj.attempt_to_stop_test_stream_service(pipeline) _mocked_stop.assert_not_called()
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() # Make sure that sources without a user reference are still cached. inst.watch_sources(pipeline) user_pipeline = inst.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().cache_manager() if streaming_cache_manager: test_stream_service = TestStreamServiceController( streaming_cache_manager) 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, 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