def _make_beam_pipeline(self) -> beam.Pipeline: """Makes beam pipeline.""" # TODO(b/142684737): refactor when beam support multi-processing by args. pipeline_options = PipelineOptions(self._beam_pipeline_args) parallelism = pipeline_options.view_as( DirectOptions).direct_num_workers if parallelism == 0: try: parallelism = multiprocessing.cpu_count() except NotImplementedError as e: absl.logging.warning('Cannot get cpu count: %s' % e) parallelism = 1 pipeline_options.view_as( DirectOptions).direct_num_workers = parallelism absl.logging.info('Using %d process(es) for Beam pipeline execution.' % parallelism) if parallelism > 1: if beam_runner_api_pb2: env = beam_runner_api_pb2.Environment( urn=python_urns.SUBPROCESS_SDK, payload=b'%s -m apache_beam.runners.worker.sdk_worker_main' % (sys.executable or sys.argv[0]).encode('ascii')) else: env = environments.SubprocessSDKEnvironment( command_string= '%s -m apache_beam.runners.worker.sdk_worker_main' % (sys.executable or sys.argv[0])) return beam.Pipeline( options=pipeline_options, runner=fn_api_runner.FnApiRunner(default_environment=env)) return beam.Pipeline(argv=self._beam_pipeline_args)
def test_deduplication_by_proto(self): context = pipeline_context.PipelineContext() env_proto = environments.SubprocessSDKEnvironment( command_string="foo").to_runner_api(None) env_ref_1 = context.environments.get_by_proto(env_proto) env_ref_2 = context.environments.get_by_proto(env_proto, deduplicate=True) self.assertEqual(env_ref_1, env_ref_2)
def test_equal_environments_are_deduplicated_when_fetched_by_obj_or_proto( self): context = pipeline_context.PipelineContext() env = environments.SubprocessSDKEnvironment(command_string="foo") env_proto = env.to_runner_api(None) id_from_proto = context.environments.get_by_proto(env_proto) id_from_obj = context.environments.get_id(env) self.assertEqual(id_from_obj, id_from_proto) self.assertEqual( context.environments.get_by_id(id_from_obj).command_string, "foo") env = environments.SubprocessSDKEnvironment(command_string="bar") env_proto = env.to_runner_api(None) id_from_obj = context.environments.get_id(env) id_from_proto = context.environments.get_by_proto( env_proto, deduplicate=True) self.assertEqual(id_from_obj, id_from_proto) self.assertEqual( context.environments.get_by_id(id_from_obj).command_string, "bar")
def run_pipeline( self, pipeline, # type: Pipeline options # type: pipeline_options.PipelineOptions ): # type: (...) -> RunnerResult RuntimeValueProvider.set_runtime_options({}) # Setup "beam_fn_api" experiment options if lacked. experiments = (options.view_as( pipeline_options.DebugOptions).experiments or []) if not 'beam_fn_api' in experiments: experiments.append('beam_fn_api') options.view_as( pipeline_options.DebugOptions).experiments = experiments # This is sometimes needed if type checking is disabled # to enforce that the inputs (and outputs) of GroupByKey operations # are known to be KVs. from apache_beam.runners.dataflow.dataflow_runner import DataflowRunner # TODO: Move group_by_key_input_visitor() to a non-dataflow specific file. pipeline.visit( DataflowRunner.group_by_key_input_visitor( not options.view_as(pipeline_options.TypeOptions ).allow_non_deterministic_key_coders)) self._bundle_repeat = self._bundle_repeat or options.view_as( pipeline_options.DirectOptions).direct_runner_bundle_repeat pipeline_direct_num_workers = options.view_as( pipeline_options.DirectOptions).direct_num_workers if pipeline_direct_num_workers == 0: self._num_workers = multiprocessing.cpu_count() else: self._num_workers = pipeline_direct_num_workers or self._num_workers # set direct workers running mode if it is defined with pipeline options. running_mode = \ options.view_as(pipeline_options.DirectOptions).direct_running_mode if running_mode == 'multi_threading': self._default_environment = environments.EmbeddedPythonGrpcEnvironment( ) elif running_mode == 'multi_processing': command_string = '%s -m apache_beam.runners.worker.sdk_worker_main' \ % sys.executable self._default_environment = environments.SubprocessSDKEnvironment( command_string=command_string) self._profiler_factory = Profile.factory_from_options( options.view_as(pipeline_options.ProfilingOptions)) self._latest_run_result = self.run_via_runner_api( pipeline.to_runner_api( default_environment=self._default_environment)) return self._latest_run_result