def is_background_caching_job_needed(user_pipeline):
    """Determines if a background source recording job needs to be started.

  It does several state checks and recording state changes throughout the
  process. It is not idempotent to simplify the usage.
  """
    job = ie.current_env().get_background_caching_job(user_pipeline)
    # Checks if the pipeline contains any source that needs to be cached.
    need_cache = has_source_to_cache(user_pipeline)
    # If this is True, we can invalidate a previous done/running job if there is
    # one.
    cache_changed = is_source_to_cache_changed(user_pipeline)
    # When recording replay is disabled, cache is always needed for recordable
    # sources (if any).
    if need_cache and not ie.current_env().options.enable_recording_replay:
        from apache_beam.runners.interactive.options import capture_control
        capture_control.evict_captured_data()
        return True
    return (
        need_cache and
        # Checks if it's the first time running a job from the pipeline.
        (
            not job or
            # Or checks if there is no previous job.
            # DONE means a previous job has completed successfully and the
            # cached events might still be valid.
            not (
                job.is_done() or
                # RUNNING means a previous job has been started and is still
                # running.
                job.is_running()) or
            # Or checks if we can invalidate the previous job.
            cache_changed))
Exemple #2
0
    def test_capture_control_evict_captured_data(
            self, mocked_test_stream_service_stop,
            mocked_background_caching_job_cancel):
        p = _build_an_empty_streaming_pipeline()
        ie.current_env().track_user_pipelines()
        self.assertFalse(ie.current_env().tracked_user_pipelines == set())

        background_caching_job = bcj.BackgroundCachingJob(
            runner.PipelineResult(runner.PipelineState.RUNNING), limiters=[])
        ie.current_env().set_background_caching_job(p, background_caching_job)

        _fake_a_running_test_stream_service(p)
        # Fake the canceling state of the main job.
        background_caching_job._pipeline_result = runner.PipelineResult(
            runner.PipelineState.CANCELLING)
        self.assertIsNotNone(
            ie.current_env().get_test_stream_service_controller(p))
        ie.current_env().set_cached_source_signature(p, 'a signature')
        ie.current_env().mark_pcollection_computed(['fake_pcoll'])
        capture_control.evict_captured_data()
        mocked_background_caching_job_cancel.assert_called()
        mocked_test_stream_service_stop.assert_called_once()
        # Neither timer nor capture size limit is reached, thus, the cancelling
        # main job's background caching job is not considered as done.
        self.assertFalse(background_caching_job.is_done())
        self.assertIsNone(
            ie.current_env().get_test_stream_service_controller(p))
        self.assertTrue(ie.current_env().computed_pcollections == set())
        self.assertTrue(
            ie.current_env().get_cached_source_signature(p) == set())
Exemple #3
0
def evict_captured_data():
    """Forcefully evicts all captured replayable data.

  Once invoked, Interactive Beam will capture new data based on the guidance of
  options the next time it evaluates/visualizes PCollections or runs pipelines.
  """
    from apache_beam.runners.interactive.options import capture_control
    capture_control.evict_captured_data()
Exemple #4
0
def evict_recorded_data(pipeline=None):
    """Forcefully evicts all recorded replayable data for the given pipeline. If
  no pipeline is specified, evicts for all user defined pipelines.

  Once invoked, Interactive Beam will record new data based on the guidance of
  options the next time it evaluates/visualizes PCollections or runs pipelines.
  """
    from apache_beam.runners.interactive.options import capture_control
    capture_control.evict_captured_data(pipeline)
Exemple #5
0
    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
Exemple #6
0
    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