From e9ac3c27b83c4318d72da6a1fe530ed27f31a5dd Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 16 Jul 2021 12:09:28 -0500 Subject: [PATCH 01/31] Refactor to bulk insert --- lib/execution_engine2/db/MongoUtil.py | 49 +++++- lib/execution_engine2/sdk/EE2Logs.py | 4 +- lib/execution_engine2/sdk/EE2Runjob.py | 148 ++++++++++++++---- lib/execution_engine2/sdk/SDKMethodRunner.py | 14 +- test/tests_for_sdkmr/EE2Runjob_test.py | 25 +-- .../ee2_SDKMethodRunner_test.py | 10 +- 6 files changed, 198 insertions(+), 52 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 1499cf267..bf33b80b5 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -3,15 +3,15 @@ import time import traceback from contextlib import contextmanager -from typing import Dict +from typing import Dict, List from bson.objectid import ObjectId from mongoengine import connect, connection -from pymongo import MongoClient +from pymongo import MongoClient, UpdateOne from pymongo.errors import ServerSelectionTimeoutError -from lib.execution_engine2.db.models.models import JobLog, Job, Status, TerminatedCode -from lib.execution_engine2.exceptions import ( +from execution_engine2.db.models.models import JobLog, Job, Status, TerminatedCode +from execution_engine2.exceptions import ( RecordNotFoundException, InvalidStatusTransitionException, ) @@ -263,6 +263,27 @@ def check_if_already_finished(job_status): return True return False + def update_jobs_to_queued(self, job_ids, scheduler_ids): + bulk_operations = [] + now = time.time() + for i, job_id in enumerate(job_ids): + bulk_operations.append( + UpdateOne( + {"_id": job_id}, + { + "$set": { + "status": Status.queued.value, + "queued": now, + "scheduler_ids": scheduler_ids[i], + "scheduler_type": "condor", + } + }, + ) + ) + + # TODO Save it + # TODO ordered false + def cancel_job(self, job_id=None, terminated_code=None): """ #TODO Should we check for a valid state transition here also? @@ -420,6 +441,26 @@ def update_job_status(self, job_id, status, msg=None, error_message=None): def mongo_engine_connection(self): yield self.me_connection + def insert_jobs(self, jobs_to_insert: List[Job]) -> List[ObjectId]: + """ + Insert multiple job records using MongoEngine + :param jobs_to_insert: Multiple jobs to insert at once + :return: List of job ids from the insertion + """ + # TODO Look at pymongo write_concerns that may be useful + # TODO see if pymongo is faster + + inserted = Job.objects.insert(doc_or_docs=jobs_to_insert, load_bulk=False) + + return inserted + # TODO: Send Kafka messages here or elsewhere + # job_id = self.sdkmr.save_job(job) + # self.sdkmr.get_kafka_client().send_kafka_message( + # message=KafkaCreateJob(job_id=job_id, user=user_id) + # ) + + # TODO: Think about error handling + def insert_one(self, doc): """ insert a doc into collection diff --git a/lib/execution_engine2/sdk/EE2Logs.py b/lib/execution_engine2/sdk/EE2Logs.py index d96acb0b2..e71b705ff 100644 --- a/lib/execution_engine2/sdk/EE2Logs.py +++ b/lib/execution_engine2/sdk/EE2Logs.py @@ -1,8 +1,8 @@ from enum import Enum from typing import Dict, NamedTuple -from lib.execution_engine2.db.models.models import JobLog as JLModel, LogLines -from lib.execution_engine2.exceptions import RecordNotFoundException +from execution_engine2.db.models.models import JobLog as JLModel, LogLines +from execution_engine2.exceptions import RecordNotFoundException # if TYPE_CHECKING: diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index 3fb17588e..1d9aa3866 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -87,10 +87,8 @@ def __init__(self, sdkmr): self.logger = self.sdkmr.get_logger() def _init_job_rec( - self, - user_id: str, - params: Dict, - ) -> str: + self, user_id: str, params: Dict, save: bool = True + ) -> Union[str, Job]: f""" Save an initial job record to the db and send a message to kafka @@ -163,11 +161,13 @@ def _init_job_rec( job.retry_parent = str(parent_retry_job_id) job.batch_id = str(params.get(_BATCH_ID)) if params.get(_BATCH_ID) else None - job_id = self.sdkmr.save_job(job) - self.sdkmr.get_kafka_client().send_kafka_message( - message=KafkaCreateJob(job_id=job_id, user=user_id) - ) - return job_id + if save: + job_id = self.sdkmr.save_job(job) + self.sdkmr.get_kafka_client().send_kafka_message( + message=KafkaCreateJob(job_id=job_id, user=user_id) + ) + return job_id + return job def _check_ws_objects(self, source_objects) -> None: """ @@ -234,17 +234,7 @@ def _finish_created_job( error=f"{exception}", ) - def _prepare_to_run(self, params, concierge_params=None) -> JobSubmissionParameters: - """ - Creates a job record and creates the job submission params - """ - - job_id = self._init_job_rec(self.sdkmr.get_user_id(), params) - - self.logger.debug( - f"User {self.sdkmr.get_user_id()} attempting to run job {params[_METHOD]} {params}" - ) - + def _generate_job_submission_params(self, job_id, params): return JobSubmissionParameters( job_id, AppInfo(params[_METHOD], params.get(_APP_ID)), @@ -258,6 +248,101 @@ def _prepare_to_run(self, params, concierge_params=None) -> JobSubmissionParamet source_ws_objects=params.get(_SOURCE_WS_OBJECTS), ) + def _prepare_to_run(self, params, concierge_params=None) -> JobSubmissionParameters: + """ + Creates a job record and creates the job submission params + """ + + job_id = self._init_job_rec(self.sdkmr.get_user_id(), params) + self.logger.debug( + f"User {self.sdkmr.get_user_id()} attempting to run job {params[_METHOD]} {params}" + ) + return self._generate_job_submission_params(job_id, params) + + def _run_multiple(self, runjob_params): + """ + Get the job records, bulk save them, then submit to condor + :return: + """ + job_records = [] + for runjob_param in runjob_params: + job_records.append( + self._init_job_rec(self.sdkmr.get_user_id(), runjob_param, save=False) + ) + + job_ids = self.sdkmr.save_jobs(job_records) + + # TODO: Test to see job ids match rjp? + job_submission_params = [] + for i, job_id in enumerate(job_ids): + job_submission_params.append( + self._generate_job_submission_params(job_id, runjob_params[i]) + ) + + return self._submit_multiple(job_submission_params) + + def _update_to_queued_multiple(self, job_ids, scheduler_ids): + # TODO Unused + if len(job_ids) != len(scheduler_ids): + raise Exception( + "Need to provide the same amount of job ids and scheduler_ids" + ) + + # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL + # TODO PASS QUEUE TIME IN FROM SCHEDULER ITSELF? + # TODO PASS IN SCHEDULER TYPE? + self.sdkmr.get_mongo_util().update_jobs_to_queued( + job_ids=job_ids, scheduler_ids=scheduler_ids + ) + + # TODO figure out kafka message + for i, job_id in enumerate(job_ids): + + self.sdkmr.get_kafka_client().send_kafka_message( + message=KafkaQueueChange( + job_id=job_id, + new_status=Status.queued.value, + previous_status=Status.created.value, # TODO maybe change this to allow for estimating jobs + scheduler_id=scheduler_ids[i], + ) + ) + + def _submit_multiple(self, job_submission_params): + job_ids = [] + condor_job_ids = [] + for job_submit_param in job_submission_params: + job_id = job_submit_param.job_id + job_ids.append(job_id) + try: + submission_info = self.sdkmr.get_condor().run_job( + params=job_submit_param + ) + condor_job_id = submission_info.clusterid + self.logger.debug(f"Submitted job id and got '{condor_job_id}'") + except Exception as e: + self.logger.error(e) + self._finish_created_job(job_id=job_id, exception=e) + raise e + + if submission_info.error is not None and isinstance( + submission_info.error, Exception + ): + self._finish_created_job(exception=submission_info.error, job_id=job_id) + raise submission_info.error + if condor_job_id is None: + error_msg = ( + "Condor job not run, and error not found. Something went wrong" + ) + self._finish_created_job( + job_id=job_id, exception=RuntimeError(error_msg) + ) + raise RuntimeError(error_msg) + + self.update_job_to_queued(job_id=job_id, scheduler_id=condor_job_id) + condor_job_ids.append(condor_job_id) + + return job_ids + def _run(self, params): job_params = self._prepare_to_run(params=params) job_id = job_params.job_id @@ -337,14 +422,21 @@ def _run_batch(self, batch_job: Job, params): for job_param in params: job_param[_BATCH_ID] = str(batch_job.id) - try: - child_jobs.append(str(self._run(params=job_param))) - except Exception as e: - self.logger.debug( - msg=f"Failed to submit child job. Aborting entire batch job {e}" - ) - self._abort_child_jobs(child_jobs) - raise e + + try: + child_jobs = self._run_multiple(params) + except Exception as e: + # See if we can abort any of these child jobs + raise e + + # try: + # child_jobs.append(str(self._run(params=job_param))) + # except Exception as e: + # self.logger.debug( + # msg=f"Failed to submit child job. Aborting entire batch job {e}" + # ) + # self._abort_child_jobs(child_jobs) + # raise e batch_job.child_jobs = child_jobs self.sdkmr.save_job(batch_job) diff --git a/lib/execution_engine2/sdk/SDKMethodRunner.py b/lib/execution_engine2/sdk/SDKMethodRunner.py index 5091b2fee..c3d85fb66 100644 --- a/lib/execution_engine2/sdk/SDKMethodRunner.py +++ b/lib/execution_engine2/sdk/SDKMethodRunner.py @@ -12,13 +12,14 @@ from datetime import datetime from enum import Enum from logging import Logger +from typing import List import dateutil -from lib.execution_engine2.db.MongoUtil import MongoUtil +from execution_engine2.db.MongoUtil import MongoUtil from execution_engine2.db.models.models import Job -from lib.execution_engine2.exceptions import AuthError -from lib.execution_engine2.sdk import ( +from execution_engine2.exceptions import AuthError +from execution_engine2.sdk import ( EE2Runjob, EE2StatusRange, EE2Authentication, @@ -251,6 +252,13 @@ def check_as_concierge(self): # at this point since MongoEngine creates a global connection to MongoDB # and makes it available to all the model objects. + def save_jobs(self, jobs: List[Job]) -> List[str]: + """ + Save multiple jobs to the Mongo DB at once, and return all of the job ids + """ + job_ids = self.get_mongo_util().insert_jobs(jobs_to_insert=jobs) + return [str(job_id) for job_id in job_ids] + def save_job(self, job: Job) -> str: """ Save a job record to the Mongo database and return the job's ID as a string. diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 3698c9124..9d934e51b 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -788,6 +788,11 @@ def _set_up_common_return_values_batch(mocks): None, None, ] + + mocks[SDKMethodRunner].save_jobs.side_effect = [ + [_JOB_ID_1, _JOB_ID_2], + ] + mocks[Condor].run_job.side_effect = [ SubmissionInfo(_CLUSTER_1, {}, None), SubmissionInfo(_CLUSTER_2, {}, None), @@ -834,7 +839,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ] ) - assert len(sdkmr.save_job.call_args_list) == 5 + assert len(sdkmr.save_job.call_args_list) == 3 # initial child jobs data save expected_job_1 = _create_job( @@ -846,7 +851,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): wsid=parent_wsid, batch_id=_JOB_ID, ) - got_job_1 = sdkmr.save_job.call_args_list[0][0][0] + got_job_1 = sdkmr.save_jobs.call_args_list[0][0][0][0] assert_jobs_equal(got_job_1, expected_job_1) expected_job_2 = _create_job( @@ -857,8 +862,8 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): wsid=parent_wsid, batch_id=_JOB_ID, ) - # index 2 because job 1 is updated with save_job before this job is created - got_job_2 = sdkmr.save_job.call_args_list[2][0][0] + # index 1 because save_jobs returns a list of two jobs + got_job_2 = sdkmr.save_jobs.call_args_list[0][0][0][1] assert_jobs_equal(got_job_2, expected_job_2) jsp_expected_1 = JobSubmissionParameters( @@ -886,15 +891,15 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): mocks[MongoUtil].get_job.assert_has_calls([call(_JOB_ID_1), call(_JOB_ID_2)]) # update to queued state - got_queued_job_1 = sdkmr.save_job.call_args_list[1][0][0] - got_queued_job_2 = sdkmr.save_job.call_args_list[3][0][0] + got_queued_job_1 = sdkmr.save_job.call_args_list[0][0][0] + got_queued_job_2 = sdkmr.save_job.call_args_list[1][0][0] _check_queued_job_save(got_queued_job_1, _JOB_ID_1, _CLUSTER_1) _check_queued_job_save(got_queued_job_2, _JOB_ID_2, _CLUSTER_2) mocks[KafkaClient].send_kafka_message.assert_has_calls( [ - call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job - call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), + # call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job + # call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), call( KafkaQueueChange( job_id=_JOB_ID_1, @@ -903,7 +908,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): scheduler_id=_CLUSTER_1, ) ), - call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), + # call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), call( KafkaQueueChange( job_id=_JOB_ID_2, @@ -927,7 +932,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): final_expected_parent_job.id = ObjectId(_JOB_ID) final_expected_parent_job.user = _USER final_expected_parent_job.child_jobs = [_JOB_ID_1, _JOB_ID_2] - final_got_parent_job = sdkmr.save_job.call_args_list[4][0][0] + final_got_parent_job = sdkmr.save_job.call_args_list[2][0][0] assert_jobs_equal(final_got_parent_job, final_expected_parent_job) diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py index a90044703..b8b96c187 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py @@ -30,10 +30,10 @@ JobRequirementsResolver, RequirementsType, ) -from lib.execution_engine2.db.models.models import Job, Status, TerminatedCode -from lib.execution_engine2.exceptions import InvalidStatusTransitionException -from lib.execution_engine2.sdk.SDKMethodRunner import SDKMethodRunner -from lib.execution_engine2.utils.CondorTuples import SubmissionInfo +from execution_engine2.db.models.models import Job, Status, TerminatedCode +from execution_engine2.exceptions import InvalidStatusTransitionException +from execution_engine2.sdk.SDKMethodRunner import SDKMethodRunner +from execution_engine2.utils.CondorTuples import SubmissionInfo from test.tests_for_sdkmr.ee2_SDKMethodRunner_test_utils import ee2_sdkmr_test_helper from test.utils_shared.mock_utils import get_client_mocks, ALL_CLIENTS from test.utils_shared.test_utils import ( @@ -48,7 +48,7 @@ logging.basicConfig(level=logging.INFO) bootstrap() -from lib.execution_engine2.sdk.EE2Runjob import EE2RunJob +from execution_engine2.sdk.EE2Runjob import EE2RunJob from installed_clients.CatalogClient import Catalog from installed_clients.WorkspaceClient import Workspace From b3884a6e2244c845f1463184ff281b12dc6126dc Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 16 Jul 2021 15:33:50 -0500 Subject: [PATCH 02/31] Refactor to bulk update --- lib/execution_engine2/db/MongoUtil.py | 14 +++++-- lib/execution_engine2/sdk/EE2Runjob.py | 56 +++++++++++++------------- test/tests_for_sdkmr/EE2Runjob_test.py | 23 +++++------ 3 files changed, 46 insertions(+), 47 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index bf33b80b5..04745d093 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -269,20 +269,26 @@ def update_jobs_to_queued(self, job_ids, scheduler_ids): for i, job_id in enumerate(job_ids): bulk_operations.append( UpdateOne( - {"_id": job_id}, + {"_id": ObjectId(job_id)}, { "$set": { "status": Status.queued.value, "queued": now, - "scheduler_ids": scheduler_ids[i], + "scheduler_id": scheduler_ids[i], "scheduler_type": "condor", } }, ) ) + mongo_collection = self.config["mongo-jobs-collection"] + if bulk_operations: + with self.pymongo_client(mongo_collection) as pymongo_client: + bwr = pymongo_client[self.mongo_database][mongo_collection].bulk_write( + bulk_operations, ordered=False + ) + assert bwr.modified_count == len(job_ids) - # TODO Save it - # TODO ordered false + # TODO error handling for bulk write result def cancel_job(self, job_id=None, terminated_code=None): """ diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index 1d9aa3866..f1c0b1ffa 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -261,25 +261,36 @@ def _prepare_to_run(self, params, concierge_params=None) -> JobSubmissionParamet def _run_multiple(self, runjob_params): """ - Get the job records, bulk save them, then submit to condor + Get the job records, bulk save them, then submit to condor. + If any condor submission fails, abort all of the jobs :return: """ + # Save records to db job_records = [] for runjob_param in runjob_params: job_records.append( self._init_job_rec(self.sdkmr.get_user_id(), runjob_param, save=False) ) - job_ids = self.sdkmr.save_jobs(job_records) - # TODO: Test to see job ids match rjp? + # Generate job submission params job_submission_params = [] for i, job_id in enumerate(job_ids): job_submission_params.append( self._generate_job_submission_params(job_id, runjob_params[i]) ) - - return self._submit_multiple(job_submission_params) + assert job_id == job_submission_params[i].job_id + self.sdkmr.get_kafka_client().send_kafka_message( + message=KafkaCreateJob( + job_id=str(job_id), user=self.sdkmr.get_user_id() + ) + ) + # Submit to Condor + try: + return self._submit_multiple(job_submission_params) + except Exception as e: + self._abort_multiple_jobs(job_ids) + raise e def _update_to_queued_multiple(self, job_ids, scheduler_ids): # TODO Unused @@ -308,6 +319,10 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): ) def _submit_multiple(self, job_submission_params): + """ + Submit multiple jobs. If any of the submissions are a failure, raise exception in order + to fail all submitted jobs, rather than allowing the submissions to continue + """ job_ids = [] condor_job_ids = [] for job_submit_param in job_submission_params: @@ -337,10 +352,10 @@ def _submit_multiple(self, job_submission_params): job_id=job_id, exception=RuntimeError(error_msg) ) raise RuntimeError(error_msg) - - self.update_job_to_queued(job_id=job_id, scheduler_id=condor_job_id) condor_job_ids.append(condor_job_id) + self._update_to_queued_multiple(job_ids=job_ids, scheduler_ids=condor_job_ids) + return job_ids def _run(self, params): @@ -370,14 +385,14 @@ def _run(self, params): return job_id - def _abort_child_jobs(self, child_job_ids): + def _abort_multiple_jobs(self, job_ids): """ Cancel a list of child jobs, and their child jobs """ - for child_job_id in child_job_ids: + for job_id in job_ids: try: self.sdkmr.cancel_job( - job_id=child_job_id, + job_id=job_id, terminated_code=TerminatedCode.terminated_by_batch_abort.value, ) except Exception as e: @@ -411,33 +426,16 @@ def _create_batch_job(self, wsid, meta): ) j = self.sdkmr.save_and_return_job(j) - # TODO Do we need a new kafka call? + # TODO Do we need a new kafka call for batch? self.sdkmr.get_kafka_client().send_kafka_message( message=KafkaCreateJob(job_id=str(j.id), user=j.user) ) return j def _run_batch(self, batch_job: Job, params): - child_jobs = [] - for job_param in params: job_param[_BATCH_ID] = str(batch_job.id) - - try: - child_jobs = self._run_multiple(params) - except Exception as e: - # See if we can abort any of these child jobs - raise e - - # try: - # child_jobs.append(str(self._run(params=job_param))) - # except Exception as e: - # self.logger.debug( - # msg=f"Failed to submit child job. Aborting entire batch job {e}" - # ) - # self._abort_child_jobs(child_jobs) - # raise e - + child_jobs = self._run_multiple(params) batch_job.child_jobs = child_jobs self.sdkmr.save_job(batch_job) diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 9d934e51b..781de0697 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -72,7 +72,6 @@ _CLUSTER_1 = "cluster1" _CLUSTER_2 = "cluster2" - _EMPTY_JOB_REQUIREMENTS = { "cpus": None, "memory_MB": None, @@ -529,7 +528,6 @@ def test_run_job_as_concierge_sched_reqs_empty_list_as_admin(): def _run_as_concierge_empty_as_admin(concierge_params, app): - # Set up data variables client_group = "concierge" # hardcoded default for run_as_concierge cpus = 1 @@ -839,7 +837,8 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ] ) - assert len(sdkmr.save_job.call_args_list) == 3 + assert len(sdkmr.save_job.call_args_list) == 1 + assert len(sdkmr.save_jobs.call_args_list) == 1 # initial child jobs data save expected_job_1 = _create_job( @@ -887,19 +886,16 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): [call(params=jsp_expected_1), call(params=jsp_expected_2)] ) - # updated job data save - mocks[MongoUtil].get_job.assert_has_calls([call(_JOB_ID_1), call(_JOB_ID_2)]) - # update to queued state - got_queued_job_1 = sdkmr.save_job.call_args_list[0][0][0] - got_queued_job_2 = sdkmr.save_job.call_args_list[1][0][0] - _check_queued_job_save(got_queued_job_1, _JOB_ID_1, _CLUSTER_1) - _check_queued_job_save(got_queued_job_2, _JOB_ID_2, _CLUSTER_2) + mocks[MongoUtil].update_jobs_to_queued.assert_has_calls( + [call(job_ids=[_JOB_ID_1, _JOB_ID_2], scheduler_ids=[_CLUSTER_1, _CLUSTER_2])] + ) mocks[KafkaClient].send_kafka_message.assert_has_calls( [ - # call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job - # call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), + call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job + call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), + call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), call( KafkaQueueChange( job_id=_JOB_ID_1, @@ -908,7 +904,6 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): scheduler_id=_CLUSTER_1, ) ), - # call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), call( KafkaQueueChange( job_id=_JOB_ID_2, @@ -932,7 +927,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): final_expected_parent_job.id = ObjectId(_JOB_ID) final_expected_parent_job.user = _USER final_expected_parent_job.child_jobs = [_JOB_ID_1, _JOB_ID_2] - final_got_parent_job = sdkmr.save_job.call_args_list[2][0][0] + final_got_parent_job = sdkmr.save_job.call_args_list[0][0][0] assert_jobs_equal(final_got_parent_job, final_expected_parent_job) From a8cc2ae625fe7eb868da98b8d6814d87cbe814e2 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 16 Jul 2021 16:08:35 -0500 Subject: [PATCH 03/31] Timings --- lib/execution_engine2/sdk/EE2Logs.py | 1 - lib/execution_engine2/sdk/EE2Runjob.py | 40 ++++++++++++++++++++++++-- 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/lib/execution_engine2/sdk/EE2Logs.py b/lib/execution_engine2/sdk/EE2Logs.py index e71b705ff..be04acd78 100644 --- a/lib/execution_engine2/sdk/EE2Logs.py +++ b/lib/execution_engine2/sdk/EE2Logs.py @@ -104,7 +104,6 @@ def add_job_logs(self, job_id, log_lines, as_admin=False) -> AddLogResult: self.sdkmr.get_job_with_permission( job_id, JobPermissions.WRITE, as_admin=as_admin ) - self.sdkmr.logger.debug(f"About to add logs for {job_id}") try: try: job_log = self.sdkmr.mongo_util.get_job_log_pymongo(job_id) diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index f1c0b1ffa..fad9ed48c 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -266,28 +266,45 @@ def _run_multiple(self, runjob_params): :return: """ # Save records to db + init_job_rec = time.time() job_records = [] for runjob_param in runjob_params: job_records.append( self._init_job_rec(self.sdkmr.get_user_id(), runjob_param, save=False) ) + print("init_job_rec = ", time.time() - init_job_rec) + + save_jobs = time.time() job_ids = self.sdkmr.save_jobs(job_records) + print("save_jobs = ", time.time() - save_jobs) + print("init and save save_jobs = ", time.time() - init_job_rec) + # Generate job submission params + gen_sub_time = time.time() job_submission_params = [] for i, job_id in enumerate(job_ids): job_submission_params.append( self._generate_job_submission_params(job_id, runjob_params[i]) ) assert job_id == job_submission_params[i].job_id + print("gen_sub_time = ", time.time() - gen_sub_time) + + kafku = time.time() + for job_id in job_ids: self.sdkmr.get_kafka_client().send_kafka_message( message=KafkaCreateJob( job_id=str(job_id), user=self.sdkmr.get_user_id() ) ) + print("kafku = ", time.time() - kafku) + # Submit to Condor + condor_time = time.time() try: - return self._submit_multiple(job_submission_params) + submission_ids = self._submit_multiple(job_submission_params) + print("condor_time = ", time.time() - condor_time) + return submission_ids except Exception as e: self._abort_multiple_jobs(job_ids) raise e @@ -433,11 +450,17 @@ def _create_batch_job(self, wsid, meta): return j def _run_batch(self, batch_job: Job, params): + for job_param in params: job_param[_BATCH_ID] = str(batch_job.id) + run_multiple = time.time() child_jobs = self._run_multiple(params) + print("Run multiple=", time.time() - run_multiple) + + batch_save = time.time() batch_job.child_jobs = child_jobs self.sdkmr.save_job(batch_job) + print("batch_save batch_save=", time.time() - batch_save) return child_jobs @@ -461,17 +484,30 @@ def run_batch( wsid = batch_params.get(_WORKSPACE_ID) meta = batch_params.get(_META) + preflight_begin = time.time() self._preflight( runjob_params=params, batch_params=batch_params, new_batch_job=True, as_admin=as_admin, ) - + print("Preflight took", time.time() - preflight_begin) + ajr = time.time() self._add_job_requirements(params, bool(as_admin)) # as_admin checked above + print("ajr", time.time() - ajr) + + cja = time.time() self._check_job_arguments(params, batch_job=True) + print("cja", time.time() - cja) + cbj = time.time() batch_job = self._create_batch_job(wsid=wsid, meta=meta) + print("cbj", time.time() - cbj) + + print( + "Total time for presubmit (-preflight_begin) ", + time.time() - preflight_begin, + ) children_jobs = self._run_batch(batch_job=batch_job, params=params) return {_BATCH_ID: str(batch_job.id), "child_job_ids": children_jobs} From 606d29249abdb437dcb0b5f7348fba09d619d833 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 16 Jul 2021 16:22:51 -0500 Subject: [PATCH 04/31] Timings --- lib/execution_engine2/sdk/EE2Runjob.py | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index fad9ed48c..5da4045a2 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -272,13 +272,13 @@ def _run_multiple(self, runjob_params): job_records.append( self._init_job_rec(self.sdkmr.get_user_id(), runjob_param, save=False) ) - print("init_job_rec = ", time.time() - init_job_rec) + self.logger.debug(f"init_job_rec = {time.time() - init_job_rec}") save_jobs = time.time() job_ids = self.sdkmr.save_jobs(job_records) - print("save_jobs = ", time.time() - save_jobs) - print("init and save save_jobs = ", time.time() - init_job_rec) + self.logger.debug(f"save_jobs = {time.time() - save_jobs}") + self.logger.debug(f"init and save save_jobs = {time.time() - init_job_rec}") # Generate job submission params gen_sub_time = time.time() @@ -288,7 +288,10 @@ def _run_multiple(self, runjob_params): self._generate_job_submission_params(job_id, runjob_params[i]) ) assert job_id == job_submission_params[i].job_id - print("gen_sub_time = ", time.time() - gen_sub_time) + + self.logger.debug( + f"init and gen_sub_time save_jobs = {time.time() - gen_sub_time}" + ) kafku = time.time() for job_id in job_ids: @@ -297,13 +300,14 @@ def _run_multiple(self, runjob_params): job_id=str(job_id), user=self.sdkmr.get_user_id() ) ) - print("kafku = ", time.time() - kafku) + + self.logger.debug(f"kafka submit = {time.time() - kafku}") # Submit to Condor condor_time = time.time() try: submission_ids = self._submit_multiple(job_submission_params) - print("condor_time = ", time.time() - condor_time) + self.logger.debug(f"condor_time submit = {time.time() - condor_time}") return submission_ids except Exception as e: self._abort_multiple_jobs(job_ids) @@ -455,12 +459,12 @@ def _run_batch(self, batch_job: Job, params): job_param[_BATCH_ID] = str(batch_job.id) run_multiple = time.time() child_jobs = self._run_multiple(params) - print("Run multiple=", time.time() - run_multiple) + self.logger.debug(f"run_multiple = {time.time() - run_multiple}") batch_save = time.time() batch_job.child_jobs = child_jobs self.sdkmr.save_job(batch_job) - print("batch_save batch_save=", time.time() - batch_save) + self.logger.debug(f"batch_save = {time.time() - batch_save}") return child_jobs From 18b0109b4cefa5542f64f9633765b010694f80a0 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 16 Jul 2021 16:38:28 -0500 Subject: [PATCH 05/31] Timings --- lib/execution_engine2/sdk/EE2Runjob.py | 9 +++++++-- test/utils_shared/producer.py | 5 +---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index 5da4045a2..ee4c50b69 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -293,6 +293,7 @@ def _run_multiple(self, runjob_params): f"init and gen_sub_time save_jobs = {time.time() - gen_sub_time}" ) + # Takes 2.5200018882751465 for 100, can shave off 2.5 secs by making this async kafku = time.time() for job_id in job_ids: self.sdkmr.get_kafka_client().send_kafka_message( @@ -344,6 +345,7 @@ def _submit_multiple(self, job_submission_params): Submit multiple jobs. If any of the submissions are a failure, raise exception in order to fail all submitted jobs, rather than allowing the submissions to continue """ + begin = time.time() job_ids = [] condor_job_ids = [] for job_submit_param in job_submission_params: @@ -354,7 +356,7 @@ def _submit_multiple(self, job_submission_params): params=job_submit_param ) condor_job_id = submission_info.clusterid - self.logger.debug(f"Submitted job id and got '{condor_job_id}'") + except Exception as e: self.logger.error(e) self._finish_created_job(job_id=job_id, exception=e) @@ -375,7 +377,11 @@ def _submit_multiple(self, job_submission_params): raise RuntimeError(error_msg) condor_job_ids.append(condor_job_id) + self.logger.error(f"It took {time.time() - begin} to submit jobs to condor") + + update_time = time.time() self._update_to_queued_multiple(job_ids=job_ids, scheduler_ids=condor_job_ids) + self.logger.error(f"It took {time.time() - update_time} to update jobs ") return job_ids @@ -386,7 +392,6 @@ def _run(self, params): try: submission_info = self.sdkmr.get_condor().run_job(params=job_params) condor_job_id = submission_info.clusterid - self.logger.debug(f"Submitted job id and got '{condor_job_id}'") except Exception as e: self.logger.error(e) self._finish_created_job(job_id=job_id, exception=e) diff --git a/test/utils_shared/producer.py b/test/utils_shared/producer.py index b90926da4..0ff2f5ace 100644 --- a/test/utils_shared/producer.py +++ b/test/utils_shared/producer.py @@ -22,11 +22,8 @@ def send_kafka_message(self, message, topic=DEFAULT_TOPIC): producer = Producer({"bootstrap.servers": self.server_address}) producer.produce(topic, str(message), callback=_delivery_report) producer.poll(2) - logging.info( - f"Successfully sent message to kafka at topic={topic} message={json.dumps(message)} server_address={self.server_address}" - ) except Exception as e: - logging.info( + logging.error( f"Failed to send message to kafka at topic={topic} message={json.dumps(message)} server_address={self.server_address}" ) raise Exception(e) From 9dc9fdf10423370fa4a2b93327adf3f37e2bae30 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 16 Jul 2021 17:17:01 -0500 Subject: [PATCH 06/31] Timings --- lib/execution_engine2/db/MongoUtil.py | 10 +--------- lib/execution_engine2/sdk/EE2Runjob.py | 2 ++ lib/execution_engine2/utils/KafkaUtils.py | 5 +---- 3 files changed, 4 insertions(+), 13 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 04745d093..9fe242f24 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -455,17 +455,9 @@ def insert_jobs(self, jobs_to_insert: List[Job]) -> List[ObjectId]: """ # TODO Look at pymongo write_concerns that may be useful # TODO see if pymongo is faster - + # TODO: Think about error handling inserted = Job.objects.insert(doc_or_docs=jobs_to_insert, load_bulk=False) - return inserted - # TODO: Send Kafka messages here or elsewhere - # job_id = self.sdkmr.save_job(job) - # self.sdkmr.get_kafka_client().send_kafka_message( - # message=KafkaCreateJob(job_id=job_id, user=user_id) - # ) - - # TODO: Think about error handling def insert_one(self, doc): """ diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index ee4c50b69..e2a675304 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -378,9 +378,11 @@ def _submit_multiple(self, job_submission_params): condor_job_ids.append(condor_job_id) self.logger.error(f"It took {time.time() - begin} to submit jobs to condor") + # It took 4.836009502410889 to submit jobs to condor update_time = time.time() self._update_to_queued_multiple(job_ids=job_ids, scheduler_ids=condor_job_ids) + # It took 1.9239885807037354 to update jobs self.logger.error(f"It took {time.time() - update_time} to update jobs ") return job_ids diff --git a/lib/execution_engine2/utils/KafkaUtils.py b/lib/execution_engine2/utils/KafkaUtils.py index ec2b07f7c..afb1a9473 100644 --- a/lib/execution_engine2/utils/KafkaUtils.py +++ b/lib/execution_engine2/utils/KafkaUtils.py @@ -212,11 +212,8 @@ def send_kafka_message(self, message, topic: str = DEFAULT_TOPIC): ) # TODO Remove POLL? producer.poll(2) - logger.debug( - f"Successfully sent message to kafka at topic={topic} message={json.dumps(message.__dict__)} server_address={self.server_address}" - ) except Exception as e: - logger.debug( + logger.error( f"Failed to send message to kafka at topic={topic} message={json.dumps(message.__dict__)} server_address={self.server_address}" ) raise Exception(e) From 6404b869f7d312101b32e1a4c9006f37ce579d43 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 19 Jul 2021 22:13:05 -0500 Subject: [PATCH 07/31] PR review --- lib/execution_engine2/db/MongoUtil.py | 19 ++++++++++------ lib/execution_engine2/sdk/EE2Runjob.py | 30 +++++++++++++++----------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 9fe242f24..d16c1a343 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -15,6 +15,7 @@ RecordNotFoundException, InvalidStatusTransitionException, ) +from execution_engine2.sdk.EE2Runjob import JobIdPair class MongoUtil: @@ -263,19 +264,23 @@ def check_if_already_finished(job_status): return True return False - def update_jobs_to_queued(self, job_ids, scheduler_ids): + def update_jobs_to_queued( + self, job_id_pairs: List[JobIdPair], scheduler_type: str = "condor" + ): bulk_operations = [] now = time.time() - for i, job_id in enumerate(job_ids): + for job_id_pair in job_id_pairs: + if None in job_id_pairs: + raise InvalidStatusTransitionException bulk_operations.append( UpdateOne( - {"_id": ObjectId(job_id)}, + {"_id": ObjectId(job_id_pair.job_id)}, { "$set": { "status": Status.queued.value, "queued": now, - "scheduler_id": scheduler_ids[i], - "scheduler_type": "condor", + "scheduler_id": job_id_pair.scheduler_id, + "scheduler_type": scheduler_type, } }, ) @@ -286,9 +291,9 @@ def update_jobs_to_queued(self, job_ids, scheduler_ids): bwr = pymongo_client[self.mongo_database][mongo_collection].bulk_write( bulk_operations, ordered=False ) - assert bwr.modified_count == len(job_ids) + assert bwr.modified_count == len(job_id_pairs) - # TODO error handling for bulk write result + # TODO error handling for bulk write result, otherwise pymongo error will bubble up def cancel_job(self, job_id=None, terminated_code=None): """ diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index e2a675304..dd21cf637 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -74,6 +74,11 @@ class PreparedJobParams(NamedTuple): job_id: str +class JobIdPair(NamedTuple): + job_id: str + scheduler_id: str + + from typing import TYPE_CHECKING if TYPE_CHECKING: @@ -315,22 +320,19 @@ def _run_multiple(self, runjob_params): raise e def _update_to_queued_multiple(self, job_ids, scheduler_ids): - # TODO Unused if len(job_ids) != len(scheduler_ids): raise Exception( "Need to provide the same amount of job ids and scheduler_ids" ) + jobs_to_update = map(JobIdPair, job_ids, scheduler_ids) + # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL - # TODO PASS QUEUE TIME IN FROM SCHEDULER ITSELF? - # TODO PASS IN SCHEDULER TYPE? - self.sdkmr.get_mongo_util().update_jobs_to_queued( - job_ids=job_ids, scheduler_ids=scheduler_ids - ) + + self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) # TODO figure out kafka message for i, job_id in enumerate(job_ids): - self.sdkmr.get_kafka_client().send_kafka_message( message=KafkaQueueChange( job_id=job_id, @@ -461,17 +463,19 @@ def _create_batch_job(self, wsid, meta): return j def _run_batch(self, batch_job: Job, params): + """Add the batch id, save the jobs to the db, run the jobs""" for job_param in params: job_param[_BATCH_ID] = str(batch_job.id) - run_multiple = time.time() + child_jobs = self._run_multiple(params) - self.logger.debug(f"run_multiple = {time.time() - run_multiple}") - batch_save = time.time() - batch_job.child_jobs = child_jobs - self.sdkmr.save_job(batch_job) - self.logger.debug(f"batch_save = {time.time() - batch_save}") + # Cancel child jobs if we can't notify the batch job of the child jobs + try: + batch_job.modify(add_to_set__child_jobs=child_jobs) + except Exception as e: + self._abort_multiple_jobs(child_jobs) + raise e return child_jobs From 0b65043a8bb66ba79d543fc2f9854e364cf0fff5 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 19 Jul 2021 23:06:59 -0500 Subject: [PATCH 08/31] Try to prevent bad updates for cancelled jobs --- lib/execution_engine2/db/MongoUtil.py | 14 +++++++++++--- lib/execution_engine2/sdk/EE2Runjob.py | 2 +- test/tests_for_sdkmr/EE2Runjob_test.py | 2 +- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index d16c1a343..4fe3aec96 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -264,9 +264,14 @@ def check_if_already_finished(job_status): return True return False - def update_jobs_to_queued( + def update_created_jobs_to_queued( self, job_id_pairs: List[JobIdPair], scheduler_type: str = "condor" - ): + ) -> None: + """ + Updates a list of created jobs to queued. Does not work on jobs with status of "Estimating" + :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids + :param scheduler_type: The scheduler this job was queued in, default condor + """ bulk_operations = [] now = time.time() for job_id_pair in job_id_pairs: @@ -274,7 +279,10 @@ def update_jobs_to_queued( raise InvalidStatusTransitionException bulk_operations.append( UpdateOne( - {"_id": ObjectId(job_id_pair.job_id)}, + { + "_id": ObjectId(job_id_pair.job_id), + "status": Status.created.value, + }, { "$set": { "status": Status.queued.value, diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index dd21cf637..395b879b3 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -329,7 +329,7 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL - self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) + self.sdkmr.get_mongo_util().update_created_jobs_to_queued(jobs_to_update) # TODO figure out kafka message for i, job_id in enumerate(job_ids): diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 781de0697..13eb38fda 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -887,7 +887,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ) # update to queued state - mocks[MongoUtil].update_jobs_to_queued.assert_has_calls( + mocks[MongoUtil].update_created_jobs_to_queued.assert_has_calls( [call(job_ids=[_JOB_ID_1, _JOB_ID_2], scheduler_ids=[_CLUSTER_1, _CLUSTER_2])] ) From 400c1ab8080a0b6a3f98d96c9bd3f3968a72569d Mon Sep 17 00:00:00 2001 From: bio-boris Date: Tue, 20 Jul 2021 13:42:24 -0500 Subject: [PATCH 09/31] Try to prevent bad updates for cancelled jobs --- lib/execution_engine2/db/MongoUtil.py | 8 +++++--- lib/execution_engine2/sdk/EE2Runjob.py | 4 ++-- test/tests_for_sdkmr/EE2Runjob_test.py | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 4fe3aec96..969f9c196 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -264,11 +264,11 @@ def check_if_already_finished(job_status): return True return False - def update_created_jobs_to_queued( + def update_jobs_to_queued( self, job_id_pairs: List[JobIdPair], scheduler_type: str = "condor" ) -> None: """ - Updates a list of created jobs to queued. Does not work on jobs with status of "Estimating" + Updates a list of created jobs to queued. Does not work on jobs that already have gone through a status transition :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids :param scheduler_type: The scheduler this job was queued in, default condor """ @@ -281,7 +281,9 @@ def update_created_jobs_to_queued( UpdateOne( { "_id": ObjectId(job_id_pair.job_id), - "status": Status.created.value, + "status": { + "$in": [Status.created.value, Status.estimating.value] + }, }, { "$set": { diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index 395b879b3..0d5bacfd9 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -325,11 +325,11 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): "Need to provide the same amount of job ids and scheduler_ids" ) - jobs_to_update = map(JobIdPair, job_ids, scheduler_ids) + jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL - self.sdkmr.get_mongo_util().update_created_jobs_to_queued(jobs_to_update) + self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) # TODO figure out kafka message for i, job_id in enumerate(job_ids): diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 13eb38fda..781de0697 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -887,7 +887,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ) # update to queued state - mocks[MongoUtil].update_created_jobs_to_queued.assert_has_calls( + mocks[MongoUtil].update_jobs_to_queued.assert_has_calls( [call(job_ids=[_JOB_ID_1, _JOB_ID_2], scheduler_ids=[_CLUSTER_1, _CLUSTER_2])] ) From f16943a1c9f1a91b65ff325c17d9c1f31200ce28 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Tue, 20 Jul 2021 14:21:09 -0500 Subject: [PATCH 10/31] Not so trivial after all --- lib/execution_engine2/sdk/EE2Runjob.py | 17 +---------------- lib/execution_engine2/sdk/SDKMethodRunner.py | 8 ++++++++ test/tests_for_sdkmr/EE2Runjob_test.py | 16 ++++++++++------ 3 files changed, 19 insertions(+), 22 deletions(-) diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index 0d5bacfd9..e883b8fed 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -472,7 +472,7 @@ def _run_batch(self, batch_job: Job, params): # Cancel child jobs if we can't notify the batch job of the child jobs try: - batch_job.modify(add_to_set__child_jobs=child_jobs) + self.sdkmr.add_child_jobs(batch_job, child_jobs) except Exception as e: self._abort_multiple_jobs(child_jobs) raise e @@ -499,30 +499,15 @@ def run_batch( wsid = batch_params.get(_WORKSPACE_ID) meta = batch_params.get(_META) - preflight_begin = time.time() self._preflight( runjob_params=params, batch_params=batch_params, new_batch_job=True, as_admin=as_admin, ) - print("Preflight took", time.time() - preflight_begin) - ajr = time.time() self._add_job_requirements(params, bool(as_admin)) # as_admin checked above - print("ajr", time.time() - ajr) - - cja = time.time() self._check_job_arguments(params, batch_job=True) - print("cja", time.time() - cja) - - cbj = time.time() batch_job = self._create_batch_job(wsid=wsid, meta=meta) - print("cbj", time.time() - cbj) - - print( - "Total time for presubmit (-preflight_begin) ", - time.time() - preflight_begin, - ) children_jobs = self._run_batch(batch_job=batch_job, params=params) return {_BATCH_ID: str(batch_job.id), "child_job_ids": children_jobs} diff --git a/lib/execution_engine2/sdk/SDKMethodRunner.py b/lib/execution_engine2/sdk/SDKMethodRunner.py index c3d85fb66..350599960 100644 --- a/lib/execution_engine2/sdk/SDKMethodRunner.py +++ b/lib/execution_engine2/sdk/SDKMethodRunner.py @@ -266,6 +266,14 @@ def save_job(self, job: Job) -> str: job.save() return str(job.id) + def add_child_jobs(self, batch_job: Job, child_jobs: List[str]): + """ + Add child jobs to a batch job record in the Mongo Database and return the updated job. + :return: + """ + batch_job.modify(add_to_set__child_jobs=child_jobs) + return batch_job + def save_and_return_job(self, job: Job) -> Job: """ Save a job record to the Mongo database and return the updated job. diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 781de0697..c550c2e96 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -20,7 +20,7 @@ AuthError, InvalidParameterForBatch, ) -from execution_engine2.sdk.EE2Runjob import EE2RunJob, JobPermissions +from execution_engine2.sdk.EE2Runjob import EE2RunJob, JobPermissions, JobIdPair from execution_engine2.sdk.SDKMethodRunner import SDKMethodRunner from execution_engine2.sdk.job_submission_parameters import ( JobSubmissionParameters, @@ -183,7 +183,9 @@ def _set_up_common_return_values(mocks): mocks[MongoUtil].get_job.return_value = retjob -def _check_common_mock_calls(mocks, reqs, wsid, app=_APP, parent_job_id=None): +def _check_common_mock_calls( + mocks, reqs, wsid, app=_APP, parent_job_id=None, batch=False +): """ Check that mocks are called as expected when those calls are similar or the same for several tests. @@ -775,8 +777,10 @@ def _set_up_common_return_values_batch(mocks): _GIT_COMMIT_1, _GIT_COMMIT_2, ] + returned_parent_job.modify = None mocks[SDKMethodRunner].save_and_return_job.return_value = returned_parent_job + mocks[SDKMethodRunner].add_child_jobs.return_value = returned_parent_job # create job1, update job1, create job2, update job2, update parent job mocks[SDKMethodRunner].save_job.side_effect = [ @@ -837,7 +841,6 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ] ) - assert len(sdkmr.save_job.call_args_list) == 1 assert len(sdkmr.save_jobs.call_args_list) == 1 # initial child jobs data save @@ -888,7 +891,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): # update to queued state mocks[MongoUtil].update_jobs_to_queued.assert_has_calls( - [call(job_ids=[_JOB_ID_1, _JOB_ID_2], scheduler_ids=[_CLUSTER_1, _CLUSTER_2])] + [call([JobIdPair(_JOB_ID_1, _CLUSTER_1), JobIdPair(_JOB_ID_2, _CLUSTER_2)])] ) mocks[KafkaClient].send_kafka_message.assert_has_calls( @@ -927,8 +930,9 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): final_expected_parent_job.id = ObjectId(_JOB_ID) final_expected_parent_job.user = _USER final_expected_parent_job.child_jobs = [_JOB_ID_1, _JOB_ID_2] - final_got_parent_job = sdkmr.save_job.call_args_list[0][0][0] - assert_jobs_equal(final_got_parent_job, final_expected_parent_job) + # final_got_parent_job = sdkmr.add_child_jobs.call_args_list[0][0][0] + # Not sure how to get the final parent job anymore to do this test HALP + # assert_jobs_equal(final_got_parent_job, final_expected_parent_job) def test_run_job_batch_with_parent_job_wsid(): From ad74923f404cc8e214c70b6a9e1408d688fbfda3 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Wed, 21 Jul 2021 22:01:34 -0500 Subject: [PATCH 11/31] Pr feedback --- lib/execution_engine2/db/MongoUtil.py | 12 ++++---- lib/execution_engine2/sdk/EE2Runjob.py | 6 +--- lib/execution_engine2/sdk/EE2Status.py | 1 + test/tests_for_db/ee2_MongoUtil_test.py | 38 +++++++++++++++++++++++-- test/tests_for_sdkmr/EE2Runjob_test.py | 14 ++++++--- 5 files changed, 55 insertions(+), 16 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 969f9c196..ab550bc00 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -268,7 +268,8 @@ def update_jobs_to_queued( self, job_id_pairs: List[JobIdPair], scheduler_type: str = "condor" ) -> None: """ - Updates a list of created jobs to queued. Does not work on jobs that already have gone through a status transition + Updates a list of created jobs to queued. Does not work on jobs that already have gone through a status transition. + If the record is not in the CREATED status, an InvalidStatusTransitionException will be raised. :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids :param scheduler_type: The scheduler this job was queued in, default condor """ @@ -281,9 +282,7 @@ def update_jobs_to_queued( UpdateOne( { "_id": ObjectId(job_id_pair.job_id), - "status": { - "$in": [Status.created.value, Status.estimating.value] - }, + "status": Status.created.value, }, { "$set": { @@ -301,7 +300,10 @@ def update_jobs_to_queued( bwr = pymongo_client[self.mongo_database][mongo_collection].bulk_write( bulk_operations, ordered=False ) - assert bwr.modified_count == len(job_id_pairs) + if bwr.modified_count != len(job_id_pairs): + raise InvalidStatusTransitionException( + "Wasn't able to update all jobs to created " + ) # TODO error handling for bulk write result, otherwise pymongo error will bubble up diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index e883b8fed..fae5b89b2 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -324,13 +324,8 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): raise Exception( "Need to provide the same amount of job ids and scheduler_ids" ) - jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - - # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL - self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) - # TODO figure out kafka message for i, job_id in enumerate(job_ids): self.sdkmr.get_kafka_client().send_kafka_message( @@ -765,6 +760,7 @@ def retry_multiple( """ #TODO Add new job requirements/cgroups as an optional param #TODO Notify the parent container that it has multiple new children, instead of multiple transactions? + #TODO Prevent retry when multiple batch job containers? :param job_ids: The list of jobs to retry :param as_admin: Run with admin permission diff --git a/lib/execution_engine2/sdk/EE2Status.py b/lib/execution_engine2/sdk/EE2Status.py index f26f9ce56..5c2859f23 100644 --- a/lib/execution_engine2/sdk/EE2Status.py +++ b/lib/execution_engine2/sdk/EE2Status.py @@ -123,6 +123,7 @@ def cancel_job(self, job_id, terminated_code=None, as_admin=False): ) # TODO Issue #190 IF success['TotalSuccess = 0'] == FALSE, don't send a kafka message? + self.sdkmr.get_condor().cancel_job(job_id=f"{job.scheduler_id}.0") self.sdkmr.kafka_client.send_kafka_message( message=KafkaCancelJob( diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index fc0e276bf..d200ac3e7 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -2,12 +2,13 @@ import logging import os import unittest -from configparser import ConfigParser from bson.objectid import ObjectId from execution_engine2.db.MongoUtil import MongoUtil -from execution_engine2.db.models.models import Job, JobLog +from execution_engine2.db.models.models import Job, JobLog, Status +from execution_engine2.sdk.EE2Runjob import JobIdPair +from execution_engine2.exceptions import InvalidStatusTransitionException from test.utils_shared.test_utils import ( bootstrap, get_example_job, @@ -57,6 +58,39 @@ def test_init_ok(self): mongo_util = self.getMongoUtil() self.assertTrue(set(class_attri) <= set(mongo_util.__dict__.keys())) + def test_update_jobs_success(self): + job = get_example_job(status=Status.created.value) + job2 = get_example_job(status=Status.created.value) + job_id1 = job.save().id + job_id2 = job2.save().id + job_ids = [job_id1, job_id2] + scheduler_ids = [job.scheduler_id, job2.scheduler_id] + jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) + print(jobs_to_update) + assert Status.created.value in [job.status, job2.status] + + # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL + + def test_update_jobs_failure(self): + job = get_example_job(status=Status.created.value) + job2 = get_example_job(status=Status.error.value) + job3 = get_example_job(status=Status.terminated.value) + + job_id1 = job.save().id + job_id2 = job2.save().id + job_id3 = job3.save().id + job_ids = [job_id1, job_id2, job_id3] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] + jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) + print(jobs_to_update) + assert Status.created.value in [job.status, job2.status, job3.status] + + with self.assertRaisesRegex( + InvalidStatusTransitionException, + "Wasn't able to update all jobs to created ", + ): + self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + def test_get_by_cluster(self): """Get a job by its condor scheduler_id""" mongo_util = self.getMongoUtil() diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index c550c2e96..a2454df36 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -890,9 +890,11 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ) # update to queued state - mocks[MongoUtil].update_jobs_to_queued.assert_has_calls( - [call([JobIdPair(_JOB_ID_1, _CLUSTER_1), JobIdPair(_JOB_ID_2, _CLUSTER_2)])] - ) + child_job_pairs = [ + JobIdPair(_JOB_ID_1, _CLUSTER_1), + JobIdPair(_JOB_ID_2, _CLUSTER_2), + ] + mocks[MongoUtil].update_jobs_to_queued.assert_has_calls([call(child_job_pairs)]) mocks[KafkaClient].send_kafka_message.assert_has_calls( [ @@ -930,10 +932,14 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): final_expected_parent_job.id = ObjectId(_JOB_ID) final_expected_parent_job.user = _USER final_expected_parent_job.child_jobs = [_JOB_ID_1, _JOB_ID_2] - # final_got_parent_job = sdkmr.add_child_jobs.call_args_list[0][0][0] + final_got_parent_job = sdkmr.add_child_jobs.call_args_list[0][0][0] # Not sure how to get the final parent job anymore to do this test HALP # assert_jobs_equal(final_got_parent_job, final_expected_parent_job) + sdkmr.add_child_jobs.assert_called_once_with( + batch_job=final_expected_parent_job, child_jobs=child_job_pairs + ) + def test_run_job_batch_with_parent_job_wsid(): """ From dd8cfcf4345b0d8140fa14017c8703d9d5057fb9 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Thu, 22 Jul 2021 14:40:13 -0500 Subject: [PATCH 12/31] Fix up tests --- lib/execution_engine2/sdk/EE2Runjob.py | 2 +- test/tests_for_sdkmr/EE2Runjob_test.py | 21 +++++++++------------ 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index fae5b89b2..3f44a00a4 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -467,7 +467,7 @@ def _run_batch(self, batch_job: Job, params): # Cancel child jobs if we can't notify the batch job of the child jobs try: - self.sdkmr.add_child_jobs(batch_job, child_jobs) + self.sdkmr.add_child_jobs(batch_job=batch_job, child_jobs=child_jobs) except Exception as e: self._abort_multiple_jobs(child_jobs) raise e diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index a2454df36..0f800278f 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -773,14 +773,12 @@ def _set_up_common_return_values_batch(mocks): returned_parent_job = Job() returned_parent_job.id = ObjectId(_JOB_ID) returned_parent_job.user = _USER + + mocks[SDKMethodRunner].save_and_return_job.return_value = returned_parent_job mocks[CatalogCache].lookup_git_commit_version.side_effect = [ _GIT_COMMIT_1, _GIT_COMMIT_2, ] - returned_parent_job.modify = None - - mocks[SDKMethodRunner].save_and_return_job.return_value = returned_parent_job - mocks[SDKMethodRunner].add_child_jobs.return_value = returned_parent_job # create job1, update job1, create job2, update job2, update parent job mocks[SDKMethodRunner].save_job.side_effect = [ @@ -928,17 +926,16 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): # ] # ) - final_expected_parent_job = Job() - final_expected_parent_job.id = ObjectId(_JOB_ID) - final_expected_parent_job.user = _USER - final_expected_parent_job.child_jobs = [_JOB_ID_1, _JOB_ID_2] - final_got_parent_job = sdkmr.add_child_jobs.call_args_list[0][0][0] - # Not sure how to get the final parent job anymore to do this test HALP - # assert_jobs_equal(final_got_parent_job, final_expected_parent_job) + # Test to see if add_child jobs is called with correct batch_container and children + expected_batch_container = Job() + expected_batch_container.id = ObjectId(_JOB_ID) + expected_batch_container.user = _USER + batch_job = sdkmr.add_child_jobs.call_args_list[0][1]["batch_job"] sdkmr.add_child_jobs.assert_called_once_with( - batch_job=final_expected_parent_job, child_jobs=child_job_pairs + batch_job=expected_batch_container, child_jobs=[_JOB_ID_1, _JOB_ID_2] ) + assert_jobs_equal(batch_job, expected_batch_container) def test_run_job_batch_with_parent_job_wsid(): From 1d977a6e00ebf76d0c14ac245e3d8536a6bc6cf1 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Thu, 22 Jul 2021 17:21:43 -0500 Subject: [PATCH 13/31] Fix up assert --- lib/execution_engine2/db/MongoUtil.py | 2 +- test/tests_for_db/ee2_MongoUtil_test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index ab550bc00..991812562 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -302,7 +302,7 @@ def update_jobs_to_queued( ) if bwr.modified_count != len(job_id_pairs): raise InvalidStatusTransitionException( - "Wasn't able to update all jobs to created " + f"Wasn't able to update all jobs from {Status.created.value} to {Status.queued.value}" ) # TODO error handling for bulk write result, otherwise pymongo error will bubble up diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index d200ac3e7..01703aa3e 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -87,7 +87,7 @@ def test_update_jobs_failure(self): with self.assertRaisesRegex( InvalidStatusTransitionException, - "Wasn't able to update all jobs to created ", + f"Wasn't able to update all jobs from {Status.created.value} to {Status.queued.value}", ): self.getMongoUtil().update_jobs_to_queued(jobs_to_update) From cac5241c807b5ef7750d4ed2e58846cb96e43793 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Thu, 22 Jul 2021 18:16:53 -0500 Subject: [PATCH 14/31] Fix missing test --- test/tests_for_db/ee2_MongoUtil_test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 01703aa3e..4e66ca96d 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -66,10 +66,10 @@ def test_update_jobs_success(self): job_ids = [job_id1, job_id2] scheduler_ids = [job.scheduler_id, job2.scheduler_id] jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - print(jobs_to_update) - assert Status.created.value in [job.status, job2.status] - - # TODO RETRY FOR RACE CONDITION OF RUN/CANCEL + self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + job.reload() + job2.reload() + assert Status.queued.value in [job.status, job2.status] def test_update_jobs_failure(self): job = get_example_job(status=Status.created.value) From c3e08b3f6639c8a882a7eed0395f7120e0d10855 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Thu, 22 Jul 2021 18:39:01 -0500 Subject: [PATCH 15/31] FIx tests --- test/tests_for_db/ee2_MongoUtil_test.py | 6 +++--- test/tests_for_sdkmr/EE2Runjob_test.py | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 4e66ca96d..15909c004 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -69,7 +69,7 @@ def test_update_jobs_success(self): self.getMongoUtil().update_jobs_to_queued(jobs_to_update) job.reload() job2.reload() - assert Status.queued.value in [job.status, job2.status] + assert all(j.status == Status.queued.value for j in [job, job2]) def test_update_jobs_failure(self): job = get_example_job(status=Status.created.value) @@ -82,8 +82,8 @@ def test_update_jobs_failure(self): job_ids = [job_id1, job_id2, job_id3] scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - print(jobs_to_update) - assert Status.created.value in [job.status, job2.status, job3.status] + + assert not all(j.status == Status.created.value for j in [job, job2, job3]) with self.assertRaisesRegex( InvalidStatusTransitionException, diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 0f800278f..d44f7b955 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -935,6 +935,12 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): sdkmr.add_child_jobs.assert_called_once_with( batch_job=expected_batch_container, child_jobs=[_JOB_ID_1, _JOB_ID_2] ) + """ + So this test doesn't actually check that the call is correct, but the assert_jobs_equal line below does + the assert below is necessary because of how equality works for Job objects + ( print(jobs_to_update) because they have the same object ID, which is what Job equality is based on. ) + and that the assert_called_once_with doesn't correctly check the job object + """ assert_jobs_equal(batch_job, expected_batch_container) From a7256a4691a2865b9aac5ff39e7bbca247df9ba2 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 23 Jul 2021 10:49:02 -0500 Subject: [PATCH 16/31] Fix comment --- test/tests_for_sdkmr/EE2Runjob_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index d44f7b955..31c77c493 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -938,7 +938,7 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): """ So this test doesn't actually check that the call is correct, but the assert_jobs_equal line below does the assert below is necessary because of how equality works for Job objects - ( print(jobs_to_update) because they have the same object ID, which is what Job equality is based on. ) + ( because they have the same object ID, which is what Job equality is based on. ) and that the assert_called_once_with doesn't correctly check the job object """ assert_jobs_equal(batch_job, expected_batch_container) From 27ef29a9e380fdc7500ba9e829c51f5505c7388a Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 23 Jul 2021 16:53:39 -0500 Subject: [PATCH 17/31] Fix tests --- test/tests_for_db/ee2_MongoUtil_test.py | 57 +++++++++++-------------- 1 file changed, 25 insertions(+), 32 deletions(-) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 15909c004..a07df4e3d 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -58,38 +58,31 @@ def test_init_ok(self): mongo_util = self.getMongoUtil() self.assertTrue(set(class_attri) <= set(mongo_util.__dict__.keys())) - def test_update_jobs_success(self): - job = get_example_job(status=Status.created.value) - job2 = get_example_job(status=Status.created.value) - job_id1 = job.save().id - job_id2 = job2.save().id - job_ids = [job_id1, job_id2] - scheduler_ids = [job.scheduler_id, job2.scheduler_id] - jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - self.getMongoUtil().update_jobs_to_queued(jobs_to_update) - job.reload() - job2.reload() - assert all(j.status == Status.queued.value for j in [job, job2]) - - def test_update_jobs_failure(self): - job = get_example_job(status=Status.created.value) - job2 = get_example_job(status=Status.error.value) - job3 = get_example_job(status=Status.terminated.value) - - job_id1 = job.save().id - job_id2 = job2.save().id - job_id3 = job3.save().id - job_ids = [job_id1, job_id2, job_id3] - scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] - jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - - assert not all(j.status == Status.created.value for j in [job, job2, job3]) - - with self.assertRaisesRegex( - InvalidStatusTransitionException, - f"Wasn't able to update all jobs from {Status.created.value} to {Status.queued.value}", - ): - self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + def test_update_jobs_enmasse(self): + + for state in Status: + job = get_example_job(status=Status.created.value).save() + job2 = get_example_job(status=state.value).save() + job3 = get_example_job(status=state.value).save() + + job_ids = [job.id, job2.id, job3.id] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] + jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) + + # All fail cases + if state.value != Status.created.value: + with self.assertRaisesRegex( + InvalidStatusTransitionException, + f"Wasn't able to update all jobs from {Status.created.value} to {Status.queued.value}", + ): + self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + # Success Case + else: + self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + job.reload() + job2.reload() + job3.reload() + assert all(j.status == Status.queued.value for j in [job, job2, job3]) def test_get_by_cluster(self): """Get a job by its condor scheduler_id""" From 2a0722f5211a06ece757d7b6adf761120160c4f9 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 26 Jul 2021 10:49:31 -0500 Subject: [PATCH 18/31] Pr feedback --- lib/execution_engine2/db/MongoUtil.py | 36 ++++++++++++++++--------- test/tests_for_db/ee2_MongoUtil_test.py | 25 ++++++++--------- 2 files changed, 36 insertions(+), 25 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 991812562..d1307bad5 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -273,20 +273,19 @@ def update_jobs_to_queued( :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids :param scheduler_type: The scheduler this job was queued in, default condor """ - bulk_operations = [] + bulk_update_scheduler_jobs = [] + bulk_update_created_to_queued = [] now = time.time() for job_id_pair in job_id_pairs: if None in job_id_pairs: raise InvalidStatusTransitionException - bulk_operations.append( + bulk_update_scheduler_jobs.append( UpdateOne( { "_id": ObjectId(job_id_pair.job_id), - "status": Status.created.value, }, { "$set": { - "status": Status.queued.value, "queued": now, "scheduler_id": job_id_pair.scheduler_id, "scheduler_type": scheduler_type, @@ -294,18 +293,29 @@ def update_jobs_to_queued( }, ) ) + bulk_update_created_to_queued.append( + UpdateOne( + { + "_id": ObjectId(job_id_pair.job_id), + "status": Status.created.value, + }, + { + "$set": { + "status": Status.queued.value, + } + }, + ) + ) + # Update provided jobs with scheduler id. Then only update non terminated jobs into updated status. mongo_collection = self.config["mongo-jobs-collection"] - if bulk_operations: + if bulk_update_scheduler_jobs: with self.pymongo_client(mongo_collection) as pymongo_client: - bwr = pymongo_client[self.mongo_database][mongo_collection].bulk_write( - bulk_operations, ordered=False + pymongo_client[self.mongo_database][mongo_collection].bulk_write( + bulk_update_scheduler_jobs, ordered=False + ) + pymongo_client[self.mongo_database][mongo_collection].bulk_write( + bulk_update_created_to_queued, ordered=False ) - if bwr.modified_count != len(job_id_pairs): - raise InvalidStatusTransitionException( - f"Wasn't able to update all jobs from {Status.created.value} to {Status.queued.value}" - ) - - # TODO error handling for bulk write result, otherwise pymongo error will bubble up def cancel_job(self, job_id=None, terminated_code=None): """ diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index a07df4e3d..6c282c03b 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -69,20 +69,21 @@ def test_update_jobs_enmasse(self): scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - # All fail cases - if state.value != Status.created.value: - with self.assertRaisesRegex( - InvalidStatusTransitionException, - f"Wasn't able to update all jobs from {Status.created.value} to {Status.queued.value}", - ): - self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + self.getMongoUtil().update_jobs_to_queued(jobs_to_update) + job.reload() + job2.reload() + job3.reload() + # Success Case - else: - self.getMongoUtil().update_jobs_to_queued(jobs_to_update) - job.reload() - job2.reload() - job3.reload() + if state.value == Status.created.value: assert all(j.status == Status.queued.value for j in [job, job2, job3]) + # Fail Case, + else: + # The created job should be queued + assert job.status == Status.queued.value + # Other statuses should remain unchanged + assert job2.status == state.value + assert job3.status == state.value def test_get_by_cluster(self): """Get a job by its condor scheduler_id""" From 9bbbd61887227256ab813f818d280996f9333493 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 26 Jul 2021 15:54:40 -0500 Subject: [PATCH 19/31] Fix updates --- lib/execution_engine2/db/MongoUtil.py | 10 ++++++--- lib/execution_engine2/sdk/EE2Runjob.py | 30 ++++++++++++++++---------- 2 files changed, 26 insertions(+), 14 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index d1307bad5..4724a6763 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -273,12 +273,14 @@ def update_jobs_to_queued( :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids :param scheduler_type: The scheduler this job was queued in, default condor """ + job_ids_to_lookup_if_terminated = [] bulk_update_scheduler_jobs = [] bulk_update_created_to_queued = [] now = time.time() for job_id_pair in job_id_pairs: if None in job_id_pairs: raise InvalidStatusTransitionException + job_ids_to_lookup_if_terminated.append(job_id_pair.job_id) bulk_update_scheduler_jobs.append( UpdateOne( { @@ -313,9 +315,11 @@ def update_jobs_to_queued( pymongo_client[self.mongo_database][mongo_collection].bulk_write( bulk_update_scheduler_jobs, ordered=False ) - pymongo_client[self.mongo_database][mongo_collection].bulk_write( - bulk_update_created_to_queued, ordered=False - ) + bulk_update_result = pymongo_client[self.mongo_database][ + mongo_collection + ].bulk_write(bulk_update_created_to_queued, ordered=False) + + # job_ids_to_lookup_if_terminated def cancel_job(self, job_id=None, terminated_code=None): """ diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index cac6ec717..f6df0273c 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -325,17 +325,25 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): "Need to provide the same amount of job ids and scheduler_ids" ) jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) - # TODO figure out kafka message - for i, job_id in enumerate(job_ids): - self.sdkmr.get_kafka_client().send_kafka_message( - message=KafkaQueueChange( - job_id=job_id, - new_status=Status.queued.value, - previous_status=Status.created.value, # TODO maybe change this to allow for estimating jobs - scheduler_id=scheduler_ids[i], - ) - ) + latest_job_states = self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) + + + for job in latest_job_states: + if job.status == Status.queued.value: + + # TODO figure out kafka message + for i, job_id in enumerate(job_ids): + self.sdkmr.get_kafka_client().send_kafka_message( + message=KafkaQueueChange( + job_id=job_id, + new_status=Status.queued.value, + previous_status=Status.created.value, # TODO maybe change this to allow for estimating jobs + scheduler_id=scheduler_ids[i], + ) + ) + else: + self._safe_cancel(job_id=j) + def _submit_multiple(self, job_submission_params): """ From d857a37614da97a4f163d91b8675e9566079b371 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 30 Jul 2021 15:08:33 -0500 Subject: [PATCH 20/31] add post cancel --- lib/execution_engine2/db/MongoUtil.py | 28 ++++++++------ lib/execution_engine2/sdk/EE2Runjob.py | 52 ++++++++++---------------- lib/execution_engine2/sdk/EE2Status.py | 1 - 3 files changed, 35 insertions(+), 46 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 4724a6763..e5d134825 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -3,7 +3,8 @@ import time import traceback from contextlib import contextmanager -from typing import Dict, List +from datetime import datetime +from typing import Dict, List, Tuple from bson.objectid import ObjectId from mongoengine import connect, connection @@ -217,7 +218,9 @@ def get_job(self, job_id=None, exclude_fields=None) -> Job: return job - def get_jobs(self, job_ids=None, exclude_fields=None, sort_id_ascending=None): + def get_jobs( + self, job_ids=None, exclude_fields=None, sort_id_ascending=None + ) -> List[Job]: if not (job_ids and isinstance(job_ids, list)): raise ValueError("Please provide a non empty list of job ids") @@ -273,14 +276,14 @@ def update_jobs_to_queued( :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids :param scheduler_type: The scheduler this job was queued in, default condor """ - job_ids_to_lookup_if_terminated = [] + bulk_update_scheduler_jobs = [] bulk_update_created_to_queued = [] - now = time.time() + queue_time_now = datetime.utcnow().timestamp() for job_id_pair in job_id_pairs: if None in job_id_pairs: raise InvalidStatusTransitionException - job_ids_to_lookup_if_terminated.append(job_id_pair.job_id) + bulk_update_scheduler_jobs.append( UpdateOne( { @@ -288,7 +291,7 @@ def update_jobs_to_queued( }, { "$set": { - "queued": now, + "queued": queue_time_now, "scheduler_id": job_id_pair.scheduler_id, "scheduler_type": scheduler_type, } @@ -310,16 +313,17 @@ def update_jobs_to_queued( ) # Update provided jobs with scheduler id. Then only update non terminated jobs into updated status. mongo_collection = self.config["mongo-jobs-collection"] + + from pymongo.collection import Collection + if bulk_update_scheduler_jobs: with self.pymongo_client(mongo_collection) as pymongo_client: - pymongo_client[self.mongo_database][mongo_collection].bulk_write( - bulk_update_scheduler_jobs, ordered=False - ) - bulk_update_result = pymongo_client[self.mongo_database][ + ee2_jobs_col = pymongo_client[self.mongo_database][ mongo_collection - ].bulk_write(bulk_update_created_to_queued, ordered=False) + ] # type: Collection - # job_ids_to_lookup_if_terminated + # Bulk Update to add scheduler ids + ee2_jobs_col.bulk_write(bulk_update_scheduler_jobs, ordered=False) def cancel_job(self, job_id=None, terminated_code=None): """ diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index f6df0273c..eb4962944 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -271,22 +271,14 @@ def _run_multiple(self, runjob_params): :return: """ # Save records to db - init_job_rec = time.time() job_records = [] for runjob_param in runjob_params: job_records.append( self._init_job_rec(self.sdkmr.get_user_id(), runjob_param, save=False) ) - self.logger.debug(f"init_job_rec = {time.time() - init_job_rec}") - - save_jobs = time.time() job_ids = self.sdkmr.save_jobs(job_records) - self.logger.debug(f"save_jobs = {time.time() - save_jobs}") - self.logger.debug(f"init and save save_jobs = {time.time() - init_job_rec}") - # Generate job submission params - gen_sub_time = time.time() job_submission_params = [] for i, job_id in enumerate(job_ids): job_submission_params.append( @@ -294,12 +286,7 @@ def _run_multiple(self, runjob_params): ) assert job_id == job_submission_params[i].job_id - self.logger.debug( - f"init and gen_sub_time save_jobs = {time.time() - gen_sub_time}" - ) - - # Takes 2.5200018882751465 for 100, can shave off 2.5 secs by making this async - kafku = time.time() + # Takes 2.5200018882751465 for 100 records, can shave off 2.5 secs by making this async for job_id in job_ids: self.sdkmr.get_kafka_client().send_kafka_message( message=KafkaCreateJob( @@ -307,8 +294,6 @@ def _run_multiple(self, runjob_params): ) ) - self.logger.debug(f"kafka submit = {time.time() - kafku}") - # Submit to Condor condor_time = time.time() try: @@ -320,30 +305,32 @@ def _run_multiple(self, runjob_params): raise e def _update_to_queued_multiple(self, job_ids, scheduler_ids): + """ + This is called during job submission. If a job is terminated during job submission, + we have the chance to re-issue a termination and remove the job from the Job Queue + """ if len(job_ids) != len(scheduler_ids): raise Exception( "Need to provide the same amount of job ids and scheduler_ids" ) jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) - latest_job_states = self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) - + self.sdkmr.get_mongo_util().update_jobs_to_queued(jobs_to_update) + jobs = self.sdkmr.get_mongo_util().get_jobs(job_ids) - for job in latest_job_states: + for job in jobs: + job_id = str(job.id) if job.status == Status.queued.value: - - # TODO figure out kafka message - for i, job_id in enumerate(job_ids): - self.sdkmr.get_kafka_client().send_kafka_message( - message=KafkaQueueChange( - job_id=job_id, - new_status=Status.queued.value, - previous_status=Status.created.value, # TODO maybe change this to allow for estimating jobs - scheduler_id=scheduler_ids[i], - ) + self.sdkmr.get_kafka_client().send_kafka_message( + message=KafkaQueueChange( + job_id=job_id, + new_status=Status.queued.value, + previous_status=Status.created.value, # TODO maybe change this to allow for estimating jobs + scheduler_id=job.scheduler_id, ) - else: - self._safe_cancel(job_id=j) - + ) + elif job.status == Status.terminated.value: + # Remove from the queue, now that the scheduler_id is available + self._safe_cancel(job_id, TerminatedCode.terminated_by_user.value) def _submit_multiple(self, job_submission_params): """ @@ -361,7 +348,6 @@ def _submit_multiple(self, job_submission_params): params=job_submit_param ) condor_job_id = submission_info.clusterid - except Exception as e: self.logger.error(e) self._finish_created_job(job_id=job_id, exception=e) diff --git a/lib/execution_engine2/sdk/EE2Status.py b/lib/execution_engine2/sdk/EE2Status.py index 5c2859f23..9b2436d83 100644 --- a/lib/execution_engine2/sdk/EE2Status.py +++ b/lib/execution_engine2/sdk/EE2Status.py @@ -94,7 +94,6 @@ def cancel_job(self, job_id, terminated_code=None, as_admin=False): :param as_admin: Cancel the job for a different user """ # Is it inefficient to get the job twice? Is it cached? - # Maybe if the call fails, we don't actually cancel the job? job = self.sdkmr.get_job_with_permission( job_id, JobPermissions.WRITE, as_admin=as_admin From add6f44e0d7e9c586c53aa5330d843d0f828bbc1 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 2 Aug 2021 10:27:02 -0500 Subject: [PATCH 21/31] Fix tests --- lib/execution_engine2/db/MongoUtil.py | 16 ++++++-------- test/tests_for_auth/ee2_admin_mode_test.py | 12 +++++------ test/tests_for_db/ee2_MongoUtil_test.py | 15 ++++++------- test/tests_for_sdkmr/EE2Runjob_test.py | 25 +++++++++++++++++++--- 4 files changed, 42 insertions(+), 26 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index e5d134825..58db813f7 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -270,9 +270,10 @@ def check_if_already_finished(job_status): def update_jobs_to_queued( self, job_id_pairs: List[JobIdPair], scheduler_type: str = "condor" ) -> None: - """ - Updates a list of created jobs to queued. Does not work on jobs that already have gone through a status transition. - If the record is not in the CREATED status, an InvalidStatusTransitionException will be raised. + f""" + * Adds scheduler id to list of jobs + * Updates a list of {Status.created.value} jobs to queued. Does not work on jobs that already have gone through any other + status transition. If the record is not in the {Status.created.value} status, nothing will happen :param job_id_pairs: A list of pairs of Job Ids and Scheduler Ids :param scheduler_type: The scheduler this job was queued in, default condor """ @@ -314,16 +315,13 @@ def update_jobs_to_queued( # Update provided jobs with scheduler id. Then only update non terminated jobs into updated status. mongo_collection = self.config["mongo-jobs-collection"] - from pymongo.collection import Collection - if bulk_update_scheduler_jobs: with self.pymongo_client(mongo_collection) as pymongo_client: - ee2_jobs_col = pymongo_client[self.mongo_database][ - mongo_collection - ] # type: Collection - + ee2_jobs_col = pymongo_client[self.mongo_database][mongo_collection] # Bulk Update to add scheduler ids ee2_jobs_col.bulk_write(bulk_update_scheduler_jobs, ordered=False) + # Bulk Update to add queued status ids + ee2_jobs_col.bulk_write(bulk_update_created_to_queued, ordered=False) def cancel_job(self, job_id=None, terminated_code=None): """ diff --git a/test/tests_for_auth/ee2_admin_mode_test.py b/test/tests_for_auth/ee2_admin_mode_test.py index bbc16c465..15ee0682a 100644 --- a/test/tests_for_auth/ee2_admin_mode_test.py +++ b/test/tests_for_auth/ee2_admin_mode_test.py @@ -162,7 +162,7 @@ def test_regular_user(self): ws_auth.can_write.assert_called_once_with(self.ws_id) # RUNJOB BUT ATTEMPT TO BE AN ADMIN - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=PermissionError, expected_regex=lowly_user ): runner.run_job(params=job_params_1, as_admin=True) @@ -172,7 +172,7 @@ def test_regular_user(self): self.assertEqual(params["method"], job_params_1["method"]) # get_job_params BUT ATTEMPT TO BE AN ADMIN - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=PermissionError, expected_regex=lowly_user ): runner.get_job_params(job_id=job_id, as_admin=True) @@ -188,12 +188,12 @@ def test_regular_user(self): runner.view_job_logs(job_id=job_id) # add_job_logs and view them, BUT ATTEMPT TO BE AN ADMIN - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=PermissionError, expected_regex=lowly_user ): runner.add_job_logs(job_id=job_id, log_lines=lines, as_admin=True) - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=PermissionError, expected_regex=lowly_user ): runner.view_job_logs(job_id=job_id, as_admin=True) @@ -288,7 +288,7 @@ def test_no_user(self): method_1 = "module_name.function_name" job_params_1 = get_sample_job_params(method=method_1, wsid=self.ws_id) - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=RuntimeError, expected_regex=r"ServerError\('Token validation failed: Login failed! Server responded with code 401 Unauthorized'\)", ): @@ -317,7 +317,7 @@ def test_admin_reader(self): self.assertEqual(admin_type, {"permission": "r"}) # RUNJOB - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=PermissionError, expected_regex=lowly_admin ): runner.run_job(params=job_params_1, as_admin=True) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 6c282c03b..194f7f278 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -59,7 +59,7 @@ def test_init_ok(self): self.assertTrue(set(class_attri) <= set(mongo_util.__dict__.keys())) def test_update_jobs_enmasse(self): - + """Check to see that created jobs get updated to queued""" for state in Status: job = get_example_job(status=Status.created.value).save() job2 = get_example_job(status=state.value).save() @@ -74,16 +74,15 @@ def test_update_jobs_enmasse(self): job2.reload() job3.reload() - # Success Case + # First job always should transition to queued + assert job.status == Status.queued.value + + # Created jobs should transition if state.value == Status.created.value: assert all(j.status == Status.queued.value for j in [job, job2, job3]) - # Fail Case, else: - # The created job should be queued - assert job.status == Status.queued.value - # Other statuses should remain unchanged - assert job2.status == state.value - assert job3.status == state.value + # Don't change their state + assert all(j.status == state.value for j in [job2, job3]) def test_get_by_cluster(self): """Get a job by its condor scheduler_id""" diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 31c77c493..019687da3 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -14,7 +14,13 @@ from execution_engine2.authorization.workspaceauth import WorkspaceAuth from execution_engine2.db.MongoUtil import MongoUtil -from execution_engine2.db.models.models import Job, JobInput, JobRequirements, Meta +from execution_engine2.db.models.models import ( + Job, + JobInput, + JobRequirements, + Meta, + Status, +) from execution_engine2.exceptions import ( IncorrectParamsException, AuthError, @@ -56,8 +62,8 @@ _USER = "someuser" _TOKEN = "tokentokentoken" _OTHER_USER = "some_sucker" -_CREATED_STATE = "created" -_QUEUED_STATE = "queued" +_CREATED_STATE = Status.created.value +_QUEUED_STATE = Status.queued.value # batch common variables _BATCH = "batch" @@ -803,7 +809,20 @@ def _set_up_common_return_values_batch(mocks): retjob_2 = Job() retjob_2.id = ObjectId(_JOB_ID_2) retjob_2.status = _CREATED_STATE + + retjob_1_after_submit = Job() + retjob_1_after_submit.id = ObjectId(_JOB_ID_1) + retjob_1_after_submit.status = _QUEUED_STATE + retjob_1_after_submit.scheduler_id = _CLUSTER_1 + retjob_2_after_submit = Job() + retjob_2_after_submit.id = ObjectId(_JOB_ID_2) + retjob_2_after_submit.status = _QUEUED_STATE + retjob_2_after_submit.scheduler_id = _CLUSTER_2 + mocks[MongoUtil].get_job.side_effect = [retjob_1, retjob_2] + mocks[MongoUtil].get_jobs.side_effect = [ + [retjob_1_after_submit, retjob_2_after_submit] + ] def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): From a637bb4b7cc0e39831d0dee56c1d6d3de007588d Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 2 Aug 2021 12:36:37 -0500 Subject: [PATCH 22/31] Add a test --- lib/execution_engine2/db/MongoUtil.py | 10 ++++++-- test/tests_for_db/ee2_MongoUtil_test.py | 24 +++++++++++++++++++ ...ee2_SDKMethodRunner_test_EE2Runjob_test.py | 4 ++-- test/tests_for_sdkmr/ee2_kafka_test.py | 2 +- 4 files changed, 35 insertions(+), 5 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 58db813f7..f0397eb57 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -282,8 +282,14 @@ def update_jobs_to_queued( bulk_update_created_to_queued = [] queue_time_now = datetime.utcnow().timestamp() for job_id_pair in job_id_pairs: - if None in job_id_pairs: - raise InvalidStatusTransitionException + if job_id_pair.job_id is None: + raise ValueError( + f"Provided a bad job_id_pair, missing job_id for {job_id_pair.scheduler_id}" + ) + elif job_id_pair.scheduler_id is None: + raise ValueError( + f"Provided a bad job_id_pair, missing scheduler_id for {job_id_pair.job_id}" + ) bulk_update_scheduler_jobs.append( UpdateOne( diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 194f7f278..a8db51e10 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -84,6 +84,30 @@ def test_update_jobs_enmasse(self): # Don't change their state assert all(j.status == state.value for j in [job2, job3]) + def test_update_jobs_enmasse_bad_job_pairs(self): + job = get_example_job(status=Status.created.value).save() + job2 = get_example_job(status=Status.created.value).save() + job3 = get_example_job(status=Status.created.value).save() + job_ids = [job.id, job2.id, job3.id] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, None] + job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) + + with self.assertRaisesRegex( + expected_exception=ValueError, + expected_regex=f"Provided a bad job_id_pair, missing scheduler_id for {job3.id}", + ): + self.getMongoUtil().update_jobs_to_queued(job_id_pairs) + + job_ids = [job.id, job2.id, None] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] + job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) + + with self.assertRaisesRegex( + expected_exception=ValueError, + expected_regex=f"Provided a bad job_id_pair, missing job_id for {job3.scheduler_id}", + ): + self.getMongoUtil().update_jobs_to_queued(job_id_pairs) + def test_get_by_cluster(self): """Get a job by its condor scheduler_id""" mongo_util = self.getMongoUtil() diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Runjob_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Runjob_test.py index 96ecd7349..f30c153f7 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Runjob_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Runjob_test.py @@ -334,7 +334,7 @@ def test_retry_job_multiple(self, rq_mock, condor_mock): "'123' is not a valid ObjectId, it must be a 12-byte input or a 24-character " "hex string" ) - with self.assertRaisesRegexp(RetryFailureException, errmsg): + with self.assertRaisesRegex(RetryFailureException, errmsg): runner.retry_multiple(job_ids=[parent_job_id1, 123]) # 3. Retry the jobs with duplicate job ids @@ -382,7 +382,7 @@ def test_retry_job_multiple(self, rq_mock, condor_mock): self.check_retry_job_state(parent_job_id4, job4["job_id"]) # Test no job ids - with self.assertRaisesRegexp(ValueError, "No job_ids provided to retry"): + with self.assertRaisesRegex(ValueError, "No job_ids provided to retry"): runner.retry_multiple(job_ids=None) # Test error during retry, but passing validate diff --git a/test/tests_for_sdkmr/ee2_kafka_test.py b/test/tests_for_sdkmr/ee2_kafka_test.py index ebd73a845..60856d18c 100644 --- a/test/tests_for_sdkmr/ee2_kafka_test.py +++ b/test/tests_for_sdkmr/ee2_kafka_test.py @@ -29,7 +29,7 @@ def setUpClass(cls): def test_status_change(self): - with self.assertRaisesRegexp( + with self.assertRaisesRegex( expected_exception=TypeError, expected_regex=r"__init__\(\) missing 1 required positional argument: 'scheduler_id'", ): From 9be3578a4c19a94b77568dabcafccd34f5245a99 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 2 Aug 2021 15:07:22 -0500 Subject: [PATCH 23/31] Increase coverage --- lib/execution_engine2/sdk/EE2Runjob.py | 5 +- test/tests_for_sdkmr/EE2Runjob_test.py | 167 +++++++++++++++++++++---- 2 files changed, 141 insertions(+), 31 deletions(-) diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index eb4962944..941d9b269 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -293,12 +293,9 @@ def _run_multiple(self, runjob_params): job_id=str(job_id), user=self.sdkmr.get_user_id() ) ) - # Submit to Condor - condor_time = time.time() try: submission_ids = self._submit_multiple(job_submission_params) - self.logger.debug(f"condor_time submit = {time.time() - condor_time}") return submission_ids except Exception as e: self._abort_multiple_jobs(job_ids) @@ -330,7 +327,7 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): ) elif job.status == Status.terminated.value: # Remove from the queue, now that the scheduler_id is available - self._safe_cancel(job_id, TerminatedCode.terminated_by_user.value) + self._safe_cancel(job_id, TerminatedCode.terminated_by_user) def _submit_multiple(self, job_submission_params): """ diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index 019687da3..a821cd34d 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -769,7 +769,7 @@ def _run_and_run_batch_fail( _run_batch_fail(rj, [params], {}, as_admin, expected) -def _set_up_common_return_values_batch(mocks): +def _set_up_common_return_values_batch(mocks, returned_job_state=None): """ Set up return values on mocks that are the same for several tests. """ @@ -812,11 +812,15 @@ def _set_up_common_return_values_batch(mocks): retjob_1_after_submit = Job() retjob_1_after_submit.id = ObjectId(_JOB_ID_1) - retjob_1_after_submit.status = _QUEUED_STATE + retjob_1_after_submit.status = ( + _QUEUED_STATE if not returned_job_state else returned_job_state + ) retjob_1_after_submit.scheduler_id = _CLUSTER_1 retjob_2_after_submit = Job() retjob_2_after_submit.id = ObjectId(_JOB_ID_2) - retjob_2_after_submit.status = _QUEUED_STATE + retjob_2_after_submit.status = ( + _QUEUED_STATE if not returned_job_state else returned_job_state + ) retjob_2_after_submit.scheduler_id = _CLUSTER_2 mocks[MongoUtil].get_job.side_effect = [retjob_1, retjob_2] @@ -825,7 +829,9 @@ def _set_up_common_return_values_batch(mocks): ] -def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): +def _check_common_mock_calls_batch( + mocks, reqs1, reqs2, parent_wsid, terminated_during_submit=False +): """ Check that mocks are called as expected when those calls are similar or the same for several tests. @@ -913,29 +919,44 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): ] mocks[MongoUtil].update_jobs_to_queued.assert_has_calls([call(child_job_pairs)]) - mocks[KafkaClient].send_kafka_message.assert_has_calls( - [ - call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job - call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), - call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), - call( - KafkaQueueChange( - job_id=_JOB_ID_1, - new_status=_QUEUED_STATE, - previous_status=_CREATED_STATE, - scheduler_id=_CLUSTER_1, - ) - ), - call( - KafkaQueueChange( - job_id=_JOB_ID_2, - new_status=_QUEUED_STATE, - previous_status=_CREATED_STATE, - scheduler_id=_CLUSTER_2, - ) - ), - ] - ) + if not terminated_during_submit: + mocks[KafkaClient].send_kafka_message.assert_has_calls( + [ + call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job + call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), + call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), + call( + KafkaQueueChange( + job_id=_JOB_ID_1, + new_status=_QUEUED_STATE, + previous_status=_CREATED_STATE, + scheduler_id=_CLUSTER_1, + ) + ), + call( + KafkaQueueChange( + job_id=_JOB_ID_2, + new_status=_QUEUED_STATE, + previous_status=_CREATED_STATE, + scheduler_id=_CLUSTER_2, + ) + ), + ] + ) + else: + mocks[KafkaClient].send_kafka_message.assert_has_calls( + [ + call(KafkaCreateJob(job_id=_JOB_ID, user=_USER)), # parent job + call(KafkaCreateJob(job_id=_JOB_ID_1, user=_USER)), + call(KafkaCreateJob(job_id=_JOB_ID_2, user=_USER)), + ] + ) + mocks[SDKMethodRunner].cancel_job.assert_has_calls( + [ + call(job_id=_JOB_ID_1, terminated_code=0), + call(job_id=_JOB_ID_2, terminated_code=0), + ] + ) # Removed for now, but might be added back in if run_job_message is re-added # mocks[SlackClient].run_job_message.assert_has_calls( @@ -963,6 +984,98 @@ def _check_common_mock_calls_batch(mocks, reqs1, reqs2, parent_wsid): assert_jobs_equal(batch_job, expected_batch_container) +def test_run_job_batch_with_cancellation_during_submit(): + """ + A basic unit test of the run_batch() method, providing a workspace ID for the parent job. This one also checks for + cancellation during submit causing a job cancellation request to be processed . + + This test is a fairly minimal test of the run_batch() method. It does not exercise all the + potential code paths or provide all the possible run inputs, such as job parameters, cell + metadata, etc. + """ + # When an assertion is failed, this test doesn't show you where failed in PyCharm, so use + # Additional arguments `--no-cov -s` or run from cmd line + # PYTHONPATH=.:lib:test pytest test/tests_for_sdkmr/EE2Runjob_test.py::test_run_job_batch_with_parent_job_wsid --no-cov + + # set up variables + parent_wsid = 89 + wsid = 32 + + # set up mocks + mocks = _set_up_mocks(_USER, _TOKEN) + sdkmr = mocks[SDKMethodRunner] + jrr = mocks[JobRequirementsResolver] + # We intentionally do not check the logger methods as there are a lot of them and this is + # already a very large test. This may be something to be added later when needed. + + # Set up call returns. These calls are in the order they occur in the code + mocks[WorkspaceAuth].can_write.return_value = True + mocks[WorkspaceAuth].can_write_list.return_value = {wsid: True} + + jrr.normalize_job_reqs.side_effect = [{}, {}] + jrr.get_requirements_type.side_effect = [ + RequirementsType.STANDARD, + RequirementsType.STANDARD, + ] + reqs1 = ResolvedRequirements( + cpus=1, + memory_MB=2, + disk_GB=3, + client_group="cg1", + ) + reqs2 = ResolvedRequirements( + cpus=10, + memory_MB=20, + disk_GB=30, + client_group="cg2", + ) + jrr.resolve_requirements.side_effect = [reqs1, reqs2] + + _set_up_common_return_values_batch( + mocks, returned_job_state=Status.terminated.value + ) + + # set up the class to be tested and run the method + rj = EE2RunJob(sdkmr) + params = [ + { + "method": _METHOD_1, + "app_id": _APP_1, + "source_ws_objects": [_WS_REF_1, _WS_REF_2], + }, + { + "method": _METHOD_2, + "app_id": _APP_2, + "wsid": wsid, + }, + ] + params[1]["wsid"] = None + assert rj.run_batch(params, {"wsid": parent_wsid}) == { + "batch_id": _JOB_ID, + "child_job_ids": [_JOB_ID_1, _JOB_ID_2], + } + + # check mocks called as expected. The order here is the order that they're called in the code. + mocks[WorkspaceAuth].can_write.assert_called_once_with(parent_wsid) + + jrr = mocks[JobRequirementsResolver] + jrr.normalize_job_reqs.assert_has_calls( + [call({}, "input job"), call({}, "input job")] + ) + jrr.get_requirements_type.assert_has_calls( + [call(**_EMPTY_JOB_REQUIREMENTS), call(**_EMPTY_JOB_REQUIREMENTS)] + ) + jrr.resolve_requirements.assert_has_calls( + [ + call(_METHOD_1, mocks[CatalogCache], **_EMPTY_JOB_REQUIREMENTS), + call(_METHOD_2, mocks[CatalogCache], **_EMPTY_JOB_REQUIREMENTS), + ] + ) + _check_common_mock_calls_batch( + mocks, reqs1, reqs2, parent_wsid, terminated_during_submit=True + ) + + def test_run_job_batch_with_parent_job_wsid(): """ A basic unit test of the run_batch() method, providing a workspace ID for the parent job. From 0183f9842a47616b08df3735255d4f531c174920 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 2 Aug 2021 22:29:08 -0500 Subject: [PATCH 24/31] Removed unused, fixed tests --- lib/execution_engine2/db/MongoUtil.py | 2 +- lib/execution_engine2/sdk/EE2Runjob.py | 2 ++ lib/execution_engine2/sdk/EE2Status.py | 6 +++--- test/tests_for_db/ee2_MongoUtil_test.py | 1 - .../ee2_SDKMethodRunner_test_EE2Status_test.py | 8 ++++++-- 5 files changed, 12 insertions(+), 7 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index f0397eb57..0bc0957bc 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -4,7 +4,7 @@ import traceback from contextlib import contextmanager from datetime import datetime -from typing import Dict, List, Tuple +from typing import Dict, List from bson.objectid import ObjectId from mongoengine import connect, connection diff --git a/lib/execution_engine2/sdk/EE2Runjob.py b/lib/execution_engine2/sdk/EE2Runjob.py index 941d9b269..caaa7cdbd 100644 --- a/lib/execution_engine2/sdk/EE2Runjob.py +++ b/lib/execution_engine2/sdk/EE2Runjob.py @@ -327,6 +327,8 @@ def _update_to_queued_multiple(self, job_ids, scheduler_ids): ) elif job.status == Status.terminated.value: # Remove from the queue, now that the scheduler_id is available + # The job record doesn't actually get updated in the db a 2nd time, and this TerminatedCode is only + # used by the initial transition to Terminated self._safe_cancel(job_id, TerminatedCode.terminated_by_user) def _submit_multiple(self, job_submission_params): diff --git a/lib/execution_engine2/sdk/EE2Status.py b/lib/execution_engine2/sdk/EE2Status.py index 9b2436d83..053cfb77d 100644 --- a/lib/execution_engine2/sdk/EE2Status.py +++ b/lib/execution_engine2/sdk/EE2Status.py @@ -373,9 +373,9 @@ def finish_job( def _update_finished_job_with_usage(self, job_id, as_admin=None) -> Dict: """ # TODO Does this need a kafka message? - :param job_id: - :param as_admin: - :return: + # TODO EE2 issue #251 : The saved job stats are inaccurate: + # The usage is not recorded until the job is completely finished. + :return: Resources at the time the job almost finished. """ # note this method is replaced by a magic mock in some tests job = self.sdkmr.get_job_with_permission( diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index a8db51e10..268f25d6e 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -8,7 +8,6 @@ from execution_engine2.db.MongoUtil import MongoUtil from execution_engine2.db.models.models import Job, JobLog, Status from execution_engine2.sdk.EE2Runjob import JobIdPair -from execution_engine2.exceptions import InvalidStatusTransitionException from test.utils_shared.test_utils import ( bootstrap, get_example_job, diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py index 98e73cf70..5d36513b7 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py @@ -131,7 +131,7 @@ def test_check_job(self, rq_mock, condor_mock): @patch("lib.execution_engine2.utils.Condor.Condor", autospec=True) def test_run_job_and_handle_held(self, rq_mock, condor_mock): """ - Run a job, then call it held as an admin, and then check to see if the record contains condor info about the job + Run a job, then call it held as an admin, and then check to see if the record is set to error :param rq_mock: :param condor_mock: :return: @@ -157,8 +157,12 @@ def test_run_job_and_handle_held(self, rq_mock, condor_mock): print( f"Job id is {job_id}. Status is {check_job.get('status')} Cluster is {check_job.get('scheduler_id')} " ) + self.assertEqual(check_job.get("status"), Status.queued.value) job_record = runner.handle_held_job(cluster_id=check_job.get("scheduler_id")) - self.assertEqual(self.fake_used_resources, job_record.get("condor_job_ads")) + self.assertEqual(job_record.get("status"), Status.error.value) + # Condor ads are actually wrong and should only be updated after the job is completed, + # so we don't need to check them in this test right now. + # See EE2 issue #251 def test_update_job_status(self): runner = self.getRunner() From af03d77aa43d4888596a9494b009327a8a2035c5 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 2 Aug 2021 22:38:59 -0500 Subject: [PATCH 25/31] Removed unused, fixed tests --- .../ee2_SDKMethodRunner_test_EE2Status_test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py index 5d36513b7..cc2cddeff 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py @@ -159,7 +159,10 @@ def test_run_job_and_handle_held(self, rq_mock, condor_mock): ) self.assertEqual(check_job.get("status"), Status.queued.value) job_record = runner.handle_held_job(cluster_id=check_job.get("scheduler_id")) - self.assertEqual(job_record.get("status"), Status.error.value) + # This flaky test changes depending on your test environment + self.assertIn( + job_record.get("status"), [Status.terminated.value, Status.error.value] + ) # Condor ads are actually wrong and should only be updated after the job is completed, # so we don't need to check them in this test right now. # See EE2 issue #251 From 76fcb1af2156ec700dd0ea7267291f4571f50709 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Mon, 2 Aug 2021 22:41:20 -0500 Subject: [PATCH 26/31] Removed unused, fixed tests --- test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py index cc2cddeff..f356d8ce0 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test_EE2Status_test.py @@ -131,7 +131,7 @@ def test_check_job(self, rq_mock, condor_mock): @patch("lib.execution_engine2.utils.Condor.Condor", autospec=True) def test_run_job_and_handle_held(self, rq_mock, condor_mock): """ - Run a job, then call it held as an admin, and then check to see if the record is set to error + Run a job, then call it held as an admin, and then check to see if the record is set to error or terminated :param rq_mock: :param condor_mock: :return: From 3d517d94d62873865ea0d02412faeaafdc2498ce Mon Sep 17 00:00:00 2001 From: bio-boris Date: Thu, 5 Aug 2021 01:07:50 -0500 Subject: [PATCH 27/31] Add tests --- lib/execution_engine2/db/MongoUtil.py | 2 +- test/tests_for_db/ee2_MongoUtil_test.py | 663 +++++++++++++----------- test/tests_for_sdkmr/EE2Runjob_test.py | 16 +- 3 files changed, 356 insertions(+), 325 deletions(-) diff --git a/lib/execution_engine2/db/MongoUtil.py b/lib/execution_engine2/db/MongoUtil.py index 0bc0957bc..0845cd84b 100644 --- a/lib/execution_engine2/db/MongoUtil.py +++ b/lib/execution_engine2/db/MongoUtil.py @@ -298,7 +298,6 @@ def update_jobs_to_queued( }, { "$set": { - "queued": queue_time_now, "scheduler_id": job_id_pair.scheduler_id, "scheduler_type": scheduler_type, } @@ -314,6 +313,7 @@ def update_jobs_to_queued( { "$set": { "status": Status.queued.value, + "queued": queue_time_now, } }, ) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 268f25d6e..fbaed3146 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -2,6 +2,7 @@ import logging import os import unittest +from datetime import datetime from bson.objectid import ObjectId @@ -12,6 +13,7 @@ bootstrap, get_example_job, read_config_into_dict, + assert_close_to_now, ) from tests_for_db.mongo_test_helper import MongoTestHelper @@ -57,365 +59,398 @@ def test_init_ok(self): mongo_util = self.getMongoUtil() self.assertTrue(set(class_attri) <= set(mongo_util.__dict__.keys())) + def test_insert_jobs(self): + """Check to see that jobs are inserted into mongo""" + job = get_example_job(status=Status.created.value) + job2 = get_example_job(status=Status.created.value) + jobs_to_insert = [job, job2] + job_ids = self.getMongoUtil().insert_jobs(jobs_to_insert) + assert len(job_ids) == len(jobs_to_insert) + def test_update_jobs_enmasse(self): """Check to see that created jobs get updated to queued""" for state in Status: - job = get_example_job(status=Status.created.value).save() - job2 = get_example_job(status=state.value).save() - job3 = get_example_job(status=state.value).save() + job = get_example_job(status=Status.created.value, scheduler_id=None) + job2 = get_example_job(status=state.value, scheduler_id=None) + job3 = get_example_job(status=state.value, scheduler_id=None) + jobs = [job, job2, job3] + + for j in jobs: + j.scheduler_id = None + j.save() + assert j.scheduler_id is None job_ids = [job.id, job2.id, job3.id] - scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] + scheduler_ids = ["humpty", "dumpty", "alice"] jobs_to_update = list(map(JobIdPair, job_ids, scheduler_ids)) + now_ms = datetime.utcnow().timestamp() + self.getMongoUtil().update_jobs_to_queued(jobs_to_update) job.reload() job2.reload() job3.reload() - # First job always should transition to queued - assert job.status == Status.queued.value - - # Created jobs should transition - if state.value == Status.created.value: - assert all(j.status == Status.queued.value for j in [job, job2, job3]) - else: - # Don't change their state - assert all(j.status == state.value for j in [job2, job3]) - - def test_update_jobs_enmasse_bad_job_pairs(self): - job = get_example_job(status=Status.created.value).save() - job2 = get_example_job(status=Status.created.value).save() - job3 = get_example_job(status=Status.created.value).save() - job_ids = [job.id, job2.id, job3.id] - scheduler_ids = [job.scheduler_id, job2.scheduler_id, None] - job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) - - with self.assertRaisesRegex( - expected_exception=ValueError, - expected_regex=f"Provided a bad job_id_pair, missing scheduler_id for {job3.id}", - ): - self.getMongoUtil().update_jobs_to_queued(job_id_pairs) - - job_ids = [job.id, job2.id, None] - scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] - job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) - - with self.assertRaisesRegex( - expected_exception=ValueError, - expected_regex=f"Provided a bad job_id_pair, missing job_id for {job3.scheduler_id}", - ): - self.getMongoUtil().update_jobs_to_queued(job_id_pairs) - - def test_get_by_cluster(self): - """Get a job by its condor scheduler_id""" - mongo_util = self.getMongoUtil() - with mongo_util.mongo_engine_connection(): - job = get_example_job() - job_id = job.save().id - batch = mongo_util.get_job_batch_name(job.scheduler_id) - self.assertEqual(str(job_id), batch) + # Check that sched ids are set + for i, val in enumerate(scheduler_ids): + assert jobs[i].scheduler_id == val + assert jobs[i].scheduler_type == "condor" + + # Checks that a timestamp in seconds since the epoch is within a second of the current time. + for j in jobs: + assert now_ms + 1 > j.updated + assert now_ms - 1 < j.updated + + # First job always should transition to queued + assert job.status == Status.queued.value + + # Created jobs should transition + if state.value == Status.created.value: + assert all(j.status == Status.queued.value for j in [job, job2, job3]) + + else: + # Don't change their state + assert all(j.status == state.value for j in [job2, job3]) + + +def test_update_jobs_enmasse_bad_job_pairs(self): + job = get_example_job(status=Status.created.value).save() + job2 = get_example_job(status=Status.created.value).save() + job3 = get_example_job(status=Status.created.value).save() + job_ids = [job.id, job2.id, job3.id] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, None] + job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) + + with self.assertRaisesRegex( + expected_exception=ValueError, + expected_regex=f"Provided a bad job_id_pair, missing scheduler_id for {job3.id}", + ): + self.getMongoUtil().update_jobs_to_queued(job_id_pairs) + + job_ids = [job.id, job2.id, None] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] + job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) + + with self.assertRaisesRegex( + expected_exception=ValueError, + expected_regex=f"Provided a bad job_id_pair, missing job_id for {job3.scheduler_id}", + ): + self.getMongoUtil().update_jobs_to_queued(job_id_pairs) + + +def test_get_by_cluster(self): + """Get a job by its condor scheduler_id""" + mongo_util = self.getMongoUtil() + with mongo_util.mongo_engine_connection(): + job = get_example_job() + job_id = job.save().id + batch = mongo_util.get_job_batch_name(job.scheduler_id) + self.assertEqual(str(job_id), batch) + + +def test_get_job_ok(self): + mongo_util = self.getMongoUtil() + + with mongo_util.mongo_engine_connection(): + ori_job_count = Job.objects.count() + job = get_example_job() + job_id = job.save().id + self.assertEqual(ori_job_count, Job.objects.count() - 1) + + # get job with no exclude_fields + job = mongo_util.get_job(job_id=job_id).to_mongo().to_dict() + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "job_input", + "scheduler_id", + "child_jobs", + "batch_job", + "retry_ids", + "retry_saved_toggle", + ] + self.assertCountEqual(job.keys(), expected_keys) - def test_get_job_ok(self): + # get job with exclude_fields + job = ( + mongo_util.get_job(job_id=job_id, exclude_fields=["job_input"]) + .to_mongo() + .to_dict() + ) - mongo_util = self.getMongoUtil() + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + self.assertCountEqual(job.keys(), expected_keys) - with mongo_util.mongo_engine_connection(): - ori_job_count = Job.objects.count() - job = get_example_job() - job_id = job.save().id - self.assertEqual(ori_job_count, Job.objects.count() - 1) - - # get job with no exclude_fields - job = mongo_util.get_job(job_id=job_id).to_mongo().to_dict() - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "job_input", - "scheduler_id", - "child_jobs", - "batch_job", - "retry_ids", - "retry_saved_toggle", - ] - self.assertCountEqual(job.keys(), expected_keys) - - # get job with exclude_fields - job = ( - mongo_util.get_job(job_id=job_id, exclude_fields=["job_input"]) - .to_mongo() - .to_dict() - ) - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - self.assertCountEqual(job.keys(), expected_keys) - - # get job with multiple exclude_fields - job = ( - mongo_util.get_job(job_id=job_id, exclude_fields=["user", "wsid"]) - .to_mongo() - .to_dict() - ) - - expected_keys = [ - "_id", - "authstrat", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - self.assertCountEqual(job.keys(), expected_keys) - - mongo_util.get_job(job_id=job_id).delete() - self.assertEqual(ori_job_count, Job.objects.count()) - - def test_get_jobs_ok(self): + # get job with multiple exclude_fields + job = ( + mongo_util.get_job(job_id=job_id, exclude_fields=["user", "wsid"]) + .to_mongo() + .to_dict() + ) - mongo_util = self.getMongoUtil() + expected_keys = [ + "_id", + "authstrat", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + self.assertCountEqual(job.keys(), expected_keys) + + mongo_util.get_job(job_id=job_id).delete() + self.assertEqual(ori_job_count, Job.objects.count()) + + +def test_get_jobs_ok(self): + mongo_util = self.getMongoUtil() + + with mongo_util.mongo_engine_connection(): + ori_job_count = Job.objects.count() + job = get_example_job() + job_id_1 = job.save().id + job = get_example_job() + job_id_2 = job.save().id + self.assertEqual(ori_job_count, Job.objects.count() - 2) + + # get jobs with no exclude_fields + jobs = mongo_util.get_jobs(job_ids=[job_id_1, job_id_2]) + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] - with mongo_util.mongo_engine_connection(): - ori_job_count = Job.objects.count() - job = get_example_job() - job_id_1 = job.save().id - job = get_example_job() - job_id_2 = job.save().id - self.assertEqual(ori_job_count, Job.objects.count() - 2) - - # get jobs with no exclude_fields - jobs = mongo_util.get_jobs(job_ids=[job_id_1, job_id_2]) - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - - for job in jobs: - self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) - - # get jobs with multiple exclude_fields - jobs = mongo_util.get_jobs( - job_ids=[job_id_1, job_id_2], exclude_fields=["user", "wsid"] - ) - - expected_keys = [ - "_id", - "authstrat", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - for job in jobs: - self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) - - mongo_util.get_job(job_id=job_id_1).delete() - mongo_util.get_job(job_id=job_id_2).delete() - self.assertEqual(ori_job_count, Job.objects.count()) - - def test_connection_ok(self): + for job in jobs: + self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) - mongo_util = self.getMongoUtil() + # get jobs with multiple exclude_fields + jobs = mongo_util.get_jobs( + job_ids=[job_id_1, job_id_2], exclude_fields=["user", "wsid"] + ) + + expected_keys = [ + "_id", + "authstrat", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + for job in jobs: + self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) + + mongo_util.get_job(job_id=job_id_1).delete() + mongo_util.get_job(job_id=job_id_2).delete() + self.assertEqual(ori_job_count, Job.objects.count()) + + +def test_connection_ok(self): + mongo_util = self.getMongoUtil() + + with mongo_util.mongo_engine_connection(): + ori_job_count = Job.objects.count() + j = get_example_job() + j.save() + self.assertEqual(ori_job_count, Job.objects.count() - 1) + + job = mongo_util.get_job(job_id=j.id).to_mongo().to_dict() + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + + self.assertCountEqual(job.keys(), expected_keys) + self.assertEqual(job["user"], j.user) + self.assertEqual(job["authstrat"], "kbaseworkspace") + self.assertEqual(job["wsid"], j.wsid) + + mongo_util.get_job(job_id=j.id).delete() + self.assertEqual(ori_job_count, Job.objects.count()) - with mongo_util.mongo_engine_connection(): - ori_job_count = Job.objects.count() - j = get_example_job() - j.save() - self.assertEqual(ori_job_count, Job.objects.count() - 1) - - job = mongo_util.get_job(job_id=j.id).to_mongo().to_dict() - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - - self.assertCountEqual(job.keys(), expected_keys) - self.assertEqual(job["user"], j.user) - self.assertEqual(job["authstrat"], "kbaseworkspace") - self.assertEqual(job["wsid"], j.wsid) - - mongo_util.get_job(job_id=j.id).delete() - self.assertEqual(ori_job_count, Job.objects.count()) - - def test_insert_one_ok(self): - mongo_util = self.getMongoUtil() - with mongo_util.pymongo_client( +def test_insert_one_ok(self): + mongo_util = self.getMongoUtil() + + with mongo_util.pymongo_client( + self.config["mongo-jobs-collection"] + ) as pymongo_client: + col = pymongo_client[self.config["mongo-database"]][ self.config["mongo-jobs-collection"] - ) as pymongo_client: - col = pymongo_client[self.config["mongo-database"]][ - self.config["mongo-jobs-collection"] - ] + ] - ori_job_count = col.count_documents({}) - doc = {"test_key": "foo"} - job_id = mongo_util.insert_one(doc) - self.assertEqual(ori_job_count, col.count_documents({}) - 1) + ori_job_count = col.count_documents({}) + doc = {"test_key": "foo"} + job_id = mongo_util.insert_one(doc) + self.assertEqual(ori_job_count, col.count_documents({}) - 1) - result = list(col.find({"_id": ObjectId(job_id)}))[0] - self.assertEqual(result["test_key"], "foo") + result = list(col.find({"_id": ObjectId(job_id)}))[0] + self.assertEqual(result["test_key"], "foo") - col.delete_one({"_id": ObjectId(job_id)}) - self.assertEqual(col.count_documents({}), ori_job_count) + col.delete_one({"_id": ObjectId(job_id)}) + self.assertEqual(col.count_documents({}), ori_job_count) - def test_find_in_ok(self): - mongo_util = self.getMongoUtil() - with mongo_util.pymongo_client( - self.config["mongo-jobs-collection"] - ) as pymongo_client: - col = pymongo_client[self.config["mongo-database"]][ - self.config["mongo-jobs-collection"] - ] - - ori_job_count = col.count_documents({}) - doc = {"test_key_1": "foo", "test_key_2": "bar"} - job_id = mongo_util.insert_one(doc) - self.assertEqual(ori_job_count, col.count_documents({}) - 1) - - # test query empty field - elements = ["foobar"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 0) - - # test query "foo" - elements = ["foo"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 1) - doc = docs.next() - self.assertTrue("_id" in doc.keys()) - self.assertTrue(doc.get("_id"), job_id) - self.assertEqual(doc.get("test_key_1"), "foo") - - col.delete_one({"_id": ObjectId(job_id)}) - self.assertEqual(col.count_documents({}), ori_job_count) - - def test_update_one_ok(self): - mongo_util = self.getMongoUtil() +def test_find_in_ok(self): + mongo_util = self.getMongoUtil() - with mongo_util.pymongo_client( + with mongo_util.pymongo_client( + self.config["mongo-jobs-collection"] + ) as pymongo_client: + col = pymongo_client[self.config["mongo-database"]][ self.config["mongo-jobs-collection"] - ) as pymongo_client: - col = pymongo_client[self.config["mongo-database"]][ - self.config["mongo-jobs-collection"] - ] + ] - ori_job_count = col.count_documents({}) - doc = {"test_key_1": "foo"} - job_id = mongo_util.insert_one(doc) - self.assertEqual(ori_job_count, col.count_documents({}) - 1) + ori_job_count = col.count_documents({}) + doc = {"test_key_1": "foo", "test_key_2": "bar"} + job_id = mongo_util.insert_one(doc) + self.assertEqual(ori_job_count, col.count_documents({}) - 1) - elements = ["foo"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 1) - doc = docs.next() - self.assertTrue("_id" in doc.keys()) - self.assertTrue(doc.get("_id"), job_id) - self.assertEqual(doc.get("test_key_1"), "foo") + # test query empty field + elements = ["foobar"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 0) - mongo_util.update_one({"test_key_1": "bar"}, job_id) + # test query "foo" + elements = ["foo"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 1) + doc = docs.next() + self.assertTrue("_id" in doc.keys()) + self.assertTrue(doc.get("_id"), job_id) + self.assertEqual(doc.get("test_key_1"), "foo") - elements = ["foo"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 0) + col.delete_one({"_id": ObjectId(job_id)}) + self.assertEqual(col.count_documents({}), ori_job_count) - elements = ["bar"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 1) - col.delete_one({"_id": ObjectId(job_id)}) - self.assertEqual(col.count_documents({}), ori_job_count) +def test_update_one_ok(self): + mongo_util = self.getMongoUtil() - def test_delete_one_ok(self): - mongo_util = MongoUtil(self.config) - with mongo_util.pymongo_client(self.config["mongo-jobs-collection"]) as pc: - col = pc.get_database(self.config["mongo-database"]).get_collection( - self.config["mongo-jobs-collection"] - ) + with mongo_util.pymongo_client( + self.config["mongo-jobs-collection"] + ) as pymongo_client: + col = pymongo_client[self.config["mongo-database"]][ + self.config["mongo-jobs-collection"] + ] - doc_count = col.count_documents({}) - logging.info("Found {} documents".format(doc_count)) + ori_job_count = col.count_documents({}) + doc = {"test_key_1": "foo"} + job_id = mongo_util.insert_one(doc) + self.assertEqual(ori_job_count, col.count_documents({}) - 1) - doc = {"test_key_1": "foo", "test_key_2": "bar"} - job_id = mongo_util.insert_one(doc) + elements = ["foo"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 1) + doc = docs.next() + self.assertTrue("_id" in doc.keys()) + self.assertTrue(doc.get("_id"), job_id) + self.assertEqual(doc.get("test_key_1"), "foo") - self.assertEqual(col.count_documents({}), doc_count + 1) - logging.info("Assert 0 documents") - mongo_util.delete_one(job_id) - self.assertEqual(col.count_documents({}), doc_count) + mongo_util.update_one({"test_key_1": "bar"}, job_id) - def test_get_job_log_pymongo_ok(self): + elements = ["foo"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 0) - mongo_util = self.getMongoUtil() + elements = ["bar"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 1) - primary_key = ObjectId() + col.delete_one({"_id": ObjectId(job_id)}) + self.assertEqual(col.count_documents({}), ori_job_count) - jl = JobLog() - jl.primary_key = primary_key - jl.original_line_count = 0 - jl.stored_line_count = 0 - jl.lines = [] - with mongo_util.pymongo_client( +def test_delete_one_ok(self): + mongo_util = MongoUtil(self.config) + with mongo_util.pymongo_client(self.config["mongo-jobs-collection"]) as pc: + col = pc.get_database(self.config["mongo-database"]).get_collection( self.config["mongo-jobs-collection"] - ) as pymongo_client: - jl_col = pymongo_client[self.config["mongo-database"]][ - self.config["mongo-logs-collection"] - ] + ) + + doc_count = col.count_documents({}) + logging.info("Found {} documents".format(doc_count)) + + doc = {"test_key_1": "foo", "test_key_2": "bar"} + job_id = mongo_util.insert_one(doc) + + self.assertEqual(col.count_documents({}), doc_count + 1) + logging.info("Assert 0 documents") + mongo_util.delete_one(job_id) + self.assertEqual(col.count_documents({}), doc_count) + + +def test_get_job_log_pymongo_ok(self): + mongo_util = self.getMongoUtil() + + primary_key = ObjectId() + + jl = JobLog() + jl.primary_key = primary_key + jl.original_line_count = 0 + jl.stored_line_count = 0 + jl.lines = [] + + with mongo_util.pymongo_client( + self.config["mongo-jobs-collection"] + ) as pymongo_client: + jl_col = pymongo_client[self.config["mongo-database"]][ + self.config["mongo-logs-collection"] + ] - ori_jl_count = jl_col.count_documents({}) + ori_jl_count = jl_col.count_documents({}) - jl.save() # save job log + jl.save() # save job log - self.assertEqual(JobLog.objects.count(), ori_jl_count + 1) - job_log = mongo_util.get_job_log_pymongo(str(primary_key)) + self.assertEqual(JobLog.objects.count(), ori_jl_count + 1) + job_log = mongo_util.get_job_log_pymongo(str(primary_key)) - self.assertEqual(job_log.get("original_line_count"), 0) - self.assertEqual(job_log.get("stored_line_count"), 0) - self.assertIsNone(job_log.get("lines")) + self.assertEqual(job_log.get("original_line_count"), 0) + self.assertEqual(job_log.get("stored_line_count"), 0) + self.assertIsNone(job_log.get("lines")) diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index a821cd34d..eefa2841e 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -189,9 +189,7 @@ def _set_up_common_return_values(mocks): mocks[MongoUtil].get_job.return_value = retjob -def _check_common_mock_calls( - mocks, reqs, wsid, app=_APP, parent_job_id=None, batch=False -): +def _check_common_mock_calls(mocks, reqs, wsid, app=_APP, parent_job_id=None): """ Check that mocks are called as expected when those calls are similar or the same for several tests. @@ -769,7 +767,7 @@ def _run_and_run_batch_fail( _run_batch_fail(rj, [params], {}, as_admin, expected) -def _set_up_common_return_values_batch(mocks, returned_job_state=None): +def _set_up_common_return_values_batch(mocks, returned_job_state=_QUEUED_STATE): """ Set up return values on mocks that are the same for several tests. """ @@ -812,15 +810,11 @@ def _set_up_common_return_values_batch(mocks, returned_job_state=None): retjob_1_after_submit = Job() retjob_1_after_submit.id = ObjectId(_JOB_ID_1) - retjob_1_after_submit.status = ( - _QUEUED_STATE if not returned_job_state else returned_job_state - ) + retjob_1_after_submit.status = returned_job_state retjob_1_after_submit.scheduler_id = _CLUSTER_1 retjob_2_after_submit = Job() retjob_2_after_submit.id = ObjectId(_JOB_ID_2) - retjob_2_after_submit.status = ( - _QUEUED_STATE if not returned_job_state else returned_job_state - ) + retjob_2_after_submit.status = returned_job_state retjob_2_after_submit.scheduler_id = _CLUSTER_2 mocks[MongoUtil].get_job.side_effect = [retjob_1, retjob_2] @@ -918,6 +912,8 @@ def _check_common_mock_calls_batch( JobIdPair(_JOB_ID_2, _CLUSTER_2), ] mocks[MongoUtil].update_jobs_to_queued.assert_has_calls([call(child_job_pairs)]) + job_ids = [child_job_pair.job_id for child_job_pair in child_job_pairs] + mocks[MongoUtil].get_jobs.assert_has_calls([call(job_ids)]) if not terminated_during_submit: mocks[KafkaClient].send_kafka_message.assert_has_calls( From 8e19ccf9ab90367c1272bd78d6160e46d63734a9 Mon Sep 17 00:00:00 2001 From: bio-boris Date: Thu, 5 Aug 2021 13:54:49 -0500 Subject: [PATCH 28/31] Add tests --- .../ee2_SDKMethodRunner_test.py | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py index b8b96c187..aa047cc20 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py @@ -174,7 +174,14 @@ def test_getters(self): is clients_and_mocks[JobRequirementsResolver] ) - def test_save_job(self): + def test_save_jobs(self): + ws = Workspace("https://fake.com") + wsa = WorkspaceAuth("user", ws) + cliset = UserClientSet("user", "token", ws, wsa) + clients_and_mocks = get_client_mocks(self.cfg, self.config_file, *ALL_CLIENTS) + sdkmr = SDKMethodRunner(cliset, clients_and_mocks[ClientSet]) + + def test_save_job_and_save_jobs(self): ws = Workspace("https://fake.com") wsa = WorkspaceAuth("user", ws) cliset = UserClientSet("user", "token", ws, wsa) @@ -190,9 +197,30 @@ def test_save_job(self): j = create_autospec(Job, spec_set=False, instance=True) j.id = bson.objectid.ObjectId("603051cfaf2e3401b0500982") assert sdkmr.save_job(j) == "603051cfaf2e3401b0500982" - j.save.assert_called_once_with() + # Test Save Jobs + job1 = Job() + job1.id = bson.objectid.ObjectId("603051cfaf2e3401b0500980") + job2 = Job() + job2.id = bson.objectid.ObjectId("603051cfaf2e3401b0500981") + sdkmr.get_mongo_util().insert_jobs.return_value = [job1.id, job2.id] + jobs = sdkmr.save_jobs([job1, job2]) + sdkmr.get_mongo_util().insert_jobs.assert_called_with( + jobs_to_insert=[job1, job2] + ) + assert jobs == [str(job1.id), str(job2.id)] + + def test_add_child_jobs(self): + ws = Workspace("https://fake.com") + wsa = WorkspaceAuth("user", ws) + cliset = UserClientSet("user", "token", ws, wsa) + clients_and_mocks = get_client_mocks(self.cfg, self.config_file, *ALL_CLIENTS) + sdkmr = SDKMethodRunner(cliset, clients_and_mocks[ClientSet]) + j = create_autospec(Job, spec_set=False, instance=True) + sdkmr.add_child_jobs(batch_job=j, child_jobs=["a", "b", "c"]) + j.modify.assert_called_once_with(add_to_set__child_jobs=["a", "b", "c"]) + def test_save_and_return_job(self): ws = Workspace("https://fake.com") wsa = WorkspaceAuth("user", ws) From 2822e777fe8cab188299d7fdb9f5320e1f4b1a8a Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 6 Aug 2021 01:19:15 -0500 Subject: [PATCH 29/31] pr feedback --- test/tests_for_db/ee2_MongoUtil_test.py | 5 ++++- test/tests_for_sdkmr/EE2Runjob_test.py | 4 ++-- .../ee2_SDKMethodRunner_test.py | 18 ++++++------------ 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index fbaed3146..7df79c278 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -13,7 +13,6 @@ bootstrap, get_example_job, read_config_into_dict, - assert_close_to_now, ) from tests_for_db.mongo_test_helper import MongoTestHelper @@ -66,6 +65,10 @@ def test_insert_jobs(self): jobs_to_insert = [job, job2] job_ids = self.getMongoUtil().insert_jobs(jobs_to_insert) assert len(job_ids) == len(jobs_to_insert) + retrieved_jobs = self.getMongoUtil().get_jobs(job_ids=job_ids) + + for i, retrieved_job in enumerate(retrieved_jobs): + assert jobs_to_insert[i] == retrieved_job def test_update_jobs_enmasse(self): """Check to see that created jobs get updated to queued""" diff --git a/test/tests_for_sdkmr/EE2Runjob_test.py b/test/tests_for_sdkmr/EE2Runjob_test.py index eefa2841e..3794104c9 100644 --- a/test/tests_for_sdkmr/EE2Runjob_test.py +++ b/test/tests_for_sdkmr/EE2Runjob_test.py @@ -62,8 +62,8 @@ _USER = "someuser" _TOKEN = "tokentokentoken" _OTHER_USER = "some_sucker" -_CREATED_STATE = Status.created.value -_QUEUED_STATE = Status.queued.value +_CREATED_STATE = "created" +_QUEUED_STATE = "queued" # batch common variables _BATCH = "batch" diff --git a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py index aa047cc20..ce98c0fb9 100644 --- a/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py +++ b/test/tests_for_sdkmr/ee2_SDKMethodRunner_test.py @@ -19,9 +19,13 @@ from execution_engine2.authorization.workspaceauth import WorkspaceAuth from execution_engine2.db.MongoUtil import MongoUtil +from execution_engine2.db.models.models import Job, Status, TerminatedCode from execution_engine2.exceptions import AuthError +from execution_engine2.exceptions import InvalidStatusTransitionException +from execution_engine2.sdk.SDKMethodRunner import SDKMethodRunner from execution_engine2.sdk.job_submission_parameters import JobRequirements from execution_engine2.utils.Condor import Condor +from execution_engine2.utils.CondorTuples import SubmissionInfo from execution_engine2.utils.KafkaUtils import KafkaClient from execution_engine2.utils.SlackUtils import SlackClient from execution_engine2.utils.clients import UserClientSet, ClientSet @@ -30,10 +34,6 @@ JobRequirementsResolver, RequirementsType, ) -from execution_engine2.db.models.models import Job, Status, TerminatedCode -from execution_engine2.exceptions import InvalidStatusTransitionException -from execution_engine2.sdk.SDKMethodRunner import SDKMethodRunner -from execution_engine2.utils.CondorTuples import SubmissionInfo from test.tests_for_sdkmr.ee2_SDKMethodRunner_test_utils import ee2_sdkmr_test_helper from test.utils_shared.mock_utils import get_client_mocks, ALL_CLIENTS from test.utils_shared.test_utils import ( @@ -174,13 +174,6 @@ def test_getters(self): is clients_and_mocks[JobRequirementsResolver] ) - def test_save_jobs(self): - ws = Workspace("https://fake.com") - wsa = WorkspaceAuth("user", ws) - cliset = UserClientSet("user", "token", ws, wsa) - clients_and_mocks = get_client_mocks(self.cfg, self.config_file, *ALL_CLIENTS) - sdkmr = SDKMethodRunner(cliset, clients_and_mocks[ClientSet]) - def test_save_job_and_save_jobs(self): ws = Workspace("https://fake.com") wsa = WorkspaceAuth("user", ws) @@ -218,8 +211,9 @@ def test_add_child_jobs(self): clients_and_mocks = get_client_mocks(self.cfg, self.config_file, *ALL_CLIENTS) sdkmr = SDKMethodRunner(cliset, clients_and_mocks[ClientSet]) j = create_autospec(Job, spec_set=False, instance=True) - sdkmr.add_child_jobs(batch_job=j, child_jobs=["a", "b", "c"]) + returned_job = sdkmr.add_child_jobs(batch_job=j, child_jobs=["a", "b", "c"]) j.modify.assert_called_once_with(add_to_set__child_jobs=["a", "b", "c"]) + assert returned_job == j def test_save_and_return_job(self): ws = Workspace("https://fake.com") From ac4b6af84d4637de0916fc8a72aae183e71cc0ee Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 6 Aug 2021 01:22:42 -0500 Subject: [PATCH 30/31] fix indent --- test/tests_for_db/ee2_MongoUtil_test.py | 630 ++++++++++++------------ 1 file changed, 310 insertions(+), 320 deletions(-) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 7df79c278..6009cb683 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -115,345 +115,335 @@ def test_update_jobs_enmasse(self): # Don't change their state assert all(j.status == state.value for j in [job2, job3]) + def test_update_jobs_enmasse_bad_job_pairs(self): + job = get_example_job(status=Status.created.value).save() + job2 = get_example_job(status=Status.created.value).save() + job3 = get_example_job(status=Status.created.value).save() + job_ids = [job.id, job2.id, job3.id] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, None] + job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) + + with self.assertRaisesRegex( + expected_exception=ValueError, + expected_regex=f"Provided a bad job_id_pair, missing scheduler_id for {job3.id}", + ): + self.getMongoUtil().update_jobs_to_queued(job_id_pairs) + + job_ids = [job.id, job2.id, None] + scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] + job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) + + with self.assertRaisesRegex( + expected_exception=ValueError, + expected_regex=f"Provided a bad job_id_pair, missing job_id for {job3.scheduler_id}", + ): + self.getMongoUtil().update_jobs_to_queued(job_id_pairs) + + def test_get_by_cluster(self): + """Get a job by its condor scheduler_id""" + mongo_util = self.getMongoUtil() + with mongo_util.mongo_engine_connection(): + job = get_example_job() + job_id = job.save().id + batch = mongo_util.get_job_batch_name(job.scheduler_id) + self.assertEqual(str(job_id), batch) -def test_update_jobs_enmasse_bad_job_pairs(self): - job = get_example_job(status=Status.created.value).save() - job2 = get_example_job(status=Status.created.value).save() - job3 = get_example_job(status=Status.created.value).save() - job_ids = [job.id, job2.id, job3.id] - scheduler_ids = [job.scheduler_id, job2.scheduler_id, None] - job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) - - with self.assertRaisesRegex( - expected_exception=ValueError, - expected_regex=f"Provided a bad job_id_pair, missing scheduler_id for {job3.id}", - ): - self.getMongoUtil().update_jobs_to_queued(job_id_pairs) - - job_ids = [job.id, job2.id, None] - scheduler_ids = [job.scheduler_id, job2.scheduler_id, job3.scheduler_id] - job_id_pairs = list(map(JobIdPair, job_ids, scheduler_ids)) - - with self.assertRaisesRegex( - expected_exception=ValueError, - expected_regex=f"Provided a bad job_id_pair, missing job_id for {job3.scheduler_id}", - ): - self.getMongoUtil().update_jobs_to_queued(job_id_pairs) - - -def test_get_by_cluster(self): - """Get a job by its condor scheduler_id""" - mongo_util = self.getMongoUtil() - with mongo_util.mongo_engine_connection(): - job = get_example_job() - job_id = job.save().id - batch = mongo_util.get_job_batch_name(job.scheduler_id) - self.assertEqual(str(job_id), batch) - - -def test_get_job_ok(self): - mongo_util = self.getMongoUtil() - - with mongo_util.mongo_engine_connection(): - ori_job_count = Job.objects.count() - job = get_example_job() - job_id = job.save().id - self.assertEqual(ori_job_count, Job.objects.count() - 1) - - # get job with no exclude_fields - job = mongo_util.get_job(job_id=job_id).to_mongo().to_dict() - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "job_input", - "scheduler_id", - "child_jobs", - "batch_job", - "retry_ids", - "retry_saved_toggle", - ] - self.assertCountEqual(job.keys(), expected_keys) - - # get job with exclude_fields - job = ( - mongo_util.get_job(job_id=job_id, exclude_fields=["job_input"]) - .to_mongo() - .to_dict() - ) - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - self.assertCountEqual(job.keys(), expected_keys) - - # get job with multiple exclude_fields - job = ( - mongo_util.get_job(job_id=job_id, exclude_fields=["user", "wsid"]) - .to_mongo() - .to_dict() - ) - - expected_keys = [ - "_id", - "authstrat", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - self.assertCountEqual(job.keys(), expected_keys) - - mongo_util.get_job(job_id=job_id).delete() - self.assertEqual(ori_job_count, Job.objects.count()) - - -def test_get_jobs_ok(self): - mongo_util = self.getMongoUtil() - - with mongo_util.mongo_engine_connection(): - ori_job_count = Job.objects.count() - job = get_example_job() - job_id_1 = job.save().id - job = get_example_job() - job_id_2 = job.save().id - self.assertEqual(ori_job_count, Job.objects.count() - 2) - - # get jobs with no exclude_fields - jobs = mongo_util.get_jobs(job_ids=[job_id_1, job_id_2]) - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - - for job in jobs: - self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) - - # get jobs with multiple exclude_fields - jobs = mongo_util.get_jobs( - job_ids=[job_id_1, job_id_2], exclude_fields=["user", "wsid"] - ) - - expected_keys = [ - "_id", - "authstrat", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - for job in jobs: - self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) - - mongo_util.get_job(job_id=job_id_1).delete() - mongo_util.get_job(job_id=job_id_2).delete() - self.assertEqual(ori_job_count, Job.objects.count()) - - -def test_connection_ok(self): - mongo_util = self.getMongoUtil() - - with mongo_util.mongo_engine_connection(): - ori_job_count = Job.objects.count() - j = get_example_job() - j.save() - self.assertEqual(ori_job_count, Job.objects.count() - 1) - - job = mongo_util.get_job(job_id=j.id).to_mongo().to_dict() - - expected_keys = [ - "_id", - "user", - "authstrat", - "wsid", - "status", - "updated", - "job_input", - "scheduler_id", - "batch_job", - "child_jobs", - "retry_ids", - "retry_saved_toggle", - ] - - self.assertCountEqual(job.keys(), expected_keys) - self.assertEqual(job["user"], j.user) - self.assertEqual(job["authstrat"], "kbaseworkspace") - self.assertEqual(job["wsid"], j.wsid) + def test_get_job_ok(self): + mongo_util = self.getMongoUtil() - mongo_util.get_job(job_id=j.id).delete() - self.assertEqual(ori_job_count, Job.objects.count()) + with mongo_util.mongo_engine_connection(): + ori_job_count = Job.objects.count() + job = get_example_job() + job_id = job.save().id + self.assertEqual(ori_job_count, Job.objects.count() - 1) + + # get job with no exclude_fields + job = mongo_util.get_job(job_id=job_id).to_mongo().to_dict() + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "job_input", + "scheduler_id", + "child_jobs", + "batch_job", + "retry_ids", + "retry_saved_toggle", + ] + self.assertCountEqual(job.keys(), expected_keys) + + # get job with exclude_fields + job = ( + mongo_util.get_job(job_id=job_id, exclude_fields=["job_input"]) + .to_mongo() + .to_dict() + ) + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + self.assertCountEqual(job.keys(), expected_keys) + + # get job with multiple exclude_fields + job = ( + mongo_util.get_job(job_id=job_id, exclude_fields=["user", "wsid"]) + .to_mongo() + .to_dict() + ) + + expected_keys = [ + "_id", + "authstrat", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + self.assertCountEqual(job.keys(), expected_keys) + + mongo_util.get_job(job_id=job_id).delete() + self.assertEqual(ori_job_count, Job.objects.count()) + + def test_get_jobs_ok(self): + mongo_util = self.getMongoUtil() + with mongo_util.mongo_engine_connection(): + ori_job_count = Job.objects.count() + job = get_example_job() + job_id_1 = job.save().id + job = get_example_job() + job_id_2 = job.save().id + self.assertEqual(ori_job_count, Job.objects.count() - 2) + + # get jobs with no exclude_fields + jobs = mongo_util.get_jobs(job_ids=[job_id_1, job_id_2]) + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + + for job in jobs: + self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) + + # get jobs with multiple exclude_fields + jobs = mongo_util.get_jobs( + job_ids=[job_id_1, job_id_2], exclude_fields=["user", "wsid"] + ) + + expected_keys = [ + "_id", + "authstrat", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + for job in jobs: + self.assertCountEqual(job.to_mongo().to_dict().keys(), expected_keys) + + mongo_util.get_job(job_id=job_id_1).delete() + mongo_util.get_job(job_id=job_id_2).delete() + self.assertEqual(ori_job_count, Job.objects.count()) + + def test_connection_ok(self): + mongo_util = self.getMongoUtil() -def test_insert_one_ok(self): - mongo_util = self.getMongoUtil() + with mongo_util.mongo_engine_connection(): + ori_job_count = Job.objects.count() + j = get_example_job() + j.save() + self.assertEqual(ori_job_count, Job.objects.count() - 1) + + job = mongo_util.get_job(job_id=j.id).to_mongo().to_dict() + + expected_keys = [ + "_id", + "user", + "authstrat", + "wsid", + "status", + "updated", + "job_input", + "scheduler_id", + "batch_job", + "child_jobs", + "retry_ids", + "retry_saved_toggle", + ] + + self.assertCountEqual(job.keys(), expected_keys) + self.assertEqual(job["user"], j.user) + self.assertEqual(job["authstrat"], "kbaseworkspace") + self.assertEqual(job["wsid"], j.wsid) + + mongo_util.get_job(job_id=j.id).delete() + self.assertEqual(ori_job_count, Job.objects.count()) + + def test_insert_one_ok(self): + mongo_util = self.getMongoUtil() - with mongo_util.pymongo_client( - self.config["mongo-jobs-collection"] - ) as pymongo_client: - col = pymongo_client[self.config["mongo-database"]][ + with mongo_util.pymongo_client( self.config["mongo-jobs-collection"] - ] - - ori_job_count = col.count_documents({}) - doc = {"test_key": "foo"} - job_id = mongo_util.insert_one(doc) - self.assertEqual(ori_job_count, col.count_documents({}) - 1) + ) as pymongo_client: + col = pymongo_client[self.config["mongo-database"]][ + self.config["mongo-jobs-collection"] + ] - result = list(col.find({"_id": ObjectId(job_id)}))[0] - self.assertEqual(result["test_key"], "foo") + ori_job_count = col.count_documents({}) + doc = {"test_key": "foo"} + job_id = mongo_util.insert_one(doc) + self.assertEqual(ori_job_count, col.count_documents({}) - 1) - col.delete_one({"_id": ObjectId(job_id)}) - self.assertEqual(col.count_documents({}), ori_job_count) + result = list(col.find({"_id": ObjectId(job_id)}))[0] + self.assertEqual(result["test_key"], "foo") + col.delete_one({"_id": ObjectId(job_id)}) + self.assertEqual(col.count_documents({}), ori_job_count) -def test_find_in_ok(self): - mongo_util = self.getMongoUtil() + def test_find_in_ok(self): + mongo_util = self.getMongoUtil() - with mongo_util.pymongo_client( - self.config["mongo-jobs-collection"] - ) as pymongo_client: - col = pymongo_client[self.config["mongo-database"]][ + with mongo_util.pymongo_client( self.config["mongo-jobs-collection"] - ] - - ori_job_count = col.count_documents({}) - doc = {"test_key_1": "foo", "test_key_2": "bar"} - job_id = mongo_util.insert_one(doc) - self.assertEqual(ori_job_count, col.count_documents({}) - 1) - - # test query empty field - elements = ["foobar"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 0) - - # test query "foo" - elements = ["foo"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 1) - doc = docs.next() - self.assertTrue("_id" in doc.keys()) - self.assertTrue(doc.get("_id"), job_id) - self.assertEqual(doc.get("test_key_1"), "foo") - - col.delete_one({"_id": ObjectId(job_id)}) - self.assertEqual(col.count_documents({}), ori_job_count) - - -def test_update_one_ok(self): - mongo_util = self.getMongoUtil() + ) as pymongo_client: + col = pymongo_client[self.config["mongo-database"]][ + self.config["mongo-jobs-collection"] + ] + + ori_job_count = col.count_documents({}) + doc = {"test_key_1": "foo", "test_key_2": "bar"} + job_id = mongo_util.insert_one(doc) + self.assertEqual(ori_job_count, col.count_documents({}) - 1) + + # test query empty field + elements = ["foobar"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 0) + + # test query "foo" + elements = ["foo"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 1) + doc = docs.next() + self.assertTrue("_id" in doc.keys()) + self.assertTrue(doc.get("_id"), job_id) + self.assertEqual(doc.get("test_key_1"), "foo") + + col.delete_one({"_id": ObjectId(job_id)}) + self.assertEqual(col.count_documents({}), ori_job_count) + + def test_update_one_ok(self): + mongo_util = self.getMongoUtil() - with mongo_util.pymongo_client( - self.config["mongo-jobs-collection"] - ) as pymongo_client: - col = pymongo_client[self.config["mongo-database"]][ + with mongo_util.pymongo_client( self.config["mongo-jobs-collection"] - ] - - ori_job_count = col.count_documents({}) - doc = {"test_key_1": "foo"} - job_id = mongo_util.insert_one(doc) - self.assertEqual(ori_job_count, col.count_documents({}) - 1) - - elements = ["foo"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 1) - doc = docs.next() - self.assertTrue("_id" in doc.keys()) - self.assertTrue(doc.get("_id"), job_id) - self.assertEqual(doc.get("test_key_1"), "foo") - - mongo_util.update_one({"test_key_1": "bar"}, job_id) - - elements = ["foo"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 0) - - elements = ["bar"] - docs = mongo_util.find_in(elements, "test_key_1") - self.assertEqual(docs.count(), 1) + ) as pymongo_client: + col = pymongo_client[self.config["mongo-database"]][ + self.config["mongo-jobs-collection"] + ] + + ori_job_count = col.count_documents({}) + doc = {"test_key_1": "foo"} + job_id = mongo_util.insert_one(doc) + self.assertEqual(ori_job_count, col.count_documents({}) - 1) + + elements = ["foo"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 1) + doc = docs.next() + self.assertTrue("_id" in doc.keys()) + self.assertTrue(doc.get("_id"), job_id) + self.assertEqual(doc.get("test_key_1"), "foo") + + mongo_util.update_one({"test_key_1": "bar"}, job_id) + + elements = ["foo"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 0) + + elements = ["bar"] + docs = mongo_util.find_in(elements, "test_key_1") + self.assertEqual(docs.count(), 1) + + col.delete_one({"_id": ObjectId(job_id)}) + self.assertEqual(col.count_documents({}), ori_job_count) + + def test_delete_one_ok(self): + mongo_util = MongoUtil(self.config) + with mongo_util.pymongo_client(self.config["mongo-jobs-collection"]) as pc: + col = pc.get_database(self.config["mongo-database"]).get_collection( + self.config["mongo-jobs-collection"] + ) + + doc_count = col.count_documents({}) + logging.info("Found {} documents".format(doc_count)) + + doc = {"test_key_1": "foo", "test_key_2": "bar"} + job_id = mongo_util.insert_one(doc) + + self.assertEqual(col.count_documents({}), doc_count + 1) + logging.info("Assert 0 documents") + mongo_util.delete_one(job_id) + self.assertEqual(col.count_documents({}), doc_count) + + def test_get_job_log_pymongo_ok(self): + mongo_util = self.getMongoUtil() - col.delete_one({"_id": ObjectId(job_id)}) - self.assertEqual(col.count_documents({}), ori_job_count) + primary_key = ObjectId() + jl = JobLog() + jl.primary_key = primary_key + jl.original_line_count = 0 + jl.stored_line_count = 0 + jl.lines = [] -def test_delete_one_ok(self): - mongo_util = MongoUtil(self.config) - with mongo_util.pymongo_client(self.config["mongo-jobs-collection"]) as pc: - col = pc.get_database(self.config["mongo-database"]).get_collection( + with mongo_util.pymongo_client( self.config["mongo-jobs-collection"] - ) - - doc_count = col.count_documents({}) - logging.info("Found {} documents".format(doc_count)) - - doc = {"test_key_1": "foo", "test_key_2": "bar"} - job_id = mongo_util.insert_one(doc) - - self.assertEqual(col.count_documents({}), doc_count + 1) - logging.info("Assert 0 documents") - mongo_util.delete_one(job_id) - self.assertEqual(col.count_documents({}), doc_count) - - -def test_get_job_log_pymongo_ok(self): - mongo_util = self.getMongoUtil() - - primary_key = ObjectId() - - jl = JobLog() - jl.primary_key = primary_key - jl.original_line_count = 0 - jl.stored_line_count = 0 - jl.lines = [] - - with mongo_util.pymongo_client( - self.config["mongo-jobs-collection"] - ) as pymongo_client: - jl_col = pymongo_client[self.config["mongo-database"]][ - self.config["mongo-logs-collection"] - ] + ) as pymongo_client: + jl_col = pymongo_client[self.config["mongo-database"]][ + self.config["mongo-logs-collection"] + ] - ori_jl_count = jl_col.count_documents({}) + ori_jl_count = jl_col.count_documents({}) - jl.save() # save job log + jl.save() # save job log - self.assertEqual(JobLog.objects.count(), ori_jl_count + 1) - job_log = mongo_util.get_job_log_pymongo(str(primary_key)) + self.assertEqual(JobLog.objects.count(), ori_jl_count + 1) + job_log = mongo_util.get_job_log_pymongo(str(primary_key)) - self.assertEqual(job_log.get("original_line_count"), 0) - self.assertEqual(job_log.get("stored_line_count"), 0) - self.assertIsNone(job_log.get("lines")) + self.assertEqual(job_log.get("original_line_count"), 0) + self.assertEqual(job_log.get("stored_line_count"), 0) + self.assertIsNone(job_log.get("lines")) From f244544bf2516a1a61fca835cc8a48fa31d0574c Mon Sep 17 00:00:00 2001 From: bio-boris Date: Fri, 6 Aug 2021 15:45:24 -0500 Subject: [PATCH 31/31] Check job contents --- test/tests_for_db/ee2_MongoUtil_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/tests_for_db/ee2_MongoUtil_test.py b/test/tests_for_db/ee2_MongoUtil_test.py index 6009cb683..9591f16a1 100644 --- a/test/tests_for_db/ee2_MongoUtil_test.py +++ b/test/tests_for_db/ee2_MongoUtil_test.py @@ -68,7 +68,7 @@ def test_insert_jobs(self): retrieved_jobs = self.getMongoUtil().get_jobs(job_ids=job_ids) for i, retrieved_job in enumerate(retrieved_jobs): - assert jobs_to_insert[i] == retrieved_job + assert jobs_to_insert[i].to_json() == retrieved_job.to_json() def test_update_jobs_enmasse(self): """Check to see that created jobs get updated to queued"""