def default_job_server(self, options): flink_master = options.view_as(FlinkRunnerOptions).flink_master if flink_master == '[local]' or sys.version_info < (3, 6): # TOOD(BEAM-8396): Also default to LOOPBACK for [local]. return job_server.StopOnExitJobServer(FlinkJarJobServer(options)) else: return flink_uber_jar_job_server.FlinkUberJarJobServer( flink_master)
def default_job_server(self, options): flink_master = options.view_as(FlinkRunnerOptions).flink_master if flink_master == '[local]' or sys.version_info < (3, 6): portable_options = options.view_as(pipeline_options.PortableOptions) if flink_master == '[local]' and not portable_options.environment_type: portable_options.environment_type == 'LOOPBACK' return job_server.StopOnExitJobServer(FlinkJarJobServer(options)) else: return flink_uber_jar_job_server.FlinkUberJarJobServer(flink_master)
def default_job_server(self, options): flink_master = self.add_http_scheme( options.view_as(FlinkRunnerOptions).flink_master) options.view_as(FlinkRunnerOptions).flink_master = flink_master if flink_master in MAGIC_HOST_NAMES or sys.version_info < (3, 6): return job_server.StopOnExitJobServer(FlinkJarJobServer(options)) else: # This has to be changed [auto], otherwise we will attempt to submit a # the pipeline remotely on the Flink JobMaster which will _fail_. # DO NOT CHANGE the following line, unless you have tested this. options.view_as(FlinkRunnerOptions).flink_master = '[auto]' return flink_uber_jar_job_server.FlinkUberJarJobServer(flink_master)
def test_bad_url_placeholder_version(self): options = pipeline_options.FlinkRunnerOptions() options.flink_job_server_jar = "bad url" job_server = flink_uber_jar_job_server.FlinkUberJarJobServer( 'http://example.com/bad', options) with self.assertRaises(ValueError) as context: job_server.executable_jar() self.assertEqual( 'Unable to parse jar URL "bad url". If using a full URL, make sure ' 'the scheme is specified. If using a local file path, make sure ' 'the file exists; you may have to first build the job server ' 'using `./gradlew runners:flink:$FLINK_VERSION:job-server:shadowJar`.', str(context.exception))
def test_bad_url_flink_version(self, http_mock): http_mock.get('http://flink/v1/config', json={'flink-version': '1.2.3.4'}) options = pipeline_options.FlinkRunnerOptions() options.flink_job_server_jar = "bad url" job_server = flink_uber_jar_job_server.FlinkUberJarJobServer( 'http://flink', options) with self.assertRaises(ValueError) as context: job_server.executable_jar() self.assertEqual( 'Unable to parse jar URL "bad url". If using a full URL, make sure ' 'the scheme is specified. If using a local file path, make sure ' 'the file exists; you may have to first build the job server ' 'using `./gradlew runners:flink:1.2:job-server:shadowJar`.', str(context.exception))
def default_job_server(self, options): flink_options = options.view_as(pipeline_options.FlinkRunnerOptions) flink_master = self.add_http_scheme(flink_options.flink_master) flink_options.flink_master = flink_master if (flink_options.flink_submit_uber_jar and flink_master not in MAGIC_HOST_NAMES): if sys.version_info < (3, 6): raise ValueError( 'flink_submit_uber_jar requires Python 3.6+, current version %s' % sys.version) # This has to be changed [auto], otherwise we will attempt to submit a # the pipeline remotely on the Flink JobMaster which will _fail_. # DO NOT CHANGE the following line, unless you have tested this. flink_options.flink_master = '[auto]' return flink_uber_jar_job_server.FlinkUberJarJobServer( flink_master, options) else: return job_server.StopOnExitJobServer(FlinkJarJobServer(options))
def test_get_job_metrics(self, http_mock): response = { "user-task-accumulators": [{ "name": "__metricscontainers", "type": "MetricsAccumulator", "value": "{\"metrics\": {\"attempted\": [{\"urn\": " "\"metric_urn\", \"type\": \"beam:metrics:sum_int64:v1\", " "\"payload\": \"AA==\", \"labels\": " "{\"PTRANSFORM\": \"ptransform_id\"}}]}}" }] } http_mock.get('http://flink/v1/jobs/flink_job_id/accumulators', json=response) options = pipeline_options.FlinkRunnerOptions() job_server = flink_uber_jar_job_server.FlinkUberJarJobServer( 'http://flink', options) job = flink_uber_jar_job_server.FlinkBeamJob('http://flink', None, 'job_id', 'job_name', None, options) job._flink_job_id = 'flink_job_id' job_server._jobs['job_id'] = job request = beam_job_api_pb2.GetJobMetricsRequest(job_id='job_id') expected = beam_job_api_pb2.GetJobMetricsResponse( metrics=beam_job_api_pb2.MetricResults(attempted=[{ "urn": "metric_urn", "type": "beam:metrics:sum_int64:v1", "payload": b'\000', "labels": { "PTRANSFORM": "ptransform_id" } }])) actual = job_server.GetJobMetrics(request) self.assertEqual(actual, expected)
def test_end_to_end(self, http_mock): with temp_name(suffix='fake.jar') as fake_jar: # Create the jar file with some trivial contents. with zipfile.ZipFile(fake_jar, 'w') as zip: with zip.open('FakeClass.class', 'w') as fout: fout.write(b'[original_contents]') job_server = flink_uber_jar_job_server.FlinkUberJarJobServer( 'http://flink', fake_jar) # Prepare the job. prepare_response = job_server.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() # Now actually run the job. http_mock.post('http://flink/v1/jars/upload', json={'filename': '/path/to/jar/nonce'}) http_mock.post('http://flink/v1/jars/nonce/run', json={'jobid': 'some_job_id'}) job_server.Run( beam_job_api_pb2.RunJobRequest( preparation_id=prepare_response.preparation_id, retrieval_token=retrieval_token)) # Check the status until the job is "done" and get all error messages. http_mock.get('http://flink/v1/jobs/some_job_id/execution-result', [{ 'json': { 'status': { 'id': 'IN_PROGRESS' } } }, { 'json': { 'status': { 'id': 'IN_PROGRESS' } } }, { 'json': { 'status': { 'id': 'COMPLETED' } } }]) http_mock.get('http://flink/v1/jobs/some_job_id', json={'state': 'FINISHED'}) http_mock.delete('http://flink/v1/jars/nonce') state_stream = job_server.GetStateStream( beam_job_api_pb2.GetJobStateRequest( job_id=prepare_response.preparation_id)) self.assertEqual([s.state for s in state_stream], [ beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.DONE ]) http_mock.get('http://flink/v1/jobs/some_job_id/exceptions', json={ 'all-exceptions': [{ 'exception': 'exc_text', 'timestamp': 0 }] }) message_stream = job_server.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=prepare_response.preparation_id)) self.assertEqual([m for m in message_stream], [ beam_job_api_pb2.JobMessagesResponse( message_response=beam_job_api_pb2.JobMessage( message_id='message0', time='0', importance=beam_job_api_pb2.JobMessage. MessageImportance.JOB_MESSAGE_ERROR, message_text='exc_text')), beam_job_api_pb2.JobMessagesResponse( state_response=beam_job_api_pb2.GetJobStateResponse( state=beam_job_api_pb2.JobState.DONE)), ])
def test_flink_version(self, http_mock): http_mock.get('http://flink/v1/config', json={'flink-version': '3.1.4.1'}) job_server = flink_uber_jar_job_server.FlinkUberJarJobServer( 'http://flink', None) self.assertEqual(job_server.flink_version(), "3.1")
def test_end_to_end(self, http_mock): with temp_name(suffix='fake.jar') as fake_jar: # Create the jar file with some trivial contents. with zipfile.ZipFile(fake_jar, 'w') as zip: with zip.open('FakeClass.class', 'w') as fout: fout.write(b'[original_contents]') options = pipeline_options.FlinkRunnerOptions() options.flink_job_server_jar = fake_jar job_server = flink_uber_jar_job_server.FlinkUberJarJobServer( 'http://flink', options) plan = TestJobServicePlan(job_server) # Prepare the job. prepare_response = plan.prepare(beam_runner_api_pb2.Pipeline()) plan.stage(beam_runner_api_pb2.Pipeline(), prepare_response.artifact_staging_endpoint.url, prepare_response.staging_session_token) # Now actually run the job. http_mock.post('http://flink/v1/jars/upload', json={'filename': '/path/to/jar/nonce'}) http_mock.post('http://flink/v1/jars/nonce/run', json={'jobid': 'some_job_id'}) _, message_stream, state_stream = plan.run( prepare_response.preparation_id) # Check the status until the job is "done" and get all error messages. http_mock.get('http://flink/v1/jobs/some_job_id/execution-result', [{ 'json': { 'status': { 'id': 'IN_PROGRESS' } } }, { 'json': { 'status': { 'id': 'IN_PROGRESS' } } }, { 'json': { 'status': { 'id': 'COMPLETED' } } }]) http_mock.get('http://flink/v1/jobs/some_job_id', json={'state': 'FINISHED'}) http_mock.delete('http://flink/v1/jars/nonce') self.assertEqual([s.state for s in state_stream], [ beam_job_api_pb2.JobState.STOPPED, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.DONE ]) http_mock.get('http://flink/v1/jobs/some_job_id/exceptions', json={ 'all-exceptions': [{ 'exception': 'exc_text', 'timestamp': 0 }] }) def get_item(x): if x.HasField('message_response'): return x.message_response else: return x.state_response.state self.assertEqual([get_item(m) for m in message_stream], [ beam_job_api_pb2.JobState.STOPPED, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobMessage( message_id='message0', time='0', importance=beam_job_api_pb2.JobMessage.MessageImportance. JOB_MESSAGE_ERROR, message_text='exc_text'), beam_job_api_pb2.JobState.DONE, ])