def _get_message_iter(self): """Returns an iterator of messages from the Spark server. Note that while message history is de-duped, this function's returned iterator may contain duplicate values.""" sleep_secs = 1.0 message_ix = 0 while True: response = self._get_spark_status() state = self._get_beam_state(response) timestamp = Timestamp.now() message = None if 'message' in response: importance = ( beam_job_api_pb2.JobMessage.MessageImportance.JOB_MESSAGE_ERROR if state == beam_job_api_pb2.JobState.FAILED else beam_job_api_pb2.JobMessage.MessageImportance.JOB_MESSAGE_BASIC) message = beam_job_api_pb2.JobMessage( message_id='message%d' % message_ix, time=str(int(timestamp)), importance=importance, message_text=response['message']) yield message message_ix += 1 # TODO(BEAM-8983) In the event of a failure, query # additional info from Spark master and/or workers. check_timestamp = self.set_state(state) if check_timestamp is not None: if message: self._message_history.append(message) self._message_history.append((state, check_timestamp)) yield state, timestamp sleep_secs = min(60, sleep_secs * 1.2) time.sleep(sleep_secs)
def emit(self, record): if self._logged_thread is threading.current_thread(): self._message_queue.put(beam_job_api_pb2.JobMessagesResponse( message_response=beam_job_api_pb2.JobMessage( message_id=self._next_id(), time=time.strftime( '%Y-%m-%d %H:%M:%S.', time.localtime(record.created)), importance=self.LOG_LEVEL_MAP[record.levelno], message_text=self.format(record))))
def emit(self, record): if self._logged_thread is threading.current_thread(): msg = beam_job_api_pb2.JobMessage( message_id=self._next_id(), time=time.strftime('%Y-%m-%d %H:%M:%S.', time.localtime(record.created)), importance=self.LOG_LEVEL_MAP[record.levelno], message_text=self.format(record)) # Inform all message consumers. self._log_queues.put(msg)
def get_message_stream(self): for state in self.get_state_stream(): if self.is_terminal_state(state): response = self.get('v1/jobs/%s/exceptions' % self._flink_job_id) for ix, exc in enumerate(response['all-exceptions']): yield beam_job_api_pb2.JobMessage( message_id='message%d' % ix, time=str(exc['timestamp']), importance= beam_job_api_pb2.JobMessage.MessageImportance.JOB_MESSAGE_ERROR, message_text=exc['exception']) yield state
def _run_job(self): with JobLogHandler(self._log_queues) as log_handler: self._update_dependencies() try: start = time.time() result = self._invoke_runner() _LOGGER.info('Successfully completed job in %s seconds.', time.time() - start) self.set_state(beam_job_api_pb2.JobState.DONE) self.result = result except: # pylint: disable=bare-except self._log_queues.put( beam_job_api_pb2.JobMessage( message_id=log_handler._next_id(), time=time.strftime('%Y-%m-%d %H:%M:%S.'), importance=beam_job_api_pb2.JobMessage. JOB_MESSAGE_ERROR, message_text=traceback.format_exc())) _LOGGER.exception('Error running pipeline.') self.set_state(beam_job_api_pb2.JobState.FAILED) raise
def test_end_to_end(self, http_mock): submission_id = "submission-id" worker_host_port = "workerhost:12345" worker_id = "worker-id" server_spark_version = "1.2.3" def spark_submission_status_response(state): return { 'json': { "action": "SubmissionStatusResponse", "driverState": state, "serverSparkVersion": server_spark_version, "submissionId": submission_id, "success": "true", "workerHostPort": worker_host_port, "workerId": worker_id } } with temp_name(suffix='fake.jar') as fake_jar: with zipfile.ZipFile(fake_jar, 'w') as zip: with zip.open('spark-version-info.properties', 'w') as fout: fout.write(b'version=4.5.6') options = pipeline_options.SparkRunnerOptions() options.spark_job_server_jar = fake_jar job_server = spark_uber_jar_job_server.SparkUberJarJobServer( 'http://host:6066', options) # Prepare the job. plan = TestJobServicePlan(job_server) # Prepare the job. prepare_response = plan.prepare(beam_runner_api_pb2.Pipeline()) retrieval_token = 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://host:6066/v1/submissions/create', json={ "action": "CreateSubmissionResponse", "message": "Driver successfully submitted as submission-id", "serverSparkVersion": "1.2.3", "submissionId": "submission-id", "success": "true" }) 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://host:6066/v1/submissions/status/submission-id', [ spark_submission_status_response('RUNNING'), spark_submission_status_response('RUNNING'), { 'json': { "action": "SubmissionStatusResponse", "driverState": "ERROR", "message": "oops", "serverSparkVersion": "1.2.3", "submissionId": submission_id, "success": "true", "workerHostPort": worker_host_port, "workerId": worker_id } } ]) 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.STOPPED, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.RUNNING, beam_job_api_pb2.JobState.FAILED ]) message_stream = job_server.GetMessageStream( beam_job_api_pb2.JobMessagesRequest( job_id=prepare_response.preparation_id)) 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="oops"), beam_job_api_pb2.JobState.FAILED, ])
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_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, ])