def run_pipeline(self, pipeline): docker_image = ( pipeline.options.view_as(PortableOptions).harness_docker_image or self.default_docker_image()) job_endpoint = pipeline.options.view_as(PortableOptions).job_endpoint if not job_endpoint: raise ValueError( 'job_endpoint should be provided while creating runner.') proto_context = pipeline_context.PipelineContext( default_environment_url=docker_image) proto_pipeline = pipeline.to_runner_api(context=proto_context) if not self.is_embedded_fnapi_runner: # Java has different expectations about coders # (windowed in Fn API, but *un*windowed in runner API), whereas the # embedded FnApiRunner treats them consistently, so we must guard this # for now, until FnApiRunner is fixed. # See also BEAM-2717. for pcoll in proto_pipeline.components.pcollections.values(): if pcoll.coder_id not in proto_context.coders: # This is not really a coder id, but a pickled coder. coder = coders.registry.get_coder(pickler.loads(pcoll.coder_id)) pcoll.coder_id = proto_context.coders.get_id(coder) proto_context.coders.populate_map(proto_pipeline.components.coders) # Some runners won't detect the GroupByKey transform unless it has no # subtransforms. Remove all sub-transforms until BEAM-4605 is resolved. for _, transform_proto in list( proto_pipeline.components.transforms.items()): if transform_proto.spec.urn == common_urns.primitives.GROUP_BY_KEY.urn: for sub_transform in transform_proto.subtransforms: del proto_pipeline.components.transforms[sub_transform] del transform_proto.subtransforms[:] # TODO: Define URNs for options. options = {'beam:option:' + k + ':v1': v for k, v in pipeline._options.get_all_options().iteritems() if v is not None} job_service = beam_job_api_pb2_grpc.JobServiceStub( grpc.insecure_channel(job_endpoint)) prepare_response = job_service.Prepare( beam_job_api_pb2.PrepareJobRequest( job_name='job', pipeline=proto_pipeline, pipeline_options=job_utils.dict_to_struct(options))) if prepare_response.artifact_staging_endpoint.url: stager = portable_stager.PortableStager( grpc.insecure_channel(prepare_response.artifact_staging_endpoint.url), prepare_response.staging_session_token) retrieval_token, _ = stager.stage_job_resources( pipeline._options, staging_location='') else: retrieval_token = None run_response = job_service.Run( beam_job_api_pb2.RunJobRequest( preparation_id=prepare_response.preparation_id, retrieval_token=retrieval_token)) return PipelineResult(job_service, run_response.job_id)
def run(self, pipeline): """Remotely executes entire pipeline or parts reachable from node.""" # Performing configured PTransform overrides. pipeline.replace_all(PythonRPCDirectRunner._PTRANSFORM_OVERRIDES) # Start the RPC co-process manager = DockerRPCManager() # Submit the job to the RPC co-process jobName = ('Job-' + ''.join(random.choice(string.ascii_uppercase) for _ in range(6))) options = {k: v for k, v in pipeline._options.get_all_options().iteritems() if v is not None} try: response = manager.service.run(beam_job_api_pb2.SubmitJobRequest( pipeline=pipeline.to_runner_api(), pipelineOptions=job_utils.dict_to_struct(options), jobName=jobName)) logging.info('Submitted a job with id: %s', response.jobId) # Return the result object that references the manager instance result = PythonRPCDirectPipelineResult(response.jobId, manager) return result except grpc.RpcError: logging.error('Failed to run the job with name: %s', jobName) raise
def send_prepare_request(max_retries=5): num_retries = 0 while True: try: # This reports channel is READY but connections may fail # Seems to be only an issue on Mac with port forwardings grpc.channel_ready_future(channel).result() return job_service.Prepare( beam_job_api_pb2.PrepareJobRequest( job_name='job', pipeline=proto_pipeline, pipeline_options=job_utils.dict_to_struct(options))) except grpc._channel._Rendezvous as e: num_retries += 1 if num_retries > max_retries: raise e
def run_pipeline(self, pipeline, options): portable_options = options.view_as(PortableOptions) job_endpoint = portable_options.job_endpoint # TODO: https://issues.apache.org/jira/browse/BEAM-5525 # portable runner specific default if options.view_as(SetupOptions).sdk_location == 'default': options.view_as(SetupOptions).sdk_location = 'container' if not job_endpoint: # TODO Provide a way to specify a container Docker URL # https://issues.apache.org/jira/browse/BEAM-6328 docker = DockerizedJobServer() job_endpoint = docker.start() job_service = None elif job_endpoint == 'embed': job_service = local_job_service.LocalJobServicer() else: job_service = None # This is needed as we start a worker server if one is requested # but none is provided. if portable_options.environment_type == 'LOOPBACK': portable_options.environment_config, server = ( BeamFnExternalWorkerPoolServicer.start( sdk_worker_main._get_worker_count(options))) globals()['x'] = server cleanup_callbacks = [functools.partial(server.stop, 1)] else: cleanup_callbacks = [] proto_pipeline = pipeline.to_runner_api( default_environment=PortableRunner._create_environment( portable_options)) # Some runners won't detect the GroupByKey transform unless it has no # subtransforms. Remove all sub-transforms until BEAM-4605 is resolved. for _, transform_proto in list( proto_pipeline.components.transforms.items()): if transform_proto.spec.urn == common_urns.primitives.GROUP_BY_KEY.urn: for sub_transform in transform_proto.subtransforms: del proto_pipeline.components.transforms[sub_transform] del transform_proto.subtransforms[:] # Preemptively apply combiner lifting, until all runners support it. # This optimization is idempotent. pre_optimize = options.view_as(DebugOptions).lookup_experiment( 'pre_optimize', 'combine').lower() if not options.view_as(StandardOptions).streaming: flink_known_urns = frozenset([ common_urns.composites.RESHUFFLE.urn, common_urns.primitives.IMPULSE.urn, common_urns.primitives.FLATTEN.urn, common_urns.primitives.GROUP_BY_KEY.urn]) if pre_optimize == 'combine': proto_pipeline = fn_api_runner_transforms.optimize_pipeline( proto_pipeline, phases=[fn_api_runner_transforms.lift_combiners], known_runner_urns=flink_known_urns, partial=True) elif pre_optimize == 'all': proto_pipeline = fn_api_runner_transforms.optimize_pipeline( proto_pipeline, phases=[fn_api_runner_transforms.annotate_downstream_side_inputs, fn_api_runner_transforms.annotate_stateful_dofns_as_roots, fn_api_runner_transforms.fix_side_input_pcoll_coders, fn_api_runner_transforms.lift_combiners, fn_api_runner_transforms.fix_flatten_coders, # fn_api_runner_transforms.sink_flattens, fn_api_runner_transforms.greedily_fuse, fn_api_runner_transforms.read_to_impulse, fn_api_runner_transforms.extract_impulse_stages, fn_api_runner_transforms.remove_data_plane_ops, fn_api_runner_transforms.sort_stages], known_runner_urns=flink_known_urns) elif pre_optimize == 'none': pass else: raise ValueError('Unknown value for pre_optimize: %s' % pre_optimize) if not job_service: channel = grpc.insecure_channel(job_endpoint) grpc.channel_ready_future(channel).result() job_service = beam_job_api_pb2_grpc.JobServiceStub(channel) else: channel = None # fetch runner options from job service # retries in case the channel is not ready def send_options_request(max_retries=5): num_retries = 0 while True: try: # This reports channel is READY but connections may fail # Seems to be only an issue on Mac with port forwardings if channel: grpc.channel_ready_future(channel).result() return job_service.DescribePipelineOptions( beam_job_api_pb2.DescribePipelineOptionsRequest()) except grpc._channel._Rendezvous as e: num_retries += 1 if num_retries > max_retries: raise e time.sleep(1) options_response = send_options_request() def add_runner_options(parser): for option in options_response.options: try: # no default values - we don't want runner options # added unless they were specified by the user add_arg_args = {'action' : 'store', 'help' : option.description} if option.type == beam_job_api_pb2.PipelineOptionType.BOOLEAN: add_arg_args['action'] = 'store_true'\ if option.default_value != 'true' else 'store_false' elif option.type == beam_job_api_pb2.PipelineOptionType.INTEGER: add_arg_args['type'] = int elif option.type == beam_job_api_pb2.PipelineOptionType.ARRAY: add_arg_args['action'] = 'append' parser.add_argument("--%s" % option.name, **add_arg_args) except Exception as e: # ignore runner options that are already present # only in this case is duplicate not treated as error if 'conflicting option string' not in str(e): raise logging.debug("Runner option '%s' was already added" % option.name) all_options = options.get_all_options(add_extra_args_fn=add_runner_options) # TODO: Define URNs for options. # convert int values: https://issues.apache.org/jira/browse/BEAM-5509 p_options = {'beam:option:' + k + ':v1': (str(v) if type(v) == int else v) for k, v in all_options.items() if v is not None} prepare_response = job_service.Prepare( beam_job_api_pb2.PrepareJobRequest( job_name='job', pipeline=proto_pipeline, pipeline_options=job_utils.dict_to_struct(p_options))) if prepare_response.artifact_staging_endpoint.url: stager = portable_stager.PortableStager( grpc.insecure_channel(prepare_response.artifact_staging_endpoint.url), prepare_response.staging_session_token) retrieval_token, _ = stager.stage_job_resources( options, staging_location='') else: retrieval_token = None try: state_stream = job_service.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=prepare_response.preparation_id)) # If there's an error, we don't always get it until we try to read. # Fortunately, there's always an immediate current state published. state_stream = itertools.chain( [next(state_stream)], state_stream) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=prepare_response.preparation_id)) except Exception: # TODO(BEAM-6442): Unify preparation_id and job_id for all runners. state_stream = message_stream = None # Run the job and wait for a result. run_response = job_service.Run( beam_job_api_pb2.RunJobRequest( preparation_id=prepare_response.preparation_id, retrieval_token=retrieval_token)) if state_stream is None: state_stream = job_service.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=run_response.job_id)) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=run_response.job_id)) return PipelineResult(job_service, run_response.job_id, message_stream, state_stream, cleanup_callbacks)
def get_pipeline_options(self): # type: () -> struct_pb2.Struct """ Get `self.options` as a protobuf Struct """ # fetch runner options from job service # retries in case the channel is not ready def send_options_request(max_retries=5): num_retries = 0 while True: try: # This reports channel is READY but connections may fail # Seems to be only an issue on Mac with port forwardings return self.job_service.DescribePipelineOptions( beam_job_api_pb2.DescribePipelineOptionsRequest(), timeout=self.timeout) except grpc.FutureTimeoutError: # no retry for timeout errors raise except grpc._channel._Rendezvous as e: num_retries += 1 if num_retries > max_retries: raise e time.sleep(1) options_response = send_options_request() def add_runner_options(parser): for option in options_response.options: try: # no default values - we don't want runner options # added unless they were specified by the user add_arg_args = { 'action': 'store', 'help': option.description } if option.type == beam_job_api_pb2.PipelineOptionType.BOOLEAN: add_arg_args['action'] = 'store_true' \ if option.default_value != 'true' else 'store_false' elif option.type == beam_job_api_pb2.PipelineOptionType.INTEGER: add_arg_args['type'] = int elif option.type == beam_job_api_pb2.PipelineOptionType.ARRAY: add_arg_args['action'] = 'append' parser.add_argument("--%s" % option.name, **add_arg_args) except Exception as e: # ignore runner options that are already present # only in this case is duplicate not treated as error if 'conflicting option string' not in str(e): raise _LOGGER.debug("Runner option '%s' was already added" % option.name) all_options = self.options.get_all_options( add_extra_args_fn=add_runner_options, retain_unknown_options=self._retain_unknown_options) # TODO: Define URNs for options. # convert int values: https://issues.apache.org/jira/browse/BEAM-5509 p_options = { 'beam:option:' + k + ':v1': (str(v) if type(v) == int else v) for k, v in all_options.items() if v is not None } return job_utils.dict_to_struct(p_options)
def run_pipeline(self, pipeline, options): portable_options = options.view_as(PortableOptions) job_endpoint = portable_options.job_endpoint # TODO: https://issues.apache.org/jira/browse/BEAM-5525 # portable runner specific default if options.view_as(SetupOptions).sdk_location == 'default': options.view_as(SetupOptions).sdk_location = 'container' if not job_endpoint: if not self._job_endpoint: self.init_dockerized_job_server() job_endpoint = self._job_endpoint job_service = None elif job_endpoint == 'embed': job_service = local_job_service.LocalJobServicer() else: job_service = None # This is needed as we start a worker server if one is requested # but none is provided. if portable_options.environment_type == 'LOOPBACK': portable_options.environment_config, server = ( BeamFnExternalWorkerPoolServicer.start( sdk_worker_main._get_worker_count(options))) globals()['x'] = server cleanup_callbacks = [functools.partial(server.stop, 1)] else: cleanup_callbacks = [] proto_pipeline = pipeline.to_runner_api( default_environment=PortableRunner._create_environment( portable_options)) # Some runners won't detect the GroupByKey transform unless it has no # subtransforms. Remove all sub-transforms until BEAM-4605 is resolved. for _, transform_proto in list( proto_pipeline.components.transforms.items()): if transform_proto.spec.urn == common_urns.primitives.GROUP_BY_KEY.urn: for sub_transform in transform_proto.subtransforms: del proto_pipeline.components.transforms[sub_transform] del transform_proto.subtransforms[:] # Preemptively apply combiner lifting, until all runners support it. # This optimization is idempotent. pre_optimize = options.view_as(DebugOptions).lookup_experiment( 'pre_optimize', 'combine').lower() if not options.view_as(StandardOptions).streaming: flink_known_urns = frozenset([ common_urns.composites.RESHUFFLE.urn, common_urns.primitives.IMPULSE.urn, common_urns.primitives.FLATTEN.urn, common_urns.primitives.GROUP_BY_KEY.urn]) if pre_optimize == 'combine': proto_pipeline = fn_api_runner_transforms.optimize_pipeline( proto_pipeline, phases=[fn_api_runner_transforms.lift_combiners], known_runner_urns=flink_known_urns, partial=True) elif pre_optimize == 'all': proto_pipeline = fn_api_runner_transforms.optimize_pipeline( proto_pipeline, phases=[fn_api_runner_transforms.annotate_downstream_side_inputs, fn_api_runner_transforms.annotate_stateful_dofns_as_roots, fn_api_runner_transforms.fix_side_input_pcoll_coders, fn_api_runner_transforms.lift_combiners, fn_api_runner_transforms.fix_flatten_coders, # fn_api_runner_transforms.sink_flattens, fn_api_runner_transforms.greedily_fuse, fn_api_runner_transforms.read_to_impulse, fn_api_runner_transforms.extract_impulse_stages, fn_api_runner_transforms.remove_data_plane_ops, fn_api_runner_transforms.sort_stages], known_runner_urns=flink_known_urns) elif pre_optimize == 'none': pass else: raise ValueError('Unknown value for pre_optimize: %s' % pre_optimize) if not job_service: channel = grpc.insecure_channel(job_endpoint) grpc.channel_ready_future(channel).result() job_service = beam_job_api_pb2_grpc.JobServiceStub(channel) else: channel = None # fetch runner options from job service # retries in case the channel is not ready def send_options_request(max_retries=5): num_retries = 0 while True: try: # This reports channel is READY but connections may fail # Seems to be only an issue on Mac with port forwardings if channel: grpc.channel_ready_future(channel).result() return job_service.DescribePipelineOptions( beam_job_api_pb2.DescribePipelineOptionsRequest()) except grpc._channel._Rendezvous as e: num_retries += 1 if num_retries > max_retries: raise e time.sleep(1) options_response = send_options_request() def add_runner_options(parser): for option in options_response.options: try: # no default values - we don't want runner options # added unless they were specified by the user add_arg_args = {'action' : 'store', 'help' : option.description} if option.type == beam_job_api_pb2.PipelineOptionType.BOOLEAN: add_arg_args['action'] = 'store_true'\ if option.default_value != 'true' else 'store_false' elif option.type == beam_job_api_pb2.PipelineOptionType.INTEGER: add_arg_args['type'] = int elif option.type == beam_job_api_pb2.PipelineOptionType.ARRAY: add_arg_args['action'] = 'append' parser.add_argument("--%s" % option.name, **add_arg_args) except Exception as e: # ignore runner options that are already present # only in this case is duplicate not treated as error if 'conflicting option string' not in str(e): raise logging.debug("Runner option '%s' was already added" % option.name) all_options = options.get_all_options(add_extra_args_fn=add_runner_options) # TODO: Define URNs for options. # convert int values: https://issues.apache.org/jira/browse/BEAM-5509 p_options = {'beam:option:' + k + ':v1': (str(v) if type(v) == int else v) for k, v in all_options.items() if v is not None} prepare_response = job_service.Prepare( beam_job_api_pb2.PrepareJobRequest( job_name='job', pipeline=proto_pipeline, pipeline_options=job_utils.dict_to_struct(p_options))) if prepare_response.artifact_staging_endpoint.url: stager = portable_stager.PortableStager( grpc.insecure_channel(prepare_response.artifact_staging_endpoint.url), prepare_response.staging_session_token) retrieval_token, _ = stager.stage_job_resources( options, staging_location='') else: retrieval_token = None try: state_stream = job_service.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=prepare_response.preparation_id)) # If there's an error, we don't always get it until we try to read. # Fortunately, there's always an immediate current state published. state_stream = itertools.chain( [next(state_stream)], state_stream) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=prepare_response.preparation_id)) except Exception: # TODO(BEAM-6442): Unify preparation_id and job_id for all runners. state_stream = message_stream = None # Run the job and wait for a result. run_response = job_service.Run( beam_job_api_pb2.RunJobRequest( preparation_id=prepare_response.preparation_id, retrieval_token=retrieval_token)) if state_stream is None: state_stream = job_service.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=run_response.job_id)) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=run_response.job_id)) return PipelineResult(job_service, run_response.job_id, message_stream, state_stream, cleanup_callbacks)
def run_pipeline(self, pipeline, options): # type: (Pipeline, PipelineOptions) -> PipelineResult portable_options = options.view_as(PortableOptions) # TODO: https://issues.apache.org/jira/browse/BEAM-5525 # portable runner specific default if options.view_as(SetupOptions).sdk_location == 'default': options.view_as(SetupOptions).sdk_location = 'container' # This is needed as we start a worker server if one is requested # but none is provided. if portable_options.environment_type == 'LOOPBACK': use_loopback_process_worker = options.view_as( DebugOptions).lookup_experiment('use_loopback_process_worker', False) portable_options.environment_config, server = ( worker_pool_main.BeamFnExternalWorkerPoolServicer.start( state_cache_size=sdk_worker_main._get_state_cache_size( options), data_buffer_time_limit_ms=sdk_worker_main. _get_data_buffer_time_limit_ms(options), use_process=use_loopback_process_worker)) cleanup_callbacks = [functools.partial(server.stop, 1)] else: cleanup_callbacks = [] proto_pipeline = pipeline.to_runner_api( default_environment=PortableRunner._create_environment( portable_options)) # Some runners won't detect the GroupByKey transform unless it has no # subtransforms. Remove all sub-transforms until BEAM-4605 is resolved. for _, transform_proto in list( proto_pipeline.components.transforms.items()): if transform_proto.spec.urn == common_urns.primitives.GROUP_BY_KEY.urn: for sub_transform in transform_proto.subtransforms: del proto_pipeline.components.transforms[sub_transform] del transform_proto.subtransforms[:] # Preemptively apply combiner lifting, until all runners support it. # These optimizations commute and are idempotent. pre_optimize = options.view_as(DebugOptions).lookup_experiment( 'pre_optimize', 'lift_combiners').lower() if not options.view_as(StandardOptions).streaming: flink_known_urns = frozenset([ common_urns.composites.RESHUFFLE.urn, common_urns.primitives.IMPULSE.urn, common_urns.primitives.FLATTEN.urn, common_urns.primitives.GROUP_BY_KEY.urn ]) if pre_optimize == 'none': pass elif pre_optimize == 'all': proto_pipeline = fn_api_runner_transforms.optimize_pipeline( proto_pipeline, phases=[ fn_api_runner_transforms. annotate_downstream_side_inputs, fn_api_runner_transforms. annotate_stateful_dofns_as_roots, fn_api_runner_transforms.fix_side_input_pcoll_coders, fn_api_runner_transforms.lift_combiners, fn_api_runner_transforms.expand_sdf, fn_api_runner_transforms.fix_flatten_coders, # fn_api_runner_transforms.sink_flattens, fn_api_runner_transforms.greedily_fuse, fn_api_runner_transforms.read_to_impulse, fn_api_runner_transforms.extract_impulse_stages, fn_api_runner_transforms.remove_data_plane_ops, fn_api_runner_transforms.sort_stages ], known_runner_urns=flink_known_urns) else: phases = [] for phase_name in pre_optimize.split(','): # For now, these are all we allow. if phase_name in 'lift_combiners': phases.append( getattr(fn_api_runner_transforms, phase_name)) else: raise ValueError( 'Unknown or inapplicable phase for pre_optimize: %s' % phase_name) proto_pipeline = fn_api_runner_transforms.optimize_pipeline( proto_pipeline, phases=phases, known_runner_urns=flink_known_urns, partial=True) job_service = self.create_job_service(options) # fetch runner options from job service # retries in case the channel is not ready def send_options_request(max_retries=5): # type: (int) -> beam_job_api_pb2.DescribePipelineOptionsResponse num_retries = 0 while True: try: # This reports channel is READY but connections may fail # Seems to be only an issue on Mac with port forwardings return job_service.DescribePipelineOptions( beam_job_api_pb2.DescribePipelineOptionsRequest(), timeout=portable_options.job_server_timeout) except grpc.FutureTimeoutError: # no retry for timeout errors raise except grpc._channel._Rendezvous as e: num_retries += 1 if num_retries > max_retries: raise e time.sleep(1) options_response = send_options_request() def add_runner_options(parser): for option in options_response.options: try: # no default values - we don't want runner options # added unless they were specified by the user add_arg_args = { 'action': 'store', 'help': option.description } if option.type == beam_job_api_pb2.PipelineOptionType.BOOLEAN: add_arg_args['action'] = 'store_true'\ if option.default_value != 'true' else 'store_false' elif option.type == beam_job_api_pb2.PipelineOptionType.INTEGER: add_arg_args['type'] = int elif option.type == beam_job_api_pb2.PipelineOptionType.ARRAY: add_arg_args['action'] = 'append' parser.add_argument("--%s" % option.name, **add_arg_args) except Exception as e: # ignore runner options that are already present # only in this case is duplicate not treated as error if 'conflicting option string' not in str(e): raise _LOGGER.debug("Runner option '%s' was already added" % option.name) all_options = options.get_all_options( add_extra_args_fn=add_runner_options) # TODO: Define URNs for options. # convert int values: https://issues.apache.org/jira/browse/BEAM-5509 p_options = { 'beam:option:' + k + ':v1': (str(v) if type(v) == int else v) for k, v in all_options.items() if v is not None } prepare_request = beam_job_api_pb2.PrepareJobRequest( job_name='job', pipeline=proto_pipeline, pipeline_options=job_utils.dict_to_struct(p_options)) _LOGGER.debug('PrepareJobRequest: %s', prepare_request) prepare_response = job_service.Prepare( prepare_request, timeout=portable_options.job_server_timeout) artifact_endpoint = (portable_options.artifact_endpoint if portable_options.artifact_endpoint else prepare_response.artifact_staging_endpoint.url) if artifact_endpoint: stager = portable_stager.PortableStager( grpc.insecure_channel(artifact_endpoint), prepare_response.staging_session_token) retrieval_token, _ = stager.stage_job_resources( options, staging_location='') else: retrieval_token = None try: state_stream = job_service.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=prepare_response.preparation_id), timeout=portable_options.job_server_timeout) # If there's an error, we don't always get it until we try to read. # Fortunately, there's always an immediate current state published. state_stream = itertools.chain([next(state_stream)], state_stream) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=prepare_response.preparation_id), timeout=portable_options.job_server_timeout) except Exception: # TODO(BEAM-6442): Unify preparation_id and job_id for all runners. state_stream = message_stream = None # Run the job and wait for a result, we don't set a timeout here because # it may take a long time for a job to complete and streaming # jobs currently never return a response. run_response = job_service.Run( beam_job_api_pb2.RunJobRequest( preparation_id=prepare_response.preparation_id, retrieval_token=retrieval_token)) if state_stream is None: state_stream = job_service.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=run_response.job_id)) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=run_response.job_id)) result = PipelineResult(job_service, run_response.job_id, message_stream, state_stream, cleanup_callbacks) if cleanup_callbacks: # We wait here to ensure that we run the cleanup callbacks. logging.info( 'Waiting until the pipeline has finished because the ' 'environment "%s" has started a component necessary for the ' 'execution.', portable_options.environment_type) result.wait_until_finish() return result