def test_end_to_end(self): job_service = local_job_service.LocalJobServicer() job_service.start_grpc_server() plan = TestJobServicePlan(job_service) _, message_stream, state_stream = plan.submit( beam_runner_api_pb2.Pipeline()) state_results = list(state_stream) message_results = list(message_stream) expected_states = [ beam_job_api_pb2.JobState.STOPPED, beam_job_api_pb2.JobState.STARTING, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.DONE, ] self.assertEqual([s.state for s in state_results], expected_states) self.assertEqual([ s.state_response.state for s in message_results if s.HasField('state_response') ], expected_states)
def test_error_messages_after_pipeline_failure(self): job_service = local_job_service.LocalJobServicer() job_service.start_grpc_server() plan = TestJobServicePlan(job_service) job_id, message_stream, state_stream = plan.submit( beam_runner_api_pb2.Pipeline( requirements=['unsupported_requirement'])) message_results = list(message_stream) state_results = list(state_stream) expected_states = [ beam_job_api_pb2.JobState.STOPPED, beam_job_api_pb2.JobState.STARTING, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.FAILED, ] self.assertEqual([s.state for s in state_results], expected_states) self.assertTrue( any('unsupported_requirement' in m.message_response.message_text for m in message_results), message_results) # Assert we still see the error message if we fetch error messages after # the job has completed. messages_again = list( plan.job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest(job_id=job_id))) self.assertTrue( any('unsupported_requirement' in m.message_response.message_text for m in message_results), messages_again)
def test_artifact_service_override(self): job_service = local_job_service.LocalJobServicer() port = job_service.start_grpc_server() test_artifact_endpoint = 'testartifactendpoint:4242' options = pipeline_options.PipelineOptions([ '--job_endpoint', 'localhost:%d' % port, '--artifact_endpoint', test_artifact_endpoint, ]) runner = PortableRunner() job_service_handle = runner.create_job_service(options) with mock.patch.object(job_service_handle, 'stage') as mocked_stage: job_service_handle.submit(beam_runner_api_pb2.Pipeline()) mocked_stage.assert_called_once_with(mock.ANY, test_artifact_endpoint, mock.ANY) # Confirm the artifact_endpoint is in the options protobuf options_proto = job_service_handle.get_pipeline_options() self.assertEqual(options_proto['beam:option:artifact_endpoint:v1'], test_artifact_endpoint)
def test_end_to_end(self): job_service = local_job_service.LocalJobServicer() job_service.start_grpc_server() # this logic is taken roughly from PortableRunner.run_pipeline() # Prepare the job. prepare_response = job_service.Prepare( beam_job_api_pb2.PrepareJobRequest( job_name='job', pipeline=beam_runner_api_pb2.Pipeline())) channel = grpc.insecure_channel( prepare_response.artifact_staging_endpoint.url) retrieval_token = beam_artifact_api_pb2_grpc.ArtifactStagingServiceStub( channel).CommitManifest( beam_artifact_api_pb2.CommitManifestRequest( staging_session_token=prepare_response. staging_session_token, manifest=beam_artifact_api_pb2.Manifest())).retrieval_token channel.close() 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_results.append(next(state_stream)) state_stream = increment_iter(state_stream) message_stream = job_service.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=prepare_response.preparation_id)) job_service.Run( beam_job_api_pb2.RunJobRequest( preparation_id=prepare_response.preparation_id, retrieval_token=retrieval_token)) state_results = list(state_stream) message_results = list(message_stream) expected_states = [ beam_job_api_pb2.JobState.STOPPED, beam_job_api_pb2.JobState.STARTING, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.DONE, ] self.assertEqual([s.state for s in state_results], expected_states) self.assertEqual([s.state_response.state for s in message_results], expected_states)
def run(argv): if argv[0] == __file__: argv = argv[1:] parser = argparse.ArgumentParser() parser.add_argument('-p', '--port', type=int, help='port on which to serve the job api') options = parser.parse_args(argv) job_servicer = local_job_service.LocalJobServicer() port = job_servicer.start_grpc_server(options.port) while True: logging.info("Listening for jobs at %d", port) time.sleep(300)
def test_dry_run(self): # Not an integration test that actually runs on Dataflow, # but does exercise (most of) the translation and setup code, # as well as the connection. job_servicer = local_job_service.LocalJobServicer( None, beam_job_type=dataflow_job_service.DataflowBeamJob) port = job_servicer.start_grpc_server(0) try: options = PipelineOptions( runner='PortableRunner', job_endpoint=f'localhost:{port}', project='some_project', temp_location='gs://bucket/dir', region='us-central1', dry_run=True, ) with beam.Pipeline(options=options) as p: _ = p | beam.Create([1, 2, 3]) | beam.Map(lambda x: x * x) finally: job_servicer.stop()
def run(argv, beam_job_type=DataflowBeamJob): if argv[0] == __file__: argv = argv[1:] parser = argparse.ArgumentParser() parser.add_argument('-p', '--port', '--job_port', type=int, default=0, help='port on which to serve the job api') parser.add_argument('--staging_dir') options = parser.parse_args(argv) job_servicer = local_job_service.LocalJobServicer( options.staging_dir, beam_job_type=beam_job_type) port = job_servicer.start_grpc_server(options.port) try: local_job_service_main.serve( "Listening for beam jobs on port %d." % port, job_servicer) finally: job_servicer.stop()
def run(argv): if argv[0] == __file__: argv = argv[1:] parser = argparse.ArgumentParser() parser.add_argument('-p', '--port', '--job_port', type=int, help='port on which to serve the job api') parser.add_argument('--staging_dir') options = parser.parse_args(argv) job_servicer = local_job_service.LocalJobServicer(options.staging_dir) port = job_servicer.start_grpc_server(options.port) delay = 60 try: while True: _LOGGER.info("Listening for jobs on port %d.", port) time.sleep(delay) delay *= 1.25 finally: job_servicer.stop()
def start(self): # type: () -> local_job_service.LocalJobServicer return local_job_service.LocalJobServicer()
def run(argv): if argv[0] == __file__: argv = argv[1:] parser = argparse.ArgumentParser() parser.add_argument('-p', '--port', '--job_port', type=int, default=0, help='port on which to serve the job api') parser.add_argument('--staging_dir') parser.add_argument( '--pid_file', help='File in which to store the process id of the server.') parser.add_argument('--port_file', help='File in which to store the port of the server.') parser.add_argument( '--background', action='store_true', help='Start the server up as a background process.' ' Will fail if pid_file already exists, unless --stop is also specified.' ) parser.add_argument( '--stderr_file', help='Where to write stderr (if not specified, merged with stdout).') parser.add_argument( '--stdout_file', help='Where to write stdout for background job service.') parser.add_argument( '--stop', action='store_true', help='Stop the existing process, if any, specified in pid_file.' ' Will not start up a new service unless --background is specified.') options = parser.parse_args(argv) if options.stop: if not options.pid_file: raise RuntimeError('--pid_file must be specified with --stop') if os.path.exists(options.pid_file): with open(options.pid_file) as fin: pid = int(fin.read()) print('Killing process at', pid) try: os.kill(pid, signal.SIGTERM) except Exception: print('Process', pid, 'already killed.') os.unlink(options.pid_file) else: print('Process id file', options.pid_file, 'already removed.') if not options.background: return if options.background: if not options.pid_file: raise RuntimeError('--pid_file must be specified with --start') if options.stop: argv.remove('--stop') argv.remove('--background') if not options.port_file: options.port_file = os.path.splitext(options.pid_file)[0] + '.port' argv.append('--port_file') argv.append(options.port_file) if not options.stdout_file: raise RuntimeError( '--stdout_file must be specified with --background') stdout_dest = open(options.stdout_file, mode='w') if options.stderr_file: stderr_dest = open(options.stderr_file, mode='w') else: stderr_dest = subprocess.STDOUT subprocess.Popen([ sys.executable, '-m', 'apache_beam.runners.portability.local_job_service_main' ] + argv, stderr=stderr_dest, stdout=stdout_dest) print('Waiting for server to start up...') while not os.path.exists(options.port_file): time.sleep(.1) with open(options.port_file) as fin: port = fin.read() print('Server started at port', port) return if options.pid_file: print('Writing process id to', options.pid_file) fd = os.open(options.pid_file, os.O_CREAT | os.O_EXCL | os.O_RDWR) with os.fdopen(fd, 'w') as fout: fout.write(str(os.getpid())) try: job_servicer = local_job_service.LocalJobServicer(options.staging_dir) port = job_servicer.start_grpc_server(options.port) try: if options.port_file: print('Writing port to', options.port_file) with open(options.port_file + '.tmp', 'w') as fout: fout.write(str(port)) os.rename(options.port_file + '.tmp', options.port_file) serve("Listening for beam jobs on port %d." % port, job_servicer) finally: job_servicer.stop() finally: if options.pid_file and os.path.exists(options.pid_file): os.unlink(options.pid_file) if options.port_file and os.path.exists(options.port_file): os.unlink(options.port_file)
def start(self): return local_job_service.LocalJobServicer()
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 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): 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))) 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. if not options.view_as(StandardOptions).streaming: stages = list(fn_api_runner_transforms.leaf_transform_stages( proto_pipeline.root_transform_ids, proto_pipeline.components)) stages = fn_api_runner_transforms.lift_combiners( stages, fn_api_runner_transforms.TransformContext(proto_pipeline.components)) proto_pipeline = fn_api_runner_transforms.with_stages( proto_pipeline, stages) # 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 options.get_all_options().items() if v is not None} 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 # Sends the PrepareRequest but retries in case the channel is not ready 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 if channel: 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(p_options))) except grpc._channel._Rendezvous as e: num_retries += 1 if num_retries > max_retries: raise e prepare_response = send_prepare_request() 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)