From 97bcbd2d7fd9285713f1741ecd33b168070cb406 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Wed, 30 Oct 2024 15:43:31 -0400 Subject: [PATCH 01/39] Make it work with a lot of duplications --- .../clp_package_utils/general.py | 1 + .../clp_package_utils/scripts/decompress.py | 64 +++++++ .../scripts/native/decompress.py | 74 +++++++- .../executor/query/celeryconfig.py | 2 + .../executor/query/extract_json_task.py | 119 +++++++++++++ .../job_orchestration/scheduler/constants.py | 1 + .../job_orchestration/scheduler/job_config.py | 5 + .../scheduler/query/query_scheduler.py | 167 +++++++++++++++++- .../scheduler/scheduler_data.py | 12 ++ 9 files changed, 442 insertions(+), 3 deletions(-) create mode 100644 components/job-orchestration/job_orchestration/executor/query/extract_json_task.py diff --git a/components/clp-package-utils/clp_package_utils/general.py b/components/clp-package-utils/clp_package_utils/general.py index 4dca481b0..620cc717b 100644 --- a/components/clp-package-utils/clp_package_utils/general.py +++ b/components/clp-package-utils/clp_package_utils/general.py @@ -33,6 +33,7 @@ # CONSTANTS EXTRACT_FILE_CMD = "x" EXTRACT_IR_CMD = "i" +EXTRACT_JSON_CMD = "j" # Paths CONTAINER_CLP_HOME = pathlib.Path("/") / "opt" / "clp" diff --git a/components/clp-package-utils/clp_package_utils/scripts/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/decompress.py index 1a2973fec..e306dba23 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/decompress.py @@ -14,6 +14,7 @@ dump_container_config, EXTRACT_FILE_CMD, EXTRACT_IR_CMD, + EXTRACT_JSON_CMD, generate_container_config, generate_container_name, generate_container_start_cmd, @@ -205,6 +206,60 @@ def handle_extract_ir_cmd( return 0 +def handle_extract_json_cmd( + parsed_args, clp_home: pathlib.Path, default_config_file_path: pathlib.Path +) -> int: + """ + Handles the Json extraction command. + :param parsed_args: + :param clp_home: + :param default_config_file_path: + :return: 0 on success, -1 otherwise. + """ + # Validate and load config file + clp_config = validate_and_load_config( + clp_home, pathlib.Path(parsed_args.config), default_config_file_path + ) + if clp_config is None: + return -1 + + container_name = generate_container_name(JobType.IR_EXTRACTION) + container_clp_config, mounts = generate_container_config(clp_config, clp_home) + generated_config_path_on_container, generated_config_path_on_host = dump_container_config( + container_clp_config, clp_config, container_name + ) + necessary_mounts = [mounts.clp_home, mounts.logs_dir] + container_start_cmd = generate_container_start_cmd( + container_name, necessary_mounts, clp_config.execution_container + ) + + # fmt: off + extract_cmd = [ + "python3", + "-m", "clp_package_utils.scripts.native.decompress", + "--config", str(generated_config_path_on_container), + EXTRACT_JSON_CMD, + parsed_args.archive_id + ] + # fmt: on + + if parsed_args.target_chunk_size: + extract_cmd.append("--target-chunk-size") + extract_cmd.append(str(parsed_args.target_chunk_size)) + cmd = container_start_cmd + extract_cmd + + try: + subprocess.run(cmd, check=True) + except subprocess.CalledProcessError: + logger.exception("Docker or Json extraction command failed.") + return -1 + + # Remove generated files + generated_config_path_on_host.unlink() + + return 0 + + def main(argv): clp_home = get_clp_home() default_config_file_path = clp_home / CLP_DEFAULT_CONFIG_FILE_RELATIVE_PATH @@ -241,6 +296,13 @@ def main(argv): group.add_argument("--orig-file-id", type=str, help="Original file's ID.") group.add_argument("--orig-file-path", type=str, help="Original file's path.") + # IR extraction command parser + json_extraction_parser = command_args_parser.add_parser(EXTRACT_JSON_CMD) + json_extraction_parser.add_argument("archive_id", type=str, help="Archive ID") + json_extraction_parser.add_argument( + "--target-chunk-size", type=int, help="Target chunk size", default=100000 + ) + parsed_args = args_parser.parse_args(argv[1:]) command = parsed_args.command @@ -248,6 +310,8 @@ def main(argv): return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) elif EXTRACT_IR_CMD == command: return handle_extract_ir_cmd(parsed_args, clp_home, default_config_file_path) + elif EXTRACT_JSON_CMD == command: + return handle_extract_json_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") return -1 diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index b6585b192..c7c70429a 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -12,12 +12,13 @@ from clp_py_utils.clp_config import CLP_METADATA_TABLE_PREFIX, CLPConfig, Database from clp_py_utils.sql_adapter import SQL_Adapter from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType -from job_orchestration.scheduler.job_config import ExtractIrJobConfig +from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig from clp_package_utils.general import ( CLP_DEFAULT_CONFIG_FILE_RELATIVE_PATH, EXTRACT_FILE_CMD, EXTRACT_IR_CMD, + EXTRACT_JSON_CMD, get_clp_home, load_config_file, ) @@ -104,6 +105,37 @@ def submit_and_monitor_ir_extraction_job_in_db( return -1 +def submit_and_monitor_json_extraction_job_in_db( + db_config: Database, + archive_id: str, + target_chunk_size: Optional[int], +) -> int: + """ + Submits an IR extraction job to the scheduler and waits until the job finishes. + :param db_config: + :param archive_id: + :param target_chunk_size: + :return: 0 on success, -1 otherwise. + """ + extract_json_config = ExtractJsonJobConfig( + archive_id=archive_id, + target_chunk_size=target_chunk_size, + ) + + sql_adapter = SQL_Adapter(db_config) + job_id = submit_query_job(sql_adapter, extract_json_config, QueryJobType.EXTRACT_JSON) + job_status = wait_for_query_job(sql_adapter, job_id) + + if QueryJobStatus.SUCCEEDED == job_status: + logger.info(f"Finished Json extraction job {job_id}.") + return 0 + + logger.error( + f"Json extraction job {job_id} finished with unexpected status: {job_status.to_str()}." + ) + return -1 + + def handle_extract_ir_cmd( parsed_args: argparse.Namespace, clp_home: pathlib.Path, default_config_file_path: pathlib.Path ) -> int: @@ -144,6 +176,37 @@ def handle_extract_ir_cmd( return -1 +def handle_extract_json_cmd( + parsed_args: argparse.Namespace, clp_home: pathlib.Path, default_config_file_path: pathlib.Path +) -> int: + """ + Handles the IR extraction command. + :param parsed_args: + :param clp_home: + :param default_config_file_path: + :return: 0 on success, -1 otherwise. + """ + # Validate and load config file + clp_config = validate_and_load_config_file( + clp_home, pathlib.Path(parsed_args.config), default_config_file_path + ) + if clp_config is None: + return -1 + + try: + return asyncio.run( + run_function_in_process( + submit_and_monitor_json_extraction_job_in_db, + clp_config.database, + parsed_args.archive_id, + parsed_args.target_chunk_size, + ) + ) + except asyncio.CancelledError: + logger.error("Json extraction cancelled.") + return -1 + + def validate_and_load_config_file( clp_home: pathlib.Path, config_file_path: pathlib.Path, @@ -278,6 +341,13 @@ def main(argv): group.add_argument("--orig-file-id", type=str, help="Original file's ID.") group.add_argument("--orig-file-path", type=str, help="Original file's path.") + # Json extraction command parser + json_extraction_parser = command_args_parser.add_parser(EXTRACT_JSON_CMD) + json_extraction_parser.add_argument("archive_id", type=str, help="Archive ID") + json_extraction_parser.add_argument( + "--target-chunk-size", type=int, help="Target chunk size.", required=True + ) + parsed_args = args_parser.parse_args(argv[1:]) command = parsed_args.command @@ -285,6 +355,8 @@ def main(argv): return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) elif EXTRACT_IR_CMD == command: return handle_extract_ir_cmd(parsed_args, clp_home, default_config_file_path) + elif EXTRACT_JSON_CMD == command: + return handle_extract_json_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") return -1 diff --git a/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py b/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py index 994c0bbcf..bfe6f6efc 100644 --- a/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py +++ b/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py @@ -5,11 +5,13 @@ imports = ( "job_orchestration.executor.query.fs_search_task", "job_orchestration.executor.query.extract_ir_task", + "job_orchestration.executor.query.extract_json_task", ) task_routes = { "job_orchestration.executor.query.fs_search_task.search": QueueName.QUERY, "job_orchestration.executor.query.extract_ir_task.extract_ir": QueueName.QUERY, + "job_orchestration.executor.query.extract_json_task.extract_json": QueueName.QUERY, } task_create_missing_queues = True diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_json_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_json_task.py new file mode 100644 index 000000000..ff2a99bfa --- /dev/null +++ b/components/job-orchestration/job_orchestration/executor/query/extract_json_task.py @@ -0,0 +1,119 @@ +import datetime +import os +from pathlib import Path +from typing import Any, Dict, List, Optional + +from celery.app.task import Task +from celery.utils.log import get_task_logger +from clp_py_utils.clp_config import Database, StorageEngine +from clp_py_utils.clp_logging import set_logging_level +from clp_py_utils.sql_adapter import SQL_Adapter +from job_orchestration.executor.query.celery import app +from job_orchestration.executor.query.utils import ( + report_command_creation_failure, + run_query_task, +) +from job_orchestration.scheduler.job_config import ExtractJsonJobConfig +from job_orchestration.scheduler.scheduler_data import QueryTaskStatus + +# Setup logging +logger = get_task_logger(__name__) + + +def make_command( + storage_engine: str, + clp_home: Path, + archives_dir: Path, + archive_id: str, + output_dir: Path, + extract_json_config: ExtractJsonJobConfig, + results_cache_uri: str, + ir_collection: str, +) -> Optional[List[str]]: + if StorageEngine.CLP_S == storage_engine: + if not extract_json_config.archive_id: + logger.error("archive_id not supplied") + return None + command = [ + str(clp_home / "bin" / "clp-s"), + "x", + str(archives_dir), + str(output_dir), + "--ordered", + "--archive-id", + archive_id, + "--mongodb-uri", + results_cache_uri, + "--mongodb-collection", + ir_collection, + ] + if extract_json_config.target_chunk_size is not None: + command.append("--ordered-chunk-size") + command.append(str(extract_json_config.target_chunk_size)) + else: + logger.error(f"Unsupported storage engine {storage_engine}") + return None + + return command + + +@app.task(bind=True) +def extract_json( + self: Task, + job_id: str, + task_id: int, + job_config_obj: dict, + archive_id: str, + clp_metadata_db_conn_params: dict, + results_cache_uri: str, +) -> Dict[str, Any]: + task_name = "Json Extraction" + + # Setup logging to file + clp_logs_dir = Path(os.getenv("CLP_LOGS_DIR")) + clp_logging_level = os.getenv("CLP_LOGGING_LEVEL") + set_logging_level(logger, clp_logging_level) + + logger.info(f"Started {task_name} task for job {job_id}") + + start_time = datetime.datetime.now() + task_status: QueryTaskStatus + sql_adapter = SQL_Adapter(Database.parse_obj(clp_metadata_db_conn_params)) + + # Make task_command + clp_home = Path(os.getenv("CLP_HOME")) + archive_directory = Path(os.getenv("CLP_ARCHIVE_OUTPUT_DIR")) + clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") + ir_output_dir = Path(os.getenv("CLP_IR_OUTPUT_DIR")) + ir_collection = os.getenv("CLP_IR_COLLECTION") + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config_obj) + + task_command = make_command( + storage_engine=clp_storage_engine, + clp_home=clp_home, + archives_dir=archive_directory, + archive_id=archive_id, + output_dir=ir_output_dir, + extract_json_config=extract_json_config, + results_cache_uri=results_cache_uri, + ir_collection=ir_collection, + ) + if not task_command: + return report_command_creation_failure( + sql_adapter=sql_adapter, + logger=logger, + task_name=task_name, + task_id=task_id, + start_time=start_time, + ) + + return run_query_task( + sql_adapter=sql_adapter, + logger=logger, + clp_logs_dir=clp_logs_dir, + task_command=task_command, + task_name=task_name, + job_id=job_id, + task_id=task_id, + start_time=start_time, + ) diff --git a/components/job-orchestration/job_orchestration/scheduler/constants.py b/components/job-orchestration/job_orchestration/scheduler/constants.py index 131719148..cd85016a3 100644 --- a/components/job-orchestration/job_orchestration/scheduler/constants.py +++ b/components/job-orchestration/job_orchestration/scheduler/constants.py @@ -72,6 +72,7 @@ def to_str(self) -> str: class QueryJobType(IntEnum): SEARCH_OR_AGGREGATION = 0 EXTRACT_IR = auto() + EXTRACT_JSON = auto() def __str__(self) -> str: return str(self.value) diff --git a/components/job-orchestration/job_orchestration/scheduler/job_config.py b/components/job-orchestration/job_orchestration/scheduler/job_config.py index e90e2ee7f..7cf8b2324 100644 --- a/components/job-orchestration/job_orchestration/scheduler/job_config.py +++ b/components/job-orchestration/job_orchestration/scheduler/job_config.py @@ -49,6 +49,11 @@ class ExtractIrJobConfig(QueryJobConfig): target_uncompressed_size: typing.Optional[int] = None +class ExtractJsonJobConfig(QueryJobConfig): + archive_id: str + target_chunk_size: typing.Optional[int] = None + + class SearchJobConfig(QueryJobConfig): query_string: str max_num_results: int diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 2a0f855a3..aa23940d4 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -40,9 +40,10 @@ from clp_py_utils.decorators import exception_default_value from clp_py_utils.sql_adapter import SQL_Adapter from job_orchestration.executor.query.extract_ir_task import extract_ir +from job_orchestration.executor.query.extract_json_task import extract_json from job_orchestration.executor.query.fs_search_task import search from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType, QueryTaskStatus -from job_orchestration.scheduler.job_config import ExtractIrJobConfig, SearchJobConfig +from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig, SearchJobConfig from job_orchestration.scheduler.query.reducer_handler import ( handle_reducer_connection, ReducerHandlerMessage, @@ -51,6 +52,7 @@ ) from job_orchestration.scheduler.scheduler_data import ( ExtractIrJob, + ExtractJsonJob, InternalJobState, QueryJob, QueryTaskResult, @@ -66,7 +68,8 @@ # Dictionary that maps IDs of file splits being extracted to IDs of jobs waiting for them active_file_split_ir_extractions: Dict[str, List[str]] = {} - +# Dictionary that maps IDs of clp-s archives being extracted to IDs of jobs waiting for them +active_archive_json_extractions: Dict[str, List[str]] = {} reducer_connection_queue: Optional[asyncio.Queue] = None @@ -366,6 +369,18 @@ def get_task_group_for_job( ) for i in range(len(archive_ids)) ) + elif QueryJobType.EXTRACT_JSON == job_type: + return celery.group( + extract_json.s( + job_id=job.id, + archive_id=archive_ids[i], + task_id=task_ids[i], + job_config_obj=job_config_obj, + clp_metadata_db_conn_params=clp_metadata_db_conn_params, + results_cache_uri=results_cache_uri, + ) + for i in range(len(archive_ids)) + ) else: error_msg = f"Unexpected job type: {job_type}" logger.error(error_msg) @@ -471,6 +486,7 @@ def handle_pending_query_jobs( ) -> List[asyncio.Task]: global active_jobs global active_file_split_ir_extractions + global active_archive_json_extractions reducer_acquisition_tasks = [] pending_search_jobs = [ @@ -614,6 +630,78 @@ def handle_pending_query_jobs( active_jobs[new_extract_ir_job.id] = new_extract_ir_job logger.info(f"Dispatched IR extraction job {job_id} on archive: {archive_id}") + elif QueryJobType.EXTRACT_JSON == job_type: + extract_json_config = ExtractJsonJobConfig.parse_obj(msgpack.unpackb(job_config)) + + # TODO: For now, let's assume the archive_id is always valid. Will need to verify this somehow + archive_id = extract_json_config.archive_id + + # NOTE: The following two if blocks should not be reordered since if we first check + # whether *an* IR file has been extracted for the requested file split, it doesn't + # mean that *all* IR files have has been extracted for the file split (since the + # extraction job may still be in progress). Thus, we must first check whether the + # file split is in the process of being extracted, and then check whether it's + # already been extracted. + + # Check if the archive is currently being extracted; if so, add the job ID to the + # list of jobs waiting for it. + if archive_id in active_archive_json_extractions: + active_archive_json_extractions[archive_id].append(job_id) + logger.info( + f"archive {archive_id} is being extracted, so mark job {job_id} as running" + ) + if not set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.RUNNING, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + ): + logger.error(f"Failed to set job {job_id} as running") + continue + + # Check if the archive with the expected timestamp has already been extracted + if json_file_exists_for_archive( + results_cache_uri, ir_collection_name, archive_id + ): + logger.info( + f"archive {archive_id} already extracted, so mark job {job_id} as done" + ) + if not set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.SUCCEEDED, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + duration=0, + ): + logger.error(f"Failed to set job {job_id} as succeeded") + continue + + active_archive_json_extractions[archive_id] = [job_id] + new_extract_json_job = ExtractJsonJob( + id=job_id, + archive_id=archive_id, + extract_json_config=extract_json_config, + state=InternalJobState.WAITING_FOR_DISPATCH, + ) + target_archive = [new_extract_json_job.archive_id] + + dispatch_job_and_update_db( + db_conn, + new_extract_json_job, + target_archive, + clp_metadata_db_conn_params, + results_cache_uri, + 1, + ) + active_jobs[new_extract_json_job.id] = new_extract_json_job + logger.info(f"Dispatched Json extraction job {job_id} on archive: {archive_id}") + else: # NOTE: We're skipping the job for this iteration, but its status will remain # unchanged. So this log will print again in the next iteration unless the user @@ -694,6 +782,15 @@ def ir_file_exists_for_file_split( return 0 != results_count +def json_file_exists_for_archive( + results_cache_uri: str, ir_collection_name: str, archive_id: str +): + with pymongo.MongoClient(results_cache_uri) as results_cache_client: + ir_collection = results_cache_client.get_default_database()[ir_collection_name] + results_count = ir_collection.count_documents({"orig_file_id": archive_id}) + return 0 != results_count + + async def handle_finished_search_job( db_conn, job: SearchJob, task_results: Optional[Any], results_cache_uri: str ) -> None: @@ -843,6 +940,69 @@ async def handle_finished_extract_ir_job( del active_jobs[job_id] +async def handle_finished_extract_json_job( + db_conn, job: ExtractJsonJob, task_results: Optional[Any] +) -> None: + global active_jobs + global active_archive_json_extractions + + job_id = job.id + archive_id = job.archive_id + new_job_status = QueryJobStatus.SUCCEEDED + num_tasks = len(task_results) + if 1 != num_tasks: + logger.error( + f"Unexpected number of tasks for Json extraction job {job_id}. " + f"Expected 1, got {num_tasks}." + ) + new_job_status = QueryJobStatus.FAILED + else: + task_result = QueryTaskResult.parse_obj(task_results[0]) + task_id = task_result.task_id + if not QueryJobStatus.SUCCEEDED == task_result.status: + logger.error( + f"Json extraction task job-{job_id}-task-{task_id} failed. " + f"Check {task_result.error_log_path} for details." + ) + new_job_status = QueryJobStatus.FAILED + else: + logger.info( + f"Json extraction task job-{job_id}-task-{task_id} succeeded in " + f"{task_result.duration} second(s)." + ) + + if set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + new_job_status, + QueryJobStatus.RUNNING, + num_tasks_completed=num_tasks, + duration=(datetime.datetime.now() - job.start_time).total_seconds(), + ): + if new_job_status == QueryJobStatus.SUCCEEDED: + logger.info(f"Completed IR extraction job {job_id}.") + else: + logger.info(f"Completed IR extraction job {job_id} with failing tasks.") + + waiting_jobs = active_archive_json_extractions[archive_id] + waiting_jobs.remove(job_id) + for waiting_job in waiting_jobs: + logger.info(f"Setting status to {new_job_status.to_str()} for waiting jobs: {waiting_job}.") + set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + waiting_job, + new_job_status, + QueryJobStatus.RUNNING, + num_tasks_completed=0, + duration=(datetime.datetime.now() - job.start_time).total_seconds(), + ) + + del active_archive_json_extractions[archive_id] + del active_jobs[job_id] + + async def check_job_status_and_update_db(db_conn_pool, results_cache_uri): global active_jobs @@ -883,6 +1043,9 @@ async def check_job_status_and_update_db(db_conn_pool, results_cache_uri): elif QueryJobType.EXTRACT_IR == job_type: extract_ir_job: ExtractIrJob = job await handle_finished_extract_ir_job(db_conn, extract_ir_job, returned_results) + elif QueryJobType.EXTRACT_JSON == job_type: + extract_json_job: ExtractJsonJob = job + await handle_finished_extract_json_job(db_conn, extract_json_job, returned_results) else: logger.error(f"Unexpected job type: {job_type}, skipping job {job_id}") diff --git a/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py b/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py index 5ef92a5d6..fe26bef97 100644 --- a/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py +++ b/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py @@ -11,6 +11,7 @@ ) from job_orchestration.scheduler.job_config import ( ExtractIrJobConfig, + ExtractJsonJobConfig, QueryJobConfig, SearchJobConfig, ) @@ -69,6 +70,17 @@ def get_config(self) -> QueryJobConfig: return self.extract_ir_config +class ExtractJsonJob(QueryJob): + extract_json_config: ExtractJsonJobConfig + archive_id: str + + def get_type(self) -> QueryJobType: + return QueryJobType.EXTRACT_JSON + + def get_config(self) -> QueryJobConfig: + return self.extract_json_config + + class SearchJob(QueryJob): search_config: SearchJobConfig num_archives_to_search: int From db1bf0a67df4f5c45e8ff4477aff6633a4232e0b Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Wed, 30 Oct 2024 16:28:01 -0400 Subject: [PATCH 02/39] deduplication the code in task launching scripts --- .../clp_package_utils/scripts/decompress.py | 95 ++++-------- .../scripts/native/decompress.py | 137 ++++++------------ 2 files changed, 67 insertions(+), 165 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/decompress.py index e306dba23..c88de19df 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/decompress.py @@ -147,11 +147,11 @@ def handle_extract_file_cmd( return 0 -def handle_extract_ir_cmd( +def handle_extract_cmd( parsed_args, clp_home: pathlib.Path, default_config_file_path: pathlib.Path ) -> int: """ - Handles the IR extraction command. + Handles the extraction command. :param parsed_args: :param clp_home: :param default_config_file_path: @@ -175,83 +175,40 @@ def handle_extract_ir_cmd( ) # fmt: off + job_command = parsed_args.command extract_cmd = [ "python3", "-m", "clp_package_utils.scripts.native.decompress", "--config", str(generated_config_path_on_container), - EXTRACT_IR_CMD, - str(parsed_args.msg_ix), + job_command ] # fmt: on - if parsed_args.orig_file_id: - extract_cmd.append("--orig-file-id") - extract_cmd.append(str(parsed_args.orig_file_id)) - else: - extract_cmd.append("--orig-file-path") - extract_cmd.append(str(parsed_args.orig_file_path)) - if parsed_args.target_uncompressed_size: - extract_cmd.append("--target-uncompressed-size") - extract_cmd.append(str(parsed_args.target_uncompressed_size)) - cmd = container_start_cmd + extract_cmd - - try: - subprocess.run(cmd, check=True) - except subprocess.CalledProcessError: - logger.exception("Docker or IR extraction command failed.") - return -1 - - # Remove generated files - generated_config_path_on_host.unlink() - - return 0 - - -def handle_extract_json_cmd( - parsed_args, clp_home: pathlib.Path, default_config_file_path: pathlib.Path -) -> int: - """ - Handles the Json extraction command. - :param parsed_args: - :param clp_home: - :param default_config_file_path: - :return: 0 on success, -1 otherwise. - """ - # Validate and load config file - clp_config = validate_and_load_config( - clp_home, pathlib.Path(parsed_args.config), default_config_file_path - ) - if clp_config is None: - return -1 - - container_name = generate_container_name(JobType.IR_EXTRACTION) - container_clp_config, mounts = generate_container_config(clp_config, clp_home) - generated_config_path_on_container, generated_config_path_on_host = dump_container_config( - container_clp_config, clp_config, container_name - ) - necessary_mounts = [mounts.clp_home, mounts.logs_dir] - container_start_cmd = generate_container_start_cmd( - container_name, necessary_mounts, clp_config.execution_container - ) - # fmt: off - extract_cmd = [ - "python3", - "-m", "clp_package_utils.scripts.native.decompress", - "--config", str(generated_config_path_on_container), - EXTRACT_JSON_CMD, - parsed_args.archive_id - ] - # fmt: on + if EXTRACT_IR_CMD == job_command: + extract_cmd.append(str(parsed_args.msg_ix)) + if parsed_args.orig_file_id: + extract_cmd.append("--orig-file-id") + extract_cmd.append(str(parsed_args.orig_file_id)) + else: + extract_cmd.append("--orig-file-path") + extract_cmd.append(str(parsed_args.orig_file_path)) + if parsed_args.target_uncompressed_size: + extract_cmd.append("--target-uncompressed-size") + extract_cmd.append(str(parsed_args.target_uncompressed_size)) + elif EXTRACT_JSON_CMD == job_command: + extract_cmd.append(str(parsed_args.archive_id)) + if parsed_args.target_chunk_size: + extract_cmd.append("--target-chunk-size") + extract_cmd.append(str(parsed_args.target_chunk_size)) + else: + logger.exception(f"Unexpected command: {job_command}") - if parsed_args.target_chunk_size: - extract_cmd.append("--target-chunk-size") - extract_cmd.append(str(parsed_args.target_chunk_size)) cmd = container_start_cmd + extract_cmd try: subprocess.run(cmd, check=True) except subprocess.CalledProcessError: - logger.exception("Docker or Json extraction command failed.") + logger.exception("Docker or IR extraction command failed.") return -1 # Remove generated files @@ -308,10 +265,8 @@ def main(argv): command = parsed_args.command if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) - elif EXTRACT_IR_CMD == command: - return handle_extract_ir_cmd(parsed_args, clp_home, default_config_file_path) - elif EXTRACT_JSON_CMD == command: - return handle_extract_json_cmd(parsed_args, clp_home, default_config_file_path) + elif command in [EXTRACT_IR_CMD, EXTRACT_JSON_CMD]: + return handle_extract_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") return -1 diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index c7c70429a..14b3276ca 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -12,7 +12,7 @@ from clp_py_utils.clp_config import CLP_METADATA_TABLE_PREFIX, CLPConfig, Database from clp_py_utils.sql_adapter import SQL_Adapter from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType -from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig +from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig, QueryJobConfig from clp_package_utils.general import ( CLP_DEFAULT_CONFIG_FILE_RELATIVE_PATH, @@ -71,73 +71,37 @@ def get_orig_file_id(db_config: Database, path: str) -> Optional[str]: return results[0]["orig_file_id"] -def submit_and_monitor_ir_extraction_job_in_db( +def submit_and_monitor_extraction_job_in_db( db_config: Database, - orig_file_id: str, - msg_ix: int, - target_uncompressed_size: Optional[int], + job_type: QueryJobType, + job_config: QueryJobConfig, ) -> int: """ - Submits an IR extraction job to the scheduler and waits until the job finishes. + Submits an extraction job to the scheduler and waits until the job finishes. :param db_config: - :param orig_file_id: - :param msg_ix: - :param target_uncompressed_size: + :param job_type: + :param job_config: :return: 0 on success, -1 otherwise. """ - extract_ir_config = ExtractIrJobConfig( - orig_file_id=orig_file_id, - msg_ix=msg_ix, - target_uncompressed_size=target_uncompressed_size, - ) - - sql_adapter = SQL_Adapter(db_config) - job_id = submit_query_job(sql_adapter, extract_ir_config, QueryJobType.EXTRACT_IR) - job_status = wait_for_query_job(sql_adapter, job_id) - - if QueryJobStatus.SUCCEEDED == job_status: - logger.info(f"Finished IR extraction job {job_id}.") - return 0 - - logger.error( - f"IR extraction job {job_id} finished with unexpected status: {job_status.to_str()}." - ) - return -1 - - -def submit_and_monitor_json_extraction_job_in_db( - db_config: Database, - archive_id: str, - target_chunk_size: Optional[int], -) -> int: - """ - Submits an IR extraction job to the scheduler and waits until the job finishes. - :param db_config: - :param archive_id: - :param target_chunk_size: - :return: 0 on success, -1 otherwise. - """ - extract_json_config = ExtractJsonJobConfig( - archive_id=archive_id, - target_chunk_size=target_chunk_size, - ) - sql_adapter = SQL_Adapter(db_config) - job_id = submit_query_job(sql_adapter, extract_json_config, QueryJobType.EXTRACT_JSON) + job_id = submit_query_job(sql_adapter, job_config, job_type) job_status = wait_for_query_job(sql_adapter, job_id) if QueryJobStatus.SUCCEEDED == job_status: - logger.info(f"Finished Json extraction job {job_id}.") + logger.info(f"Finished extraction job {job_id}.") return 0 logger.error( - f"Json extraction job {job_id} finished with unexpected status: {job_status.to_str()}." + f"Extraction job {job_id} finished with unexpected status: {job_status.to_str()}." ) return -1 -def handle_extract_ir_cmd( - parsed_args: argparse.Namespace, clp_home: pathlib.Path, default_config_file_path: pathlib.Path +def handle_extract_cmd( + parsed_args: argparse.Namespace, + job_type: QueryJobType, + clp_home: pathlib.Path, + default_config_file_path: pathlib.Path ) -> int: """ Handles the IR extraction command. @@ -153,22 +117,36 @@ def handle_extract_ir_cmd( if clp_config is None: return -1 - orig_file_id: str - if parsed_args.orig_file_id: - orig_file_id = parsed_args.orig_file_id + extraction_config: QueryJobConfig + if QueryJobType.EXTRACT_IR == job_type: + orig_file_id: str + if parsed_args.orig_file_id: + orig_file_id = parsed_args.orig_file_id + else: + orig_file_path = parsed_args.orig_file_path + orig_file_id = get_orig_file_id(clp_config.database, parsed_args.orig_file_path) + if orig_file_id is None: + logger.error(f"Cannot find orig_file_id corresponding to {orig_file_path}") + return -1 + extraction_config = ExtractIrJobConfig( + orig_file_id=orig_file_id, + msg_ix=parsed_args.msg_ix, + target_uncompressed_size=parsed_args.target_uncompressed_size + ) + elif QueryJobType.EXTRACT_JSON == job_type: + extraction_config = ExtractJsonJobConfig( + archive_id=parsed_args.archive_id, + target_chunk_size=parsed_args.target_chunk_size + ) else: - orig_file_id = get_orig_file_id(clp_config.database, parsed_args.orig_file_path) - if orig_file_id is None: - return -1 - + logger.exception(f"Unsupported extraction job type: {job_type}") try: return asyncio.run( run_function_in_process( - submit_and_monitor_ir_extraction_job_in_db, + submit_and_monitor_extraction_job_in_db, clp_config.database, - orig_file_id, - parsed_args.msg_ix, - parsed_args.target_uncompressed_size, + job_type, + extraction_config, ) ) except asyncio.CancelledError: @@ -176,37 +154,6 @@ def handle_extract_ir_cmd( return -1 -def handle_extract_json_cmd( - parsed_args: argparse.Namespace, clp_home: pathlib.Path, default_config_file_path: pathlib.Path -) -> int: - """ - Handles the IR extraction command. - :param parsed_args: - :param clp_home: - :param default_config_file_path: - :return: 0 on success, -1 otherwise. - """ - # Validate and load config file - clp_config = validate_and_load_config_file( - clp_home, pathlib.Path(parsed_args.config), default_config_file_path - ) - if clp_config is None: - return -1 - - try: - return asyncio.run( - run_function_in_process( - submit_and_monitor_json_extraction_job_in_db, - clp_config.database, - parsed_args.archive_id, - parsed_args.target_chunk_size, - ) - ) - except asyncio.CancelledError: - logger.error("Json extraction cancelled.") - return -1 - - def validate_and_load_config_file( clp_home: pathlib.Path, config_file_path: pathlib.Path, @@ -354,9 +301,9 @@ def main(argv): if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) elif EXTRACT_IR_CMD == command: - return handle_extract_ir_cmd(parsed_args, clp_home, default_config_file_path) + return handle_extract_cmd(parsed_args, QueryJobType.EXTRACT_IR, clp_home, default_config_file_path) elif EXTRACT_JSON_CMD == command: - return handle_extract_json_cmd(parsed_args, clp_home, default_config_file_path) + return handle_extract_cmd(parsed_args, QueryJobType.EXTRACT_JSON, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") return -1 From aea5630d99e4a5ca0475b636e7a1c87233dd407c Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Wed, 30 Oct 2024 16:41:34 -0400 Subject: [PATCH 03/39] Remove redundant task for json. Combine them into one --- .../executor/query/celeryconfig.py | 2 - .../executor/query/extract_ir_task.py | 30 ++++- .../executor/query/extract_json_task.py | 119 ------------------ .../scheduler/query/query_scheduler.py | 15 +-- 4 files changed, 26 insertions(+), 140 deletions(-) delete mode 100644 components/job-orchestration/job_orchestration/executor/query/extract_json_task.py diff --git a/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py b/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py index bfe6f6efc..994c0bbcf 100644 --- a/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py +++ b/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py @@ -5,13 +5,11 @@ imports = ( "job_orchestration.executor.query.fs_search_task", "job_orchestration.executor.query.extract_ir_task", - "job_orchestration.executor.query.extract_json_task", ) task_routes = { "job_orchestration.executor.query.fs_search_task.search": QueueName.QUERY, "job_orchestration.executor.query.extract_ir_task.extract_ir": QueueName.QUERY, - "job_orchestration.executor.query.extract_json_task.extract_json": QueueName.QUERY, } task_create_missing_queues = True diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py index 61fcbf549..4f01ed984 100644 --- a/components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py +++ b/components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py @@ -13,7 +13,7 @@ report_command_creation_failure, run_query_task, ) -from job_orchestration.scheduler.job_config import ExtractIrJobConfig +from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig from job_orchestration.scheduler.scheduler_data import QueryTaskStatus # Setup logging @@ -26,11 +26,13 @@ def make_command( archives_dir: Path, archive_id: str, ir_output_dir: Path, - extract_ir_config: ExtractIrJobConfig, + job_config_obj: dict, results_cache_uri: str, ir_collection: str, ) -> Optional[List[str]]: if StorageEngine.CLP == storage_engine: + logger.info("Start IR extraction") + extract_ir_config = ExtractIrJobConfig.parse_obj(job_config_obj) if not extract_ir_config.file_split_id: logger.error("file_split_id not supplied") return None @@ -46,6 +48,25 @@ def make_command( if extract_ir_config.target_uncompressed_size is not None: command.append("--target-size") command.append(str(extract_ir_config.target_uncompressed_size)) + elif StorageEngine.CLP_S == storage_engine: + logger.info("Start Json extraction") + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config_obj) + command = [ + str(clp_home / "bin" / "clp-s"), + "x", + str(archives_dir), + str(ir_output_dir), + "--ordered", + "--archive-id", + archive_id, + "--mongodb-uri", + results_cache_uri, + "--mongodb-collection", + ir_collection, + ] + if extract_json_config.target_chunk_size is not None: + command.append("--ordered-chunk-size") + command.append(str(extract_json_config.target_chunk_size)) else: logger.error(f"Unsupported storage engine {storage_engine}") return None @@ -63,7 +84,7 @@ def extract_ir( clp_metadata_db_conn_params: dict, results_cache_uri: str, ) -> Dict[str, Any]: - task_name = "IR extraction" + task_name = "Extraction" # Setup logging to file clp_logs_dir = Path(os.getenv("CLP_LOGS_DIR")) @@ -82,7 +103,6 @@ def extract_ir( clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") ir_output_dir = Path(os.getenv("CLP_IR_OUTPUT_DIR")) ir_collection = os.getenv("CLP_IR_COLLECTION") - extract_ir_config = ExtractIrJobConfig.parse_obj(job_config_obj) task_command = make_command( storage_engine=clp_storage_engine, @@ -90,7 +110,7 @@ def extract_ir( archives_dir=archive_directory, archive_id=archive_id, ir_output_dir=ir_output_dir, - extract_ir_config=extract_ir_config, + job_config_obj=job_config_obj, results_cache_uri=results_cache_uri, ir_collection=ir_collection, ) diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_json_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_json_task.py deleted file mode 100644 index ff2a99bfa..000000000 --- a/components/job-orchestration/job_orchestration/executor/query/extract_json_task.py +++ /dev/null @@ -1,119 +0,0 @@ -import datetime -import os -from pathlib import Path -from typing import Any, Dict, List, Optional - -from celery.app.task import Task -from celery.utils.log import get_task_logger -from clp_py_utils.clp_config import Database, StorageEngine -from clp_py_utils.clp_logging import set_logging_level -from clp_py_utils.sql_adapter import SQL_Adapter -from job_orchestration.executor.query.celery import app -from job_orchestration.executor.query.utils import ( - report_command_creation_failure, - run_query_task, -) -from job_orchestration.scheduler.job_config import ExtractJsonJobConfig -from job_orchestration.scheduler.scheduler_data import QueryTaskStatus - -# Setup logging -logger = get_task_logger(__name__) - - -def make_command( - storage_engine: str, - clp_home: Path, - archives_dir: Path, - archive_id: str, - output_dir: Path, - extract_json_config: ExtractJsonJobConfig, - results_cache_uri: str, - ir_collection: str, -) -> Optional[List[str]]: - if StorageEngine.CLP_S == storage_engine: - if not extract_json_config.archive_id: - logger.error("archive_id not supplied") - return None - command = [ - str(clp_home / "bin" / "clp-s"), - "x", - str(archives_dir), - str(output_dir), - "--ordered", - "--archive-id", - archive_id, - "--mongodb-uri", - results_cache_uri, - "--mongodb-collection", - ir_collection, - ] - if extract_json_config.target_chunk_size is not None: - command.append("--ordered-chunk-size") - command.append(str(extract_json_config.target_chunk_size)) - else: - logger.error(f"Unsupported storage engine {storage_engine}") - return None - - return command - - -@app.task(bind=True) -def extract_json( - self: Task, - job_id: str, - task_id: int, - job_config_obj: dict, - archive_id: str, - clp_metadata_db_conn_params: dict, - results_cache_uri: str, -) -> Dict[str, Any]: - task_name = "Json Extraction" - - # Setup logging to file - clp_logs_dir = Path(os.getenv("CLP_LOGS_DIR")) - clp_logging_level = os.getenv("CLP_LOGGING_LEVEL") - set_logging_level(logger, clp_logging_level) - - logger.info(f"Started {task_name} task for job {job_id}") - - start_time = datetime.datetime.now() - task_status: QueryTaskStatus - sql_adapter = SQL_Adapter(Database.parse_obj(clp_metadata_db_conn_params)) - - # Make task_command - clp_home = Path(os.getenv("CLP_HOME")) - archive_directory = Path(os.getenv("CLP_ARCHIVE_OUTPUT_DIR")) - clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") - ir_output_dir = Path(os.getenv("CLP_IR_OUTPUT_DIR")) - ir_collection = os.getenv("CLP_IR_COLLECTION") - extract_json_config = ExtractJsonJobConfig.parse_obj(job_config_obj) - - task_command = make_command( - storage_engine=clp_storage_engine, - clp_home=clp_home, - archives_dir=archive_directory, - archive_id=archive_id, - output_dir=ir_output_dir, - extract_json_config=extract_json_config, - results_cache_uri=results_cache_uri, - ir_collection=ir_collection, - ) - if not task_command: - return report_command_creation_failure( - sql_adapter=sql_adapter, - logger=logger, - task_name=task_name, - task_id=task_id, - start_time=start_time, - ) - - return run_query_task( - sql_adapter=sql_adapter, - logger=logger, - clp_logs_dir=clp_logs_dir, - task_command=task_command, - task_name=task_name, - job_id=job_id, - task_id=task_id, - start_time=start_time, - ) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index aa23940d4..616ca716c 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -40,7 +40,6 @@ from clp_py_utils.decorators import exception_default_value from clp_py_utils.sql_adapter import SQL_Adapter from job_orchestration.executor.query.extract_ir_task import extract_ir -from job_orchestration.executor.query.extract_json_task import extract_json from job_orchestration.executor.query.fs_search_task import search from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType, QueryTaskStatus from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig, SearchJobConfig @@ -357,7 +356,7 @@ def get_task_group_for_job( ) for i in range(len(archive_ids)) ) - elif QueryJobType.EXTRACT_IR == job_type: + elif job_type in [QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR]: return celery.group( extract_ir.s( job_id=job.id, @@ -369,18 +368,6 @@ def get_task_group_for_job( ) for i in range(len(archive_ids)) ) - elif QueryJobType.EXTRACT_JSON == job_type: - return celery.group( - extract_json.s( - job_id=job.id, - archive_id=archive_ids[i], - task_id=task_ids[i], - job_config_obj=job_config_obj, - clp_metadata_db_conn_params=clp_metadata_db_conn_params, - results_cache_uri=results_cache_uri, - ) - for i in range(len(archive_ids)) - ) else: error_msg = f"Unexpected job type: {job_type}" logger.error(error_msg) From d0e3b12431ce1633c3181e83a09ce8a80c764660 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Wed, 30 Oct 2024 16:46:15 -0400 Subject: [PATCH 04/39] Linter --- .../scripts/native/decompress.py | 25 +++++++++------- .../scheduler/query/query_scheduler.py | 30 +++++++++---------- 2 files changed, 30 insertions(+), 25 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index 14b3276ca..5ff766c93 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -12,7 +12,11 @@ from clp_py_utils.clp_config import CLP_METADATA_TABLE_PREFIX, CLPConfig, Database from clp_py_utils.sql_adapter import SQL_Adapter from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType -from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig, QueryJobConfig +from job_orchestration.scheduler.job_config import ( + ExtractIrJobConfig, + ExtractJsonJobConfig, + QueryJobConfig, +) from clp_package_utils.general import ( CLP_DEFAULT_CONFIG_FILE_RELATIVE_PATH, @@ -91,9 +95,7 @@ def submit_and_monitor_extraction_job_in_db( logger.info(f"Finished extraction job {job_id}.") return 0 - logger.error( - f"Extraction job {job_id} finished with unexpected status: {job_status.to_str()}." - ) + logger.error(f"Extraction job {job_id} finished with unexpected status: {job_status.to_str()}.") return -1 @@ -101,7 +103,7 @@ def handle_extract_cmd( parsed_args: argparse.Namespace, job_type: QueryJobType, clp_home: pathlib.Path, - default_config_file_path: pathlib.Path + default_config_file_path: pathlib.Path, ) -> int: """ Handles the IR extraction command. @@ -131,12 +133,11 @@ def handle_extract_cmd( extraction_config = ExtractIrJobConfig( orig_file_id=orig_file_id, msg_ix=parsed_args.msg_ix, - target_uncompressed_size=parsed_args.target_uncompressed_size + target_uncompressed_size=parsed_args.target_uncompressed_size, ) elif QueryJobType.EXTRACT_JSON == job_type: extraction_config = ExtractJsonJobConfig( - archive_id=parsed_args.archive_id, - target_chunk_size=parsed_args.target_chunk_size + archive_id=parsed_args.archive_id, target_chunk_size=parsed_args.target_chunk_size ) else: logger.exception(f"Unsupported extraction job type: {job_type}") @@ -301,9 +302,13 @@ def main(argv): if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) elif EXTRACT_IR_CMD == command: - return handle_extract_cmd(parsed_args, QueryJobType.EXTRACT_IR, clp_home, default_config_file_path) + return handle_extract_cmd( + parsed_args, QueryJobType.EXTRACT_IR, clp_home, default_config_file_path + ) elif EXTRACT_JSON_CMD == command: - return handle_extract_cmd(parsed_args, QueryJobType.EXTRACT_JSON, clp_home, default_config_file_path) + return handle_extract_cmd( + parsed_args, QueryJobType.EXTRACT_JSON, clp_home, default_config_file_path + ) else: logger.exception(f"Unexpected command: {command}") return -1 diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 616ca716c..bae8e6750 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -42,7 +42,11 @@ from job_orchestration.executor.query.extract_ir_task import extract_ir from job_orchestration.executor.query.fs_search_task import search from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType, QueryTaskStatus -from job_orchestration.scheduler.job_config import ExtractIrJobConfig, ExtractJsonJobConfig, SearchJobConfig +from job_orchestration.scheduler.job_config import ( + ExtractIrJobConfig, + ExtractJsonJobConfig, + SearchJobConfig, +) from job_orchestration.scheduler.query.reducer_handler import ( handle_reducer_connection, ReducerHandlerMessage, @@ -650,21 +654,19 @@ def handle_pending_query_jobs( continue # Check if the archive with the expected timestamp has already been extracted - if json_file_exists_for_archive( - results_cache_uri, ir_collection_name, archive_id - ): + if json_file_exists_for_archive(results_cache_uri, ir_collection_name, archive_id): logger.info( f"archive {archive_id} already extracted, so mark job {job_id} as done" ) if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.SUCCEEDED, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - duration=0, + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.SUCCEEDED, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + duration=0, ): logger.error(f"Failed to set job {job_id} as succeeded") continue @@ -769,9 +771,7 @@ def ir_file_exists_for_file_split( return 0 != results_count -def json_file_exists_for_archive( - results_cache_uri: str, ir_collection_name: str, archive_id: str -): +def json_file_exists_for_archive(results_cache_uri: str, ir_collection_name: str, archive_id: str): with pymongo.MongoClient(results_cache_uri) as results_cache_client: ir_collection = results_cache_client.get_default_database()[ir_collection_name] results_count = ir_collection.count_documents({"orig_file_id": archive_id}) From 6af5d5a794f84d6242fd53f147cc6dace1dd2976 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Wed, 30 Oct 2024 17:40:22 -0400 Subject: [PATCH 05/39] refactor --- .../scheduler/query/query_scheduler.py | 148 ++++++++---------- .../scheduler/scheduler_data.py | 3 - 2 files changed, 61 insertions(+), 90 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index bae8e6750..fd381802d 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -339,6 +339,22 @@ def get_archive_and_file_split_ids( return results +def check_if_archive_exists( + db_conn, + archive_id: str, +) -> bool: + + query = f"""SELECT 1 + FROM {CLP_METADATA_TABLE_PREFIX}archives WHERE + id = '{archive_id}' + """ + with contextlib.closing(db_conn.cursor(dictionary=True)) as cursor: + cursor.execute(query) + if cursor.fetchone(): + return True + + return False + def get_task_group_for_job( archive_ids: List[str], task_ids: List[int], @@ -603,12 +619,10 @@ def handle_pending_query_jobs( extract_ir_config.file_split_id = file_split_id new_extract_ir_job = ExtractIrJob( id=job_id, - archive_id=archive_id, - file_split_id=file_split_id, extract_ir_config=extract_ir_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) - target_archive = [new_extract_ir_job.archive_id] + target_archive = [archive_id] dispatch_job_and_update_db( db_conn, @@ -624,8 +638,23 @@ def handle_pending_query_jobs( elif QueryJobType.EXTRACT_JSON == job_type: extract_json_config = ExtractJsonJobConfig.parse_obj(msgpack.unpackb(job_config)) - # TODO: For now, let's assume the archive_id is always valid. Will need to verify this somehow archive_id = extract_json_config.archive_id + if not check_if_archive_exists(db_conn, archive_id): + logger.warning( + f"archive {archive_id} does not exist, mark job {job_id} as failed" + ) + if not set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.FAILED, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + duration=0, + ): + logger.error(f"Failed to set job {job_id} as failed") + continue # NOTE: The following two if blocks should not be reordered since if we first check # whether *an* IR file has been extracted for the requested file split, it doesn't @@ -674,11 +703,10 @@ def handle_pending_query_jobs( active_archive_json_extractions[archive_id] = [job_id] new_extract_json_job = ExtractJsonJob( id=job_id, - archive_id=archive_id, extract_json_config=extract_json_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) - target_archive = [new_extract_json_job.archive_id] + target_archive = [archive_id] dispatch_job_and_update_db( db_conn, @@ -864,19 +892,20 @@ async def handle_finished_search_job( del active_jobs[job_id] -async def handle_finished_extract_ir_job( - db_conn, job: ExtractIrJob, task_results: Optional[Any] +async def handle_finished_extraction_job( + db_conn, job: QueryJob, task_results: List[Any] ) -> None: global active_jobs + global active_archive_json_extractions global active_file_split_ir_extractions job_id = job.id - file_split_id = job.file_split_id new_job_status = QueryJobStatus.SUCCEEDED + num_tasks = len(task_results) if 1 != num_tasks: logger.error( - f"Unexpected number of tasks for IR extraction job {job_id}. " + f"Unexpected number of tasks for extraction job {job_id}. " f"Expected 1, got {num_tasks}." ) new_job_status = QueryJobStatus.FAILED @@ -885,94 +914,43 @@ async def handle_finished_extract_ir_job( task_id = task_result.task_id if not QueryJobStatus.SUCCEEDED == task_result.status: logger.error( - f"IR extraction task job-{job_id}-task-{task_id} failed. " + f"extraction task job-{job_id}-task-{task_id} failed. " f"Check {task_result.error_log_path} for details." ) new_job_status = QueryJobStatus.FAILED else: logger.info( - f"IR extraction task job-{job_id}-task-{task_id} succeeded in " + f"extraction task job-{job_id}-task-{task_id} succeeded in " f"{task_result.duration} second(s)." ) if set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - new_job_status, - QueryJobStatus.RUNNING, - num_tasks_completed=num_tasks, - duration=(datetime.datetime.now() - job.start_time).total_seconds(), - ): - if new_job_status == QueryJobStatus.SUCCEEDED: - logger.info(f"Completed IR extraction job {job_id}.") - else: - logger.info(f"Completed IR extraction job {job_id} with failing tasks.") - - waiting_jobs = active_file_split_ir_extractions[file_split_id] - waiting_jobs.remove(job_id) - for waiting_job in waiting_jobs: - logger.info(f"Setting status to {new_job_status.to_str()} for waiting jobs: {waiting_job}.") - set_job_or_task_status( db_conn, QUERY_JOBS_TABLE_NAME, - waiting_job, + job_id, new_job_status, QueryJobStatus.RUNNING, - num_tasks_completed=0, + num_tasks_completed=num_tasks, duration=(datetime.datetime.now() - job.start_time).total_seconds(), - ) - - del active_file_split_ir_extractions[file_split_id] - del active_jobs[job_id] - - -async def handle_finished_extract_json_job( - db_conn, job: ExtractJsonJob, task_results: Optional[Any] -) -> None: - global active_jobs - global active_archive_json_extractions - - job_id = job.id - archive_id = job.archive_id - new_job_status = QueryJobStatus.SUCCEEDED - num_tasks = len(task_results) - if 1 != num_tasks: - logger.error( - f"Unexpected number of tasks for Json extraction job {job_id}. " - f"Expected 1, got {num_tasks}." - ) - new_job_status = QueryJobStatus.FAILED - else: - task_result = QueryTaskResult.parse_obj(task_results[0]) - task_id = task_result.task_id - if not QueryJobStatus.SUCCEEDED == task_result.status: - logger.error( - f"Json extraction task job-{job_id}-task-{task_id} failed. " - f"Check {task_result.error_log_path} for details." - ) - new_job_status = QueryJobStatus.FAILED - else: - logger.info( - f"Json extraction task job-{job_id}-task-{task_id} succeeded in " - f"{task_result.duration} second(s)." - ) - - if set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - new_job_status, - QueryJobStatus.RUNNING, - num_tasks_completed=num_tasks, - duration=(datetime.datetime.now() - job.start_time).total_seconds(), ): if new_job_status == QueryJobStatus.SUCCEEDED: - logger.info(f"Completed IR extraction job {job_id}.") + logger.info(f"Completed extraction job {job_id}.") else: - logger.info(f"Completed IR extraction job {job_id} with failing tasks.") + logger.info(f"Completed extraction job {job_id} with failing tasks.") + + # TODO: find better name for these two variables + waiting_jobs_map: Dict[str, List[str]] + job_key: str + if QueryJobType.EXTRACT_IR == job.get_type(): + waiting_jobs_map = active_file_split_ir_extractions + extract_ir_config: ExtractIrJobConfig = job.get_config() + job_key = extract_ir_config.file_split_id + else: + waiting_jobs_map = active_archive_json_extractions + extract_json_config: ExtractJsonJobConfig = job.get_config() + job_key = extract_json_config.archive_id - waiting_jobs = active_archive_json_extractions[archive_id] + waiting_jobs = waiting_jobs_map[job_key] waiting_jobs.remove(job_id) for waiting_job in waiting_jobs: logger.info(f"Setting status to {new_job_status.to_str()} for waiting jobs: {waiting_job}.") @@ -986,7 +964,7 @@ async def handle_finished_extract_json_job( duration=(datetime.datetime.now() - job.start_time).total_seconds(), ) - del active_archive_json_extractions[archive_id] + del waiting_jobs_map[job_key] del active_jobs[job_id] @@ -1027,12 +1005,8 @@ async def check_job_status_and_update_db(db_conn_pool, results_cache_uri): await handle_finished_search_job( db_conn, search_job, returned_results, results_cache_uri ) - elif QueryJobType.EXTRACT_IR == job_type: - extract_ir_job: ExtractIrJob = job - await handle_finished_extract_ir_job(db_conn, extract_ir_job, returned_results) - elif QueryJobType.EXTRACT_JSON == job_type: - extract_json_job: ExtractJsonJob = job - await handle_finished_extract_json_job(db_conn, extract_json_job, returned_results) + elif job_type in [QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR]: + await handle_finished_extraction_job(db_conn, job, returned_results) else: logger.error(f"Unexpected job type: {job_type}, skipping job {job_id}") diff --git a/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py b/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py index fe26bef97..4f49a7c1a 100644 --- a/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py +++ b/components/job-orchestration/job_orchestration/scheduler/scheduler_data.py @@ -60,8 +60,6 @@ def get_config(self) -> QueryJobConfig: ... class ExtractIrJob(QueryJob): extract_ir_config: ExtractIrJobConfig - file_split_id: str - archive_id: str def get_type(self) -> QueryJobType: return QueryJobType.EXTRACT_IR @@ -72,7 +70,6 @@ def get_config(self) -> QueryJobConfig: class ExtractJsonJob(QueryJob): extract_json_config: ExtractJsonJobConfig - archive_id: str def get_type(self) -> QueryJobType: return QueryJobType.EXTRACT_JSON From 496e8d80a8ad976846b87b037dc4483c8cfe8a97 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Wed, 30 Oct 2024 21:12:59 -0400 Subject: [PATCH 06/39] Linter --- .../scheduler/query/query_scheduler.py | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index fd381802d..f9b68f7be 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -355,6 +355,7 @@ def check_if_archive_exists( return False + def get_task_group_for_job( archive_ids: List[str], task_ids: List[int], @@ -892,9 +893,7 @@ async def handle_finished_search_job( del active_jobs[job_id] -async def handle_finished_extraction_job( - db_conn, job: QueryJob, task_results: List[Any] -) -> None: +async def handle_finished_extraction_job(db_conn, job: QueryJob, task_results: List[Any]) -> None: global active_jobs global active_archive_json_extractions global active_file_split_ir_extractions @@ -925,13 +924,13 @@ async def handle_finished_extraction_job( ) if set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - new_job_status, - QueryJobStatus.RUNNING, - num_tasks_completed=num_tasks, - duration=(datetime.datetime.now() - job.start_time).total_seconds(), + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + new_job_status, + QueryJobStatus.RUNNING, + num_tasks_completed=num_tasks, + duration=(datetime.datetime.now() - job.start_time).total_seconds(), ): if new_job_status == QueryJobStatus.SUCCEEDED: logger.info(f"Completed extraction job {job_id}.") From c4e118d35fcc1176a9f2753cfa07b97895f30d54 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 1 Nov 2024 16:13:25 -0400 Subject: [PATCH 07/39] refactor --- .../scheduler/query/query_scheduler.py | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index f9b68f7be..94b487e04 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -937,20 +937,14 @@ async def handle_finished_extraction_job(db_conn, job: QueryJob, task_results: L else: logger.info(f"Completed extraction job {job_id} with failing tasks.") - # TODO: find better name for these two variables - waiting_jobs_map: Dict[str, List[str]] - job_key: str + waiting_jobs: List[str] if QueryJobType.EXTRACT_IR == job.get_type(): - waiting_jobs_map = active_file_split_ir_extractions extract_ir_config: ExtractIrJobConfig = job.get_config() - job_key = extract_ir_config.file_split_id + waiting_jobs = active_file_split_ir_extractions.pop(extract_ir_config.file_split_id) else: - waiting_jobs_map = active_archive_json_extractions extract_json_config: ExtractJsonJobConfig = job.get_config() - job_key = extract_json_config.archive_id + waiting_jobs = active_archive_json_extractions.pop(extract_json_config.archive_id) - waiting_jobs = waiting_jobs_map[job_key] - waiting_jobs.remove(job_id) for waiting_job in waiting_jobs: logger.info(f"Setting status to {new_job_status.to_str()} for waiting jobs: {waiting_job}.") set_job_or_task_status( @@ -963,7 +957,6 @@ async def handle_finished_extraction_job(db_conn, job: QueryJob, task_results: L duration=(datetime.datetime.now() - job.start_time).total_seconds(), ) - del waiting_jobs_map[job_key] del active_jobs[job_id] From a75c42d9123cfd6fd97bfc8e38afdc3b4097acb5 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 1 Nov 2024 21:00:36 -0400 Subject: [PATCH 08/39] proposal --- .../scheduler/query/query_scheduler.py | 108 ++++++++++++++++++ 1 file changed, 108 insertions(+) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 94b487e04..68935b55e 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -288,6 +288,32 @@ def get_archives_for_search( return archives_for_search +def get_archive_and_target_ids_for_extraction( + db_conn, + job_config, + job_type: QueryJobType +) -> Tuple[Optional[str], Optional[str]]: + archive_id = None + target_id = None + + if QueryJobType.EXTRACT_IR == job_type: + extract_ir_config = ExtractIrJobConfig.parse_obj(msgpack.unpackb(job_config)) + archive_id, target_id = get_archive_and_file_split_ids_for_extraction( + db_conn, extract_ir_config + ) + else: + extract_json_config = ExtractJsonJobConfig.parse_obj(msgpack.unpackb(job_config)) + if check_if_archive_exists(db_conn, extract_json_config.archive_id): + archive_id = extract_json_config.archive_id + target_id = archive_id + else: + logger.error( + f"archive {archive_id} does not exist for" + ) + return archive_id, target_id + + + def get_archive_and_file_split_ids_for_extraction( db_conn, extract_ir_config: ExtractIrJobConfig, @@ -551,6 +577,88 @@ def handle_pending_query_jobs( pending_search_jobs.append(new_search_job) active_jobs[job_id] = new_search_job + elif job_type in [QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON]: + archive_id, target_id = get_archive_and_target_ids_for_extraction( + db_conn, job_config, job_type + ) + if not target_id: + if not set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.FAILED, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + duration=0, + ): + logger.error(f"Failed to set job {job_id} as failed") + continue + + # TODO: update this + # NOTE: The following two if blocks should not be reordered since if we first check + # whether *an* IR file has been extracted for the requested file split, it doesn't + # mean that *all* IR files have has been extracted for the file split (since the + # extraction job may still be in progress). Thus, we must first check whether the + # file split is in the process of being extracted, and then check whether it's + # already been extracted. + + # Check if the target is currently being extracted; if so, add the job ID to the + # list of jobs waiting for it. + if is_target_extraction_active(target_id, job_type): + append_job_to_wait_list(target_id, job_id, job_type) + logger.info( + f"target {target_id} is being extracted, so mark job {job_id} as running" + ) + if not set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.RUNNING, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + ): + logger.error(f"Failed to set job {job_id} as running") + continue + + # Check if the target has already been extracted + if is_target_extracted(results_cache_uri, ir_collection_name, target_id): + logger.info( + f"target {target_id} already extracted, so mark job {job_id} as done" + ) + if not set_job_or_task_status( + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.SUCCEEDED, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + duration=0, + ): + logger.error(f"Failed to set job {job_id} as succeeded") + continue + + # TODO: can put mark_target_as_active into create_extraction_job + mark_target_as_active(target_id, job_id, job_type) + nex_extraction_job = create_extraction_job( + job_id, + job_config, + job_type + ) + + dispatch_job_and_update_db( + db_conn, + nex_extraction_job, + target_archive, + clp_metadata_db_conn_params, + results_cache_uri, + 1, + ) + active_jobs[job_id] = nex_extraction_job + logger.info(f"Dispatched extraction job {job_id} on archive: {archive_id}, target: {target_id}") + elif QueryJobType.EXTRACT_IR == job_type: extract_ir_config = ExtractIrJobConfig.parse_obj(msgpack.unpackb(job_config)) archive_id, file_split_id = get_archive_and_file_split_ids_for_extraction( From 66fb365eff5c3897a6f4451da16b50429fe6d969 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Mon, 4 Nov 2024 13:05:01 -0500 Subject: [PATCH 09/39] Add actual refactor --- .../scheduler/query/query_scheduler.py | 273 +++++------------- 1 file changed, 78 insertions(+), 195 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 68935b55e..dbb87b689 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -290,27 +290,25 @@ def get_archives_for_search( def get_archive_and_target_ids_for_extraction( db_conn, - job_config, + job_config: Dict[str, Any], job_type: QueryJobType ) -> Tuple[Optional[str], Optional[str]]: - archive_id = None - target_id = None - if QueryJobType.EXTRACT_IR == job_type: - extract_ir_config = ExtractIrJobConfig.parse_obj(msgpack.unpackb(job_config)) - archive_id, target_id = get_archive_and_file_split_ids_for_extraction( + extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) + return get_archive_and_file_split_ids_for_extraction( db_conn, extract_ir_config ) + + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) + archive_id = extract_json_config.archive_id + if check_if_archive_exists(db_conn, extract_json_config.archive_id): + return archive_id, archive_id else: - extract_json_config = ExtractJsonJobConfig.parse_obj(msgpack.unpackb(job_config)) - if check_if_archive_exists(db_conn, extract_json_config.archive_id): - archive_id = extract_json_config.archive_id - target_id = archive_id - else: - logger.error( - f"archive {archive_id} does not exist for" - ) - return archive_id, target_id + logger.error( + f"archive {archive_id} does not exist" + ) + + return None, None @@ -365,6 +363,61 @@ def get_archive_and_file_split_ids( return results +def is_target_extraction_active(target_id: str, job_type: QueryJobType): + if QueryJobType.EXTRACT_IR == job_type: + return target_id in active_file_split_ir_extractions + return target_id in active_archive_json_extractions + + +def mark_job_waiting_for_target(target_id: str, job_id:str, job_type: QueryJobType): + global active_file_split_ir_extractions + global active_archive_json_extractions + + active_extraction_lists: Dict[str, List[str]] + if QueryJobType.EXTRACT_IR == job_type: + active_extraction_lists = active_file_split_ir_extractions + else: + active_extraction_lists = active_archive_json_extractions + + if target_id not in active_extraction_lists: + active_extraction_lists[target_id] = [] + active_extraction_lists[target_id].append(job_id) + + +def is_target_extracted(results_cache_uri: str, ir_collection_name: str, target_id: str, job_type: QueryJobType): + if QueryJobType.EXTRACT_IR == job_type: + return ir_file_exists_for_file_split(results_cache_uri, ir_collection_name, target_id) + return json_file_exists_for_archive(results_cache_uri, ir_collection_name, target_id) + + +def create_extraction_job( + job_id: str, + job_config: Dict[str, Any], + target_id: str, + job_type: QueryJobType +): + new_extraction_job: QueryJob + new_job_state = InternalJobState.WAITING_FOR_DISPATCH + if QueryJobType.EXTRACT_IR == job_type: + extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) + extract_ir_config.file_split_id = target_id + new_extraction_job = ExtractIrJob( + id=job_id, + extract_ir_config=extract_ir_config, + state=new_job_state, + ) + logger.info(f"Created ir extraction job {job_id} on file_split: {target_id}") + else: + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) + new_extraction_job = ExtractJsonJob( + id=job_id, + extract_json_config=extract_json_config, + state=new_job_state, + ) + logger.info(f"Created json extraction job {job_id} on archive: {target_id}") + return new_extraction_job + + def check_if_archive_exists( db_conn, archive_id: str, @@ -519,8 +572,6 @@ def handle_pending_query_jobs( num_archives_to_search_per_sub_job: int, ) -> List[asyncio.Task]: global active_jobs - global active_file_split_ir_extractions - global active_archive_json_extractions reducer_acquisition_tasks = [] pending_search_jobs = [ @@ -534,14 +585,14 @@ def handle_pending_query_jobs( for job in fetch_new_query_jobs(db_conn): job_id = str(job["job_id"]) job_type = job["type"] - job_config = job["job_config"] + job_config = msgpack.unpackb(job["job_config"]) if QueryJobType.SEARCH_OR_AGGREGATION == job_type: # Avoid double-dispatch when a job is WAITING_FOR_REDUCER if job_id in active_jobs: continue - search_config = SearchJobConfig.parse_obj(msgpack.unpackb(job_config)) + search_config = SearchJobConfig.parse_obj(job_config) archives_for_search = get_archives_for_search(db_conn, search_config) if len(archives_for_search) == 0: if set_job_or_task_status( @@ -595,7 +646,7 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as failed") continue - # TODO: update this + # TODO: update this comment # NOTE: The following two if blocks should not be reordered since if we first check # whether *an* IR file has been extracted for the requested file split, it doesn't # mean that *all* IR files have has been extracted for the file split (since the @@ -606,7 +657,7 @@ def handle_pending_query_jobs( # Check if the target is currently being extracted; if so, add the job ID to the # list of jobs waiting for it. if is_target_extraction_active(target_id, job_type): - append_job_to_wait_list(target_id, job_id, job_type) + mark_job_waiting_for_target(target_id, job_id, job_type) logger.info( f"target {target_id} is being extracted, so mark job {job_id} as running" ) @@ -623,7 +674,7 @@ def handle_pending_query_jobs( continue # Check if the target has already been extracted - if is_target_extracted(results_cache_uri, ir_collection_name, target_id): + if is_target_extracted(results_cache_uri, ir_collection_name, target_id, job_type): logger.info( f"target {target_id} already extracted, so mark job {job_id} as done" ) @@ -640,193 +691,25 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as succeeded") continue - # TODO: can put mark_target_as_active into create_extraction_job - mark_target_as_active(target_id, job_id, job_type) nex_extraction_job = create_extraction_job( job_id, job_config, + target_id, job_type ) dispatch_job_and_update_db( db_conn, nex_extraction_job, - target_archive, + [archive_id], clp_metadata_db_conn_params, results_cache_uri, 1, ) - active_jobs[job_id] = nex_extraction_job - logger.info(f"Dispatched extraction job {job_id} on archive: {archive_id}, target: {target_id}") - elif QueryJobType.EXTRACT_IR == job_type: - extract_ir_config = ExtractIrJobConfig.parse_obj(msgpack.unpackb(job_config)) - archive_id, file_split_id = get_archive_and_file_split_ids_for_extraction( - db_conn, extract_ir_config - ) - if not archive_id or not file_split_id: - if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.FAILED, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - duration=0, - ): - logger.error(f"Failed to set job {job_id} as failed") - continue - - # NOTE: The following two if blocks should not be reordered since if we first check - # whether *an* IR file has been extracted for the requested file split, it doesn't - # mean that *all* IR files have has been extracted for the file split (since the - # extraction job may still be in progress). Thus, we must first check whether the - # file split is in the process of being extracted, and then check whether it's - # already been extracted. - - # Check if the file split is currently being extracted; if so, add the job ID to the - # list of jobs waiting for it. - if file_split_id in active_file_split_ir_extractions: - active_file_split_ir_extractions[file_split_id].append(job_id) - logger.info( - f"Split {file_split_id} is being extracted, so mark job {job_id} as running" - ) - if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.RUNNING, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - ): - logger.error(f"Failed to set job {job_id} as running") - continue - - # Check if the file split has already been extracted - if ir_file_exists_for_file_split( - results_cache_uri, ir_collection_name, file_split_id - ): - logger.info( - f"Split {file_split_id} already extracted, so mark job {job_id} as done" - ) - if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.SUCCEEDED, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - duration=0, - ): - logger.error(f"Failed to set job {job_id} as succeeded") - continue - - active_file_split_ir_extractions[file_split_id] = [job_id] - extract_ir_config.file_split_id = file_split_id - new_extract_ir_job = ExtractIrJob( - id=job_id, - extract_ir_config=extract_ir_config, - state=InternalJobState.WAITING_FOR_DISPATCH, - ) - target_archive = [archive_id] - - dispatch_job_and_update_db( - db_conn, - new_extract_ir_job, - target_archive, - clp_metadata_db_conn_params, - results_cache_uri, - 1, - ) - active_jobs[new_extract_ir_job.id] = new_extract_ir_job - logger.info(f"Dispatched IR extraction job {job_id} on archive: {archive_id}") - - elif QueryJobType.EXTRACT_JSON == job_type: - extract_json_config = ExtractJsonJobConfig.parse_obj(msgpack.unpackb(job_config)) - - archive_id = extract_json_config.archive_id - if not check_if_archive_exists(db_conn, archive_id): - logger.warning( - f"archive {archive_id} does not exist, mark job {job_id} as failed" - ) - if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.FAILED, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - duration=0, - ): - logger.error(f"Failed to set job {job_id} as failed") - continue - - # NOTE: The following two if blocks should not be reordered since if we first check - # whether *an* IR file has been extracted for the requested file split, it doesn't - # mean that *all* IR files have has been extracted for the file split (since the - # extraction job may still be in progress). Thus, we must first check whether the - # file split is in the process of being extracted, and then check whether it's - # already been extracted. - - # Check if the archive is currently being extracted; if so, add the job ID to the - # list of jobs waiting for it. - if archive_id in active_archive_json_extractions: - active_archive_json_extractions[archive_id].append(job_id) - logger.info( - f"archive {archive_id} is being extracted, so mark job {job_id} as running" - ) - if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.RUNNING, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - ): - logger.error(f"Failed to set job {job_id} as running") - continue - - # Check if the archive with the expected timestamp has already been extracted - if json_file_exists_for_archive(results_cache_uri, ir_collection_name, archive_id): - logger.info( - f"archive {archive_id} already extracted, so mark job {job_id} as done" - ) - if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.SUCCEEDED, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - duration=0, - ): - logger.error(f"Failed to set job {job_id} as succeeded") - continue - - active_archive_json_extractions[archive_id] = [job_id] - new_extract_json_job = ExtractJsonJob( - id=job_id, - extract_json_config=extract_json_config, - state=InternalJobState.WAITING_FOR_DISPATCH, - ) - target_archive = [archive_id] - - dispatch_job_and_update_db( - db_conn, - new_extract_json_job, - target_archive, - clp_metadata_db_conn_params, - results_cache_uri, - 1, - ) - active_jobs[new_extract_json_job.id] = new_extract_json_job - logger.info(f"Dispatched Json extraction job {job_id} on archive: {archive_id}") + mark_job_waiting_for_target(target_id, job_id, job_type) + active_jobs[job_id] = nex_extraction_job + logger.info(f"Dispatched extraction job {job_id} on archive: {archive_id}") else: # NOTE: We're skipping the job for this iteration, but its status will remain From 310f0d55e1dd9d87678bd5ae3b2d8979d9518d32 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Mon, 4 Nov 2024 13:11:30 -0500 Subject: [PATCH 10/39] Linter --- .../scheduler/query/query_scheduler.py | 61 ++++++++----------- 1 file changed, 24 insertions(+), 37 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index dbb87b689..30afa0a74 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -289,29 +289,22 @@ def get_archives_for_search( def get_archive_and_target_ids_for_extraction( - db_conn, - job_config: Dict[str, Any], - job_type: QueryJobType + db_conn, job_config: Dict[str, Any], job_type: QueryJobType ) -> Tuple[Optional[str], Optional[str]]: if QueryJobType.EXTRACT_IR == job_type: extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) - return get_archive_and_file_split_ids_for_extraction( - db_conn, extract_ir_config - ) + return get_archive_and_file_split_ids_for_extraction(db_conn, extract_ir_config) extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) archive_id = extract_json_config.archive_id if check_if_archive_exists(db_conn, extract_json_config.archive_id): return archive_id, archive_id else: - logger.error( - f"archive {archive_id} does not exist" - ) + logger.error(f"archive {archive_id} does not exist") return None, None - def get_archive_and_file_split_ids_for_extraction( db_conn, extract_ir_config: ExtractIrJobConfig, @@ -369,7 +362,7 @@ def is_target_extraction_active(target_id: str, job_type: QueryJobType): return target_id in active_archive_json_extractions -def mark_job_waiting_for_target(target_id: str, job_id:str, job_type: QueryJobType): +def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobType): global active_file_split_ir_extractions global active_archive_json_extractions @@ -384,17 +377,16 @@ def mark_job_waiting_for_target(target_id: str, job_id:str, job_type: QueryJobTy active_extraction_lists[target_id].append(job_id) -def is_target_extracted(results_cache_uri: str, ir_collection_name: str, target_id: str, job_type: QueryJobType): +def is_target_extracted( + results_cache_uri: str, ir_collection_name: str, target_id: str, job_type: QueryJobType +): if QueryJobType.EXTRACT_IR == job_type: return ir_file_exists_for_file_split(results_cache_uri, ir_collection_name, target_id) return json_file_exists_for_archive(results_cache_uri, ir_collection_name, target_id) def create_extraction_job( - job_id: str, - job_config: Dict[str, Any], - target_id: str, - job_type: QueryJobType + job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType ): new_extraction_job: QueryJob new_job_state = InternalJobState.WAITING_FOR_DISPATCH @@ -662,13 +654,13 @@ def handle_pending_query_jobs( f"target {target_id} is being extracted, so mark job {job_id} as running" ) if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.RUNNING, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.RUNNING, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, ): logger.error(f"Failed to set job {job_id} as running") continue @@ -679,24 +671,19 @@ def handle_pending_query_jobs( f"target {target_id} already extracted, so mark job {job_id} as done" ) if not set_job_or_task_status( - db_conn, - QUERY_JOBS_TABLE_NAME, - job_id, - QueryJobStatus.SUCCEEDED, - QueryJobStatus.PENDING, - start_time=datetime.datetime.now(), - num_tasks=0, - duration=0, + db_conn, + QUERY_JOBS_TABLE_NAME, + job_id, + QueryJobStatus.SUCCEEDED, + QueryJobStatus.PENDING, + start_time=datetime.datetime.now(), + num_tasks=0, + duration=0, ): logger.error(f"Failed to set job {job_id} as succeeded") continue - nex_extraction_job = create_extraction_job( - job_id, - job_config, - target_id, - job_type - ) + nex_extraction_job = create_extraction_job(job_id, job_config, target_id, job_type) dispatch_job_and_update_db( db_conn, From 069f5686e944da8419f9d2c5ca3b359dd550d4b4 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Mon, 4 Nov 2024 14:40:43 -0500 Subject: [PATCH 11/39] deduplication --- .../scheduler/query/query_scheduler.py | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 30afa0a74..500ec2a04 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -380,9 +380,16 @@ def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobT def is_target_extracted( results_cache_uri: str, ir_collection_name: str, target_id: str, job_type: QueryJobType ): + target_key: str if QueryJobType.EXTRACT_IR == job_type: - return ir_file_exists_for_file_split(results_cache_uri, ir_collection_name, target_id) - return json_file_exists_for_archive(results_cache_uri, ir_collection_name, target_id) + target_key = "file_split_id" + else: + target_key = "orig_file_id" + + with pymongo.MongoClient(results_cache_uri) as results_cache_client: + ir_collection = results_cache_client.get_default_database()[ir_collection_name] + results_count = ir_collection.count_documents({target_key: target_id}) + return 0 != results_count def create_extraction_job( @@ -769,22 +776,6 @@ def found_max_num_latest_results( return max_timestamp_in_remaining_archives <= min_timestamp_in_top_results -def ir_file_exists_for_file_split( - results_cache_uri: str, ir_collection_name: str, file_split_id: str -): - with pymongo.MongoClient(results_cache_uri) as results_cache_client: - ir_collection = results_cache_client.get_default_database()[ir_collection_name] - results_count = ir_collection.count_documents({"file_split_id": file_split_id}) - return 0 != results_count - - -def json_file_exists_for_archive(results_cache_uri: str, ir_collection_name: str, archive_id: str): - with pymongo.MongoClient(results_cache_uri) as results_cache_client: - ir_collection = results_cache_client.get_default_database()[ir_collection_name] - results_count = ir_collection.count_documents({"orig_file_id": archive_id}) - return 0 != results_count - - async def handle_finished_search_job( db_conn, job: SearchJob, task_results: Optional[Any], results_cache_uri: str ) -> None: From 30c03e3fce1eca11a63e3a332434c184825708b8 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Mon, 4 Nov 2024 13:53:25 -0500 Subject: [PATCH 12/39] Decompressor end renaming --- .../clp_package_utils/scripts/decompress.py | 10 +++---- .../scripts/native/decompress.py | 30 +++++++++---------- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/decompress.py index c88de19df..ec5280e41 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/decompress.py @@ -147,11 +147,11 @@ def handle_extract_file_cmd( return 0 -def handle_extract_cmd( +def handle_extract_stream_cmd( parsed_args, clp_home: pathlib.Path, default_config_file_path: pathlib.Path ) -> int: """ - Handles the extraction command. + Handles the stream extraction command. :param parsed_args: :param clp_home: :param default_config_file_path: @@ -208,7 +208,7 @@ def handle_extract_cmd( try: subprocess.run(cmd, check=True) except subprocess.CalledProcessError: - logger.exception("Docker or IR extraction command failed.") + logger.exception("Docker or stream extraction command failed.") return -1 # Remove generated files @@ -253,7 +253,7 @@ def main(argv): group.add_argument("--orig-file-id", type=str, help="Original file's ID.") group.add_argument("--orig-file-path", type=str, help="Original file's path.") - # IR extraction command parser + # Json extraction command parser json_extraction_parser = command_args_parser.add_parser(EXTRACT_JSON_CMD) json_extraction_parser.add_argument("archive_id", type=str, help="Archive ID") json_extraction_parser.add_argument( @@ -266,7 +266,7 @@ def main(argv): if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) elif command in [EXTRACT_IR_CMD, EXTRACT_JSON_CMD]: - return handle_extract_cmd(parsed_args, clp_home, default_config_file_path) + return handle_extract_stream_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") return -1 diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index 5ff766c93..997060ed5 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -99,14 +99,13 @@ def submit_and_monitor_extraction_job_in_db( return -1 -def handle_extract_cmd( +def handle_extract_stream_cmd( parsed_args: argparse.Namespace, - job_type: QueryJobType, clp_home: pathlib.Path, default_config_file_path: pathlib.Path, ) -> int: """ - Handles the IR extraction command. + Handles the stream extraction command. :param parsed_args: :param clp_home: :param default_config_file_path: @@ -119,8 +118,12 @@ def handle_extract_cmd( if clp_config is None: return -1 + command = parsed_args.command + extraction_config: QueryJobConfig - if QueryJobType.EXTRACT_IR == job_type: + job_type: QueryJobType + if EXTRACT_IR_CMD == command: + job_type = QueryJobType.EXTRACT_IR orig_file_id: str if parsed_args.orig_file_id: orig_file_id = parsed_args.orig_file_id @@ -135,12 +138,15 @@ def handle_extract_cmd( msg_ix=parsed_args.msg_ix, target_uncompressed_size=parsed_args.target_uncompressed_size, ) - elif QueryJobType.EXTRACT_JSON == job_type: + elif EXTRACT_JSON_CMD == command: + job_type = QueryJobType.EXTRACT_JSON extraction_config = ExtractJsonJobConfig( archive_id=parsed_args.archive_id, target_chunk_size=parsed_args.target_chunk_size ) else: - logger.exception(f"Unsupported extraction job type: {job_type}") + logger.exception(f"Unsupported stream extraction command: {command}") + return -1 + try: return asyncio.run( run_function_in_process( @@ -151,7 +157,7 @@ def handle_extract_cmd( ) ) except asyncio.CancelledError: - logger.error("IR extraction cancelled.") + logger.error("stream extraction cancelled.") return -1 @@ -301,14 +307,8 @@ def main(argv): command = parsed_args.command if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) - elif EXTRACT_IR_CMD == command: - return handle_extract_cmd( - parsed_args, QueryJobType.EXTRACT_IR, clp_home, default_config_file_path - ) - elif EXTRACT_JSON_CMD == command: - return handle_extract_cmd( - parsed_args, QueryJobType.EXTRACT_JSON, clp_home, default_config_file_path - ) + elif command in [EXTRACT_IR_CMD, EXTRACT_JSON_CMD]: + return handle_extract_stream_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") return -1 From acb0237f571e404246fb6a8adc806b107553f5e0 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Mon, 4 Nov 2024 14:08:14 -0500 Subject: [PATCH 13/39] Renaming for task and scheduler --- .../executor/query/celeryconfig.py | 4 +- ...ract_ir_task.py => extract_stream_task.py} | 24 ++++----- .../scheduler/query/query_scheduler.py | 52 ++++++++++--------- 3 files changed, 42 insertions(+), 38 deletions(-) rename components/job-orchestration/job_orchestration/executor/query/{extract_ir_task.py => extract_stream_task.py} (89%) diff --git a/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py b/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py index 994c0bbcf..6cf97dbd0 100644 --- a/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py +++ b/components/job-orchestration/job_orchestration/executor/query/celeryconfig.py @@ -4,12 +4,12 @@ imports = ( "job_orchestration.executor.query.fs_search_task", - "job_orchestration.executor.query.extract_ir_task", + "job_orchestration.executor.query.extract_stream_task", ) task_routes = { "job_orchestration.executor.query.fs_search_task.search": QueueName.QUERY, - "job_orchestration.executor.query.extract_ir_task.extract_ir": QueueName.QUERY, + "job_orchestration.executor.query.extract_stream_task.extract_stream": QueueName.QUERY, } task_create_missing_queues = True diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py similarity index 89% rename from components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py rename to components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py index 4f01ed984..4a034683e 100644 --- a/components/job-orchestration/job_orchestration/executor/query/extract_ir_task.py +++ b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py @@ -25,10 +25,10 @@ def make_command( clp_home: Path, archives_dir: Path, archive_id: str, - ir_output_dir: Path, + stream_output_dir: Path, job_config_obj: dict, results_cache_uri: str, - ir_collection: str, + stream_collection: str, ) -> Optional[List[str]]: if StorageEngine.CLP == storage_engine: logger.info("Start IR extraction") @@ -41,9 +41,9 @@ def make_command( "i", str(archives_dir / archive_id), extract_ir_config.file_split_id, - str(ir_output_dir), + str(stream_output_dir), results_cache_uri, - ir_collection, + stream_collection, ] if extract_ir_config.target_uncompressed_size is not None: command.append("--target-size") @@ -55,14 +55,14 @@ def make_command( str(clp_home / "bin" / "clp-s"), "x", str(archives_dir), - str(ir_output_dir), + str(stream_output_dir), "--ordered", "--archive-id", archive_id, "--mongodb-uri", results_cache_uri, "--mongodb-collection", - ir_collection, + stream_collection, ] if extract_json_config.target_chunk_size is not None: command.append("--ordered-chunk-size") @@ -75,7 +75,7 @@ def make_command( @app.task(bind=True) -def extract_ir( +def extract_stream( self: Task, job_id: str, task_id: int, @@ -84,7 +84,7 @@ def extract_ir( clp_metadata_db_conn_params: dict, results_cache_uri: str, ) -> Dict[str, Any]: - task_name = "Extraction" + task_name = "Stream Extraction" # Setup logging to file clp_logs_dir = Path(os.getenv("CLP_LOGS_DIR")) @@ -101,18 +101,18 @@ def extract_ir( clp_home = Path(os.getenv("CLP_HOME")) archive_directory = Path(os.getenv("CLP_ARCHIVE_OUTPUT_DIR")) clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") - ir_output_dir = Path(os.getenv("CLP_IR_OUTPUT_DIR")) - ir_collection = os.getenv("CLP_IR_COLLECTION") + stream_output_dir = Path(os.getenv("CLP_IR_OUTPUT_DIR")) + stream_collection = os.getenv("CLP_IR_COLLECTION") task_command = make_command( storage_engine=clp_storage_engine, clp_home=clp_home, archives_dir=archive_directory, archive_id=archive_id, - ir_output_dir=ir_output_dir, + stream_output_dir=stream_output_dir, job_config_obj=job_config_obj, results_cache_uri=results_cache_uri, - ir_collection=ir_collection, + stream_collection=stream_collection, ) if not task_command: return report_command_creation_failure( diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 500ec2a04..7cd8bfb59 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -39,7 +39,7 @@ from clp_py_utils.core import read_yaml_config_file from clp_py_utils.decorators import exception_default_value from clp_py_utils.sql_adapter import SQL_Adapter -from job_orchestration.executor.query.extract_ir_task import extract_ir +from job_orchestration.executor.query.extract_stream_task import extract_stream from job_orchestration.executor.query.fs_search_task import search from job_orchestration.scheduler.constants import QueryJobStatus, QueryJobType, QueryTaskStatus from job_orchestration.scheduler.job_config import ( @@ -288,12 +288,12 @@ def get_archives_for_search( return archives_for_search -def get_archive_and_target_ids_for_extraction( +def get_archive_and_target_ids_for_stream_extraction( db_conn, job_config: Dict[str, Any], job_type: QueryJobType ) -> Tuple[Optional[str], Optional[str]]: if QueryJobType.EXTRACT_IR == job_type: extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) - return get_archive_and_file_split_ids_for_extraction(db_conn, extract_ir_config) + return get_archive_and_file_split_ids_for_ir_extraction(db_conn, extract_ir_config) extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) archive_id = extract_json_config.archive_id @@ -305,7 +305,7 @@ def get_archive_and_target_ids_for_extraction( return None, None -def get_archive_and_file_split_ids_for_extraction( +def get_archive_and_file_split_ids_for_ir_extraction( db_conn, extract_ir_config: ExtractIrJobConfig, ) -> Tuple[Optional[str], Optional[str]]: @@ -356,13 +356,13 @@ def get_archive_and_file_split_ids( return results -def is_target_extraction_active(target_id: str, job_type: QueryJobType): +def is_stream_extraction_target_active(target_id: str, job_type: QueryJobType) -> bool: if QueryJobType.EXTRACT_IR == job_type: return target_id in active_file_split_ir_extractions return target_id in active_archive_json_extractions -def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobType): +def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobType) -> None: global active_file_split_ir_extractions global active_archive_json_extractions @@ -379,7 +379,7 @@ def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobT def is_target_extracted( results_cache_uri: str, ir_collection_name: str, target_id: str, job_type: QueryJobType -): +) -> bool: target_key: str if QueryJobType.EXTRACT_IR == job_type: target_key = "file_split_id" @@ -392,15 +392,15 @@ def is_target_extracted( return 0 != results_count -def create_extraction_job( +def create_stream_extraction_job( job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType -): - new_extraction_job: QueryJob +) -> QueryJob: + new_stream_extraction_job: QueryJob new_job_state = InternalJobState.WAITING_FOR_DISPATCH if QueryJobType.EXTRACT_IR == job_type: extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) extract_ir_config.file_split_id = target_id - new_extraction_job = ExtractIrJob( + new_stream_extraction_job = ExtractIrJob( id=job_id, extract_ir_config=extract_ir_config, state=new_job_state, @@ -408,13 +408,13 @@ def create_extraction_job( logger.info(f"Created ir extraction job {job_id} on file_split: {target_id}") else: extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) - new_extraction_job = ExtractJsonJob( + new_stream_extraction_job = ExtractJsonJob( id=job_id, extract_json_config=extract_json_config, state=new_job_state, ) logger.info(f"Created json extraction job {job_id} on archive: {target_id}") - return new_extraction_job + return new_stream_extraction_job def check_if_archive_exists( @@ -457,7 +457,7 @@ def get_task_group_for_job( ) elif job_type in [QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR]: return celery.group( - extract_ir.s( + extract_stream.s( job_id=job.id, archive_id=archive_ids[i], task_id=task_ids[i], @@ -628,7 +628,7 @@ def handle_pending_query_jobs( active_jobs[job_id] = new_search_job elif job_type in [QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON]: - archive_id, target_id = get_archive_and_target_ids_for_extraction( + archive_id, target_id = get_archive_and_target_ids_for_stream_extraction( db_conn, job_config, job_type ) if not target_id: @@ -655,7 +655,7 @@ def handle_pending_query_jobs( # Check if the target is currently being extracted; if so, add the job ID to the # list of jobs waiting for it. - if is_target_extraction_active(target_id, job_type): + if is_stream_extraction_target_active(target_id, job_type): mark_job_waiting_for_target(target_id, job_id, job_type) logger.info( f"target {target_id} is being extracted, so mark job {job_id} as running" @@ -690,11 +690,13 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as succeeded") continue - nex_extraction_job = create_extraction_job(job_id, job_config, target_id, job_type) + nex_stream_extraction_job = create_stream_extraction_job( + job_id, job_config, target_id, job_type + ) dispatch_job_and_update_db( db_conn, - nex_extraction_job, + nex_stream_extraction_job, [archive_id], clp_metadata_db_conn_params, results_cache_uri, @@ -702,8 +704,8 @@ def handle_pending_query_jobs( ) mark_job_waiting_for_target(target_id, job_id, job_type) - active_jobs[job_id] = nex_extraction_job - logger.info(f"Dispatched extraction job {job_id} on archive: {archive_id}") + active_jobs[job_id] = nex_stream_extraction_job + logger.info(f"Dispatched stream extraction job {job_id} on archive: {archive_id}") else: # NOTE: We're skipping the job for this iteration, but its status will remain @@ -862,7 +864,9 @@ async def handle_finished_search_job( del active_jobs[job_id] -async def handle_finished_extraction_job(db_conn, job: QueryJob, task_results: List[Any]) -> None: +async def handle_finished_stream_extraction_job( + db_conn, job: QueryJob, task_results: List[Any] +) -> None: global active_jobs global active_archive_json_extractions global active_file_split_ir_extractions @@ -902,9 +906,9 @@ async def handle_finished_extraction_job(db_conn, job: QueryJob, task_results: L duration=(datetime.datetime.now() - job.start_time).total_seconds(), ): if new_job_status == QueryJobStatus.SUCCEEDED: - logger.info(f"Completed extraction job {job_id}.") + logger.info(f"Completed stream extraction job {job_id}.") else: - logger.info(f"Completed extraction job {job_id} with failing tasks.") + logger.info(f"Completed stream extraction job {job_id} with failing tasks.") waiting_jobs: List[str] if QueryJobType.EXTRACT_IR == job.get_type(): @@ -967,7 +971,7 @@ async def check_job_status_and_update_db(db_conn_pool, results_cache_uri): db_conn, search_job, returned_results, results_cache_uri ) elif job_type in [QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR]: - await handle_finished_extraction_job(db_conn, job, returned_results) + await handle_finished_stream_extraction_job(db_conn, job, returned_results) else: logger.error(f"Unexpected job type: {job_type}, skipping job {job_id}") From 5f12b383ebb616f7bce1d9ccf5aa771c35017a22 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Mon, 4 Nov 2024 15:04:31 -0500 Subject: [PATCH 14/39] Mass renaming --- .../clp_package_utils/general.py | 16 ++++++------ .../clp_package_utils/scripts/start_clp.py | 16 ++++++------ .../clp-py-utils/clp_py_utils/clp_config.py | 26 ++++++++++--------- .../create-results-cache-indices.py | 10 ++++--- .../executor/query/extract_stream_task.py | 4 +-- .../scheduler/query/query_scheduler.py | 16 +++++++----- .../package-template/src/etc/clp-config.yml | 6 ++--- 7 files changed, 50 insertions(+), 44 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/general.py b/components/clp-package-utils/clp_package_utils/general.py index 620cc717b..6e827c1e5 100644 --- a/components/clp-package-utils/clp_package_utils/general.py +++ b/components/clp-package-utils/clp_package_utils/general.py @@ -85,7 +85,7 @@ def __init__(self, clp_home: pathlib.Path, docker_clp_home: pathlib.Path): self.data_dir: typing.Optional[DockerMount] = None self.logs_dir: typing.Optional[DockerMount] = None self.archives_output_dir: typing.Optional[DockerMount] = None - self.ir_output_dir: typing.Optional[DockerMount] = None + self.stream_output_dir: typing.Optional[DockerMount] = None def get_clp_home(): @@ -252,17 +252,17 @@ def generate_container_config( container_clp_config.archive_output.directory, ) - container_clp_config.ir_output.directory = pathlib.Path("/") / "mnt" / "ir-output" + container_clp_config.stream_output.directory = pathlib.Path("/") / "mnt" / "ir-output" if not is_path_already_mounted( clp_home, CONTAINER_CLP_HOME, - clp_config.ir_output.directory, - container_clp_config.ir_output.directory, + clp_config.stream_output.directory, + container_clp_config.stream_output.directory, ): - docker_mounts.ir_output_dir = DockerMount( + docker_mounts.stream_output_dir = DockerMount( DockerMountType.BIND, - clp_config.ir_output.directory, - container_clp_config.ir_output.directory, + clp_config.stream_output.directory, + container_clp_config.stream_output.directory, ) return container_clp_config, docker_mounts @@ -483,7 +483,7 @@ def validate_results_cache_config( def validate_worker_config(clp_config: CLPConfig): clp_config.validate_input_logs_dir() clp_config.validate_archive_output_dir() - clp_config.validate_ir_output_dir() + clp_config.validate_stream_output_dir() def validate_webui_config( diff --git a/components/clp-package-utils/clp_package_utils/scripts/start_clp.py b/components/clp-package-utils/clp_package_utils/scripts/start_clp.py index 7c6de0200..2422aa31d 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/start_clp.py +++ b/components/clp-package-utils/clp_package_utils/scripts/start_clp.py @@ -286,7 +286,7 @@ def create_results_cache_indices( "python3", str(clp_py_utils_dir / "create-results-cache-indices.py"), "--uri", container_clp_config.results_cache.get_uri(), - "--ir-collection", container_clp_config.results_cache.ir_collection_name, + "--stream-collection", container_clp_config.results_cache.stream_collection_name, ] # fmt: on @@ -660,10 +660,10 @@ def start_query_worker( celery_method = "job_orchestration.executor.query" celery_route = f"{QueueName.QUERY}" - query_worker_mount = [mounts.ir_output_dir] + query_worker_mount = [mounts.stream_output_dir] query_worker_env = { - "CLP_IR_OUTPUT_DIR": container_clp_config.ir_output.directory, - "CLP_IR_COLLECTION": clp_config.results_cache.ir_collection_name, + "CLP_STREAM_OUTPUT_DIR": container_clp_config.stream_output.directory, + "CLP_STREAM_COLLECTION": clp_config.results_cache.stream_collection_name, } generic_start_worker( @@ -710,7 +710,7 @@ def generic_start_worker( # Create necessary directories clp_config.archive_output.directory.mkdir(parents=True, exist_ok=True) - clp_config.ir_output.directory.mkdir(parents=True, exist_ok=True) + clp_config.stream_output.directory.mkdir(parents=True, exist_ok=True) clp_site_packages_dir = CONTAINER_CLP_HOME / "lib" / "python3" / "site-packages" # fmt: off @@ -933,9 +933,9 @@ def start_log_viewer_webui( "MongoDbHost": clp_config.results_cache.host, "MongoDbPort": clp_config.results_cache.port, "MongoDbName": clp_config.results_cache.db_name, - "MongoDbIrFilesCollectionName": clp_config.results_cache.ir_collection_name, + "MongoDbIrFilesCollectionName": clp_config.results_cache.stream_collection_name, "ClientDir": str(container_log_viewer_webui_dir / "client"), - "IrFilesDir": str(container_clp_config.ir_output.directory), + "IrFilesDir": str(container_clp_config.stream_output.directory), "LogViewerDir": str(container_log_viewer_webui_dir / "yscope-log-viewer"), } settings_json = read_and_update_settings_json(settings_json_path, settings_json_updates) @@ -961,7 +961,7 @@ def start_log_viewer_webui( # fmt: on necessary_mounts = [ mounts.clp_home, - mounts.ir_output_dir, + mounts.stream_output_dir, ] for mount in necessary_mounts: if mount: diff --git a/components/clp-py-utils/clp_py_utils/clp_config.py b/components/clp-py-utils/clp_py_utils/clp_config.py index f5a813057..4bb00bd9e 100644 --- a/components/clp-py-utils/clp_py_utils/clp_config.py +++ b/components/clp-py-utils/clp_py_utils/clp_config.py @@ -270,7 +270,7 @@ class ResultsCache(BaseModel): host: str = "localhost" port: int = 27017 db_name: str = "clp-query-results" - ir_collection_name: str = "ir-files" + stream_collection_name: str = "stream-files" @validator("host") def validate_host(cls, field): @@ -284,10 +284,12 @@ def validate_db_name(cls, field): raise ValueError(f"{RESULTS_CACHE_COMPONENT_NAME}.db_name cannot be empty.") return field - @validator("ir_collection_name") - def validate_ir_collection_name(cls, field): + @validator("stream_collection_name") + def validate_stream_collection_name(cls, field): if "" == field: - raise ValueError(f"{RESULTS_CACHE_COMPONENT_NAME}.ir_collection_name cannot be empty.") + raise ValueError( + f"{RESULTS_CACHE_COMPONENT_NAME}.stream_collection_name cannot be empty." + ) return field def get_uri(self): @@ -343,8 +345,8 @@ def dump_to_primitive_dict(self): return d -class IrOutput(BaseModel): - directory: pathlib.Path = pathlib.Path("var") / "data" / "ir" +class StreamOutput(BaseModel): + directory: pathlib.Path = pathlib.Path("var") / "data" / "stream" target_uncompressed_size: int = 128 * 1024 * 1024 @validator("directory") @@ -425,7 +427,7 @@ class CLPConfig(BaseModel): credentials_file_path: pathlib.Path = CLP_DEFAULT_CREDENTIALS_FILE_PATH archive_output: ArchiveOutput = ArchiveOutput() - ir_output: IrOutput = IrOutput() + stream_output: StreamOutput = StreamOutput() data_directory: pathlib.Path = pathlib.Path("var") / "data" logs_directory: pathlib.Path = pathlib.Path("var") / "log" @@ -435,7 +437,7 @@ def make_config_paths_absolute(self, clp_home: pathlib.Path): self.input_logs_directory = make_config_path_absolute(clp_home, self.input_logs_directory) self.credentials_file_path = make_config_path_absolute(clp_home, self.credentials_file_path) self.archive_output.make_config_paths_absolute(clp_home) - self.ir_output.make_config_paths_absolute(clp_home) + self.stream_output.make_config_paths_absolute(clp_home) self.data_directory = make_config_path_absolute(clp_home, self.data_directory) self.logs_directory = make_config_path_absolute(clp_home, self.logs_directory) self._os_release_file_path = make_config_path_absolute(clp_home, self._os_release_file_path) @@ -455,11 +457,11 @@ def validate_archive_output_dir(self): except ValueError as ex: raise ValueError(f"archive_output.directory is invalid: {ex}") - def validate_ir_output_dir(self): + def validate_stream_output_dir(self): try: - validate_path_could_be_dir(self.ir_output.directory) + validate_path_could_be_dir(self.stream_output.directory) except ValueError as ex: - raise ValueError(f"ir_output.directory is invalid: {ex}") + raise ValueError(f"stream_output.directory is invalid: {ex}") def validate_data_dir(self): try: @@ -528,7 +530,7 @@ def load_redis_credentials_from_file(self): def dump_to_primitive_dict(self): d = self.dict() d["archive_output"] = self.archive_output.dump_to_primitive_dict() - d["ir_output"] = self.ir_output.dump_to_primitive_dict() + d["stream_output"] = self.stream_output.dump_to_primitive_dict() # Turn paths into primitive strings d["input_logs_directory"] = str(self.input_logs_directory) d["credentials_file_path"] = str(self.credentials_file_path) diff --git a/components/clp-py-utils/clp_py_utils/create-results-cache-indices.py b/components/clp-py-utils/clp_py_utils/create-results-cache-indices.py index dafbd3bde..db03e9632 100644 --- a/components/clp-py-utils/clp_py_utils/create-results-cache-indices.py +++ b/components/clp-py-utils/clp_py_utils/create-results-cache-indices.py @@ -18,19 +18,21 @@ def main(argv): args_parser = argparse.ArgumentParser(description="Creates results cache indices for CLP.") args_parser.add_argument("--uri", required=True, help="URI of the results cache.") - args_parser.add_argument("--ir-collection", required=True, help="Collection for IR metadata.") + args_parser.add_argument( + "--stream-collection", required=True, help="Collection for stream metadata." + ) parsed_args = args_parser.parse_args(argv[1:]) results_cache_uri = parsed_args.uri - ir_collection_name = parsed_args.ir_collection + stream_collection_name = parsed_args.stream_collection try: with MongoClient(results_cache_uri) as results_cache_client: - ir_collection = results_cache_client.get_default_database()[ir_collection_name] + stream_collection = results_cache_client.get_default_database()[stream_collection_name] file_split_id_index = IndexModel(["file_split_id"]) orig_file_id_index = IndexModel(["orig_file_id", "begin_msg_ix", "end_msg_ix"]) - ir_collection.create_indexes([file_split_id_index, orig_file_id_index]) + stream_collection.create_indexes([file_split_id_index, orig_file_id_index]) except Exception: logger.exception("Failed to create clp results cache indices.") return -1 diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py index 4a034683e..46d88dabd 100644 --- a/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py +++ b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py @@ -101,8 +101,8 @@ def extract_stream( clp_home = Path(os.getenv("CLP_HOME")) archive_directory = Path(os.getenv("CLP_ARCHIVE_OUTPUT_DIR")) clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") - stream_output_dir = Path(os.getenv("CLP_IR_OUTPUT_DIR")) - stream_collection = os.getenv("CLP_IR_COLLECTION") + stream_output_dir = Path(os.getenv("CLP_STREAM_OUTPUT_DIR")) + stream_collection = os.getenv("CLP_STREAM_COLLECTION") task_command = make_command( storage_engine=clp_storage_engine, diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 7cd8bfb59..985ef65b9 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -378,7 +378,7 @@ def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobT def is_target_extracted( - results_cache_uri: str, ir_collection_name: str, target_id: str, job_type: QueryJobType + results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType ) -> bool: target_key: str if QueryJobType.EXTRACT_IR == job_type: @@ -387,7 +387,7 @@ def is_target_extracted( target_key = "orig_file_id" with pymongo.MongoClient(results_cache_uri) as results_cache_client: - ir_collection = results_cache_client.get_default_database()[ir_collection_name] + ir_collection = results_cache_client.get_default_database()[stream_collection_name] results_count = ir_collection.count_documents({target_key: target_id}) return 0 != results_count @@ -567,7 +567,7 @@ def handle_pending_query_jobs( db_conn_pool, clp_metadata_db_conn_params: Dict[str, any], results_cache_uri: str, - ir_collection_name: str, + stream_collection_name: str, num_archives_to_search_per_sub_job: int, ) -> List[asyncio.Task]: global active_jobs @@ -673,7 +673,9 @@ def handle_pending_query_jobs( continue # Check if the target has already been extracted - if is_target_extracted(results_cache_uri, ir_collection_name, target_id, job_type): + if is_target_extracted( + results_cache_uri, stream_collection_name, target_id, job_type + ): logger.info( f"target {target_id} already extracted, so mark job {job_id} as done" ) @@ -987,7 +989,7 @@ async def handle_jobs( db_conn_pool, clp_metadata_db_conn_params: Dict[str, any], results_cache_uri: str, - ir_collection_name: str, + stream_collection_name: str, jobs_poll_delay: float, num_archives_to_search_per_sub_job: int, ) -> None: @@ -1001,7 +1003,7 @@ async def handle_jobs( db_conn_pool, clp_metadata_db_conn_params, results_cache_uri, - ir_collection_name, + stream_collection_name, num_archives_to_search_per_sub_job, ) if 0 == len(reducer_acquisition_tasks): @@ -1085,7 +1087,7 @@ async def main(argv: List[str]) -> int: True ), results_cache_uri=clp_config.results_cache.get_uri(), - ir_collection_name=clp_config.results_cache.ir_collection_name, + stream_collection_name=clp_config.results_cache.stream_collection_name, jobs_poll_delay=clp_config.query_scheduler.jobs_poll_delay, num_archives_to_search_per_sub_job=batch_size, ) diff --git a/components/package-template/src/etc/clp-config.yml b/components/package-template/src/etc/clp-config.yml index cb66f40cd..875fdb11e 100644 --- a/components/package-template/src/etc/clp-config.yml +++ b/components/package-template/src/etc/clp-config.yml @@ -47,7 +47,7 @@ # host: "localhost" # port: 27017 # db_name: "clp-query-results" -# ir_collection_name: "ir-files" +# stream_collection_name: "stream-files" # #compression_worker: # logging_level: "INFO" @@ -83,8 +83,8 @@ # target_segment_size: 268435456 # 256 MB # ## Where CLP IR files should be output -#ir_output: -# directory: "var/data/ir" +#stream_output: +# directory: "var/data/stream" # # # How large each IR file should be before being split into a new IR file # target_uncompressed_size: 134217728 # 128 MB From 3c23fd2cabc8fbfed95d674e555fde4eb864f728 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Tue, 5 Nov 2024 11:49:57 -0500 Subject: [PATCH 15/39] Renaming for webui --- .../clp_package_utils/scripts/start_clp.py | 4 +- .../log-viewer-webui/server/settings.json | 4 +- .../log-viewer-webui/server/src/DbManager.js | 10 +- components/log-viewer-webui/server/src/app.js | 2 +- .../server/src/routes/static.js | 112 +++++++++--------- 5 files changed, 66 insertions(+), 66 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/start_clp.py b/components/clp-package-utils/clp_package_utils/scripts/start_clp.py index 2422aa31d..aea1ec19e 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/start_clp.py +++ b/components/clp-package-utils/clp_package_utils/scripts/start_clp.py @@ -933,9 +933,9 @@ def start_log_viewer_webui( "MongoDbHost": clp_config.results_cache.host, "MongoDbPort": clp_config.results_cache.port, "MongoDbName": clp_config.results_cache.db_name, - "MongoDbIrFilesCollectionName": clp_config.results_cache.stream_collection_name, + "MongoDbStreamFilesCollectionName": clp_config.results_cache.stream_collection_name, "ClientDir": str(container_log_viewer_webui_dir / "client"), - "IrFilesDir": str(container_clp_config.stream_output.directory), + "StreamFilesDir": str(container_clp_config.stream_output.directory), "LogViewerDir": str(container_log_viewer_webui_dir / "yscope-log-viewer"), } settings_json = read_and_update_settings_json(settings_json_path, settings_json_updates) diff --git a/components/log-viewer-webui/server/settings.json b/components/log-viewer-webui/server/settings.json index bb1aac48a..a5f7cb1e7 100644 --- a/components/log-viewer-webui/server/settings.json +++ b/components/log-viewer-webui/server/settings.json @@ -6,9 +6,9 @@ "MongoDbHost": "localhost", "MongoDbPort": 27017, "MongoDbName": "clp-query-results", - "MongoDbIrFilesCollectionName": "ir-files", + "MongoDbStreamFilesCollectionName": "stream-files", "ClientDir": "../client/dist", - "IrFilesDir": "../../../build/clp-package/var/data/ir", + "StreamFilesDir": "../../../build/clp-package/var/data/stream", "LogViewerDir": "../yscope-log-viewer/dist" } diff --git a/components/log-viewer-webui/server/src/DbManager.js b/components/log-viewer-webui/server/src/DbManager.js index 5cb442795..1da06dd34 100644 --- a/components/log-viewer-webui/server/src/DbManager.js +++ b/components/log-viewer-webui/server/src/DbManager.js @@ -84,7 +84,7 @@ class DbManager { /** * @type {import("mongodb").Collection} */ - #irFilesCollection; + #StreamFilesCollection; #queryJobsTableName; @@ -138,7 +138,7 @@ class DbManager { * @return {Promise} A promise that resolves to the extracted IR file's metadata. */ async getExtractedIrFileMetadata (origFileId, logEventIdx) { - return await this.#irFilesCollection.findOne({ + return await this.#StreamFilesCollection.findOne({ orig_file_id: origFileId, begin_msg_ix: {$lte: logEventIdx}, end_msg_ix: {$gt: logEventIdx}, @@ -177,7 +177,7 @@ class DbManager { * @param {string} config.host * @param {number} config.port * @param {string} config.database - * @param {string} config.irFilesCollectionName + * @param {string} config.StreamFilesCollectionName */ #initMongo (config) { this.#fastify.register(fastifyMongo, { @@ -187,8 +187,8 @@ class DbManager { if (err) { throw err; } - this.#irFilesCollection = - this.#fastify.mongo.db.collection(config.irFilesCollectionName); + this.#StreamFilesCollection = + this.#fastify.mongo.db.collection(config.StreamFilesCollectionName); }); } diff --git a/components/log-viewer-webui/server/src/app.js b/components/log-viewer-webui/server/src/app.js index 69918ea58..ce81ecfd8 100644 --- a/components/log-viewer-webui/server/src/app.js +++ b/components/log-viewer-webui/server/src/app.js @@ -37,7 +37,7 @@ const app = async ({ mongoConfig: { database: settings.MongoDbName, host: settings.MongoDbHost, - irFilesCollectionName: settings.MongoDbIrFilesCollectionName, + StreamFilesCollectionName: settings.MongoDbStreamFilesCollectionName, port: settings.MongoDbPort, }, }); diff --git a/components/log-viewer-webui/server/src/routes/static.js b/components/log-viewer-webui/server/src/routes/static.js index 42d9048f0..3971dd0ae 100644 --- a/components/log-viewer-webui/server/src/routes/static.js +++ b/components/log-viewer-webui/server/src/routes/static.js @@ -1,56 +1,56 @@ -import path from "node:path"; -import process from "node:process"; -import {fileURLToPath} from "node:url"; - -import {fastifyStatic} from "@fastify/static"; - -import settings from "../../settings.json" with {type: "json"}; - - -/** - * Creates static files serving routes. - * - * @param {import("fastify").FastifyInstance} fastify - * @param {import("fastify").FastifyPluginOptions} options - */ -const routes = async (fastify, options) => { - const filename = fileURLToPath(import.meta.url); - const dirname = path.dirname(filename); - const rootDirname = path.resolve(dirname, "../.."); - - let irFilesDir = settings.IrFilesDir; - if (false === path.isAbsolute(irFilesDir)) { - irFilesDir = path.resolve(rootDirname, irFilesDir); - } - await fastify.register(fastifyStatic, { - prefix: "/ir", - root: irFilesDir, - }); - - let logViewerDir = settings.LogViewerDir; - if (false === path.isAbsolute(logViewerDir)) { - logViewerDir = path.resolve(rootDirname, logViewerDir); - } - await fastify.register(fastifyStatic, { - prefix: "/log-viewer", - root: logViewerDir, - decorateReply: false, - }); - - if ("production" === process.env.NODE_ENV) { - // In the development environment, we expect the client to use a separate webserver that - // supports live reloading. - let clientDir = settings.ClientDir; - if (false === path.isAbsolute(clientDir)) { - clientDir = path.resolve(rootDirname, settings.ClientDir); - } - - await fastify.register(fastifyStatic, { - prefix: "/", - root: clientDir, - decorateReply: false, - }); - } -}; - -export default routes; +import path from "node:path"; +import process from "node:process"; +import {fileURLToPath} from "node:url"; + +import {fastifyStatic} from "@fastify/static"; + +import settings from "../../settings.json" with {type: "json"}; + + +/** + * Creates static files serving routes. + * + * @param {import("fastify").FastifyInstance} fastify + * @param {import("fastify").FastifyPluginOptions} options + */ +const routes = async (fastify, options) => { + const filename = fileURLToPath(import.meta.url); + const dirname = path.dirname(filename); + const rootDirname = path.resolve(dirname, "../.."); + + let StreamFilesDir = settings.StreamFilesDir; + if (false === path.isAbsolute(StreamFilesDir)) { + StreamFilesDir = path.resolve(rootDirname, StreamFilesDir); + } + await fastify.register(fastifyStatic, { + prefix: "/ir", + root: StreamFilesDir, + }); + + let logViewerDir = settings.LogViewerDir; + if (false === path.isAbsolute(logViewerDir)) { + logViewerDir = path.resolve(rootDirname, logViewerDir); + } + await fastify.register(fastifyStatic, { + prefix: "/log-viewer", + root: logViewerDir, + decorateReply: false, + }); + + if ("production" === process.env.NODE_ENV) { + // In the development environment, we expect the client to use a separate webserver that + // supports live reloading. + let clientDir = settings.ClientDir; + if (false === path.isAbsolute(clientDir)) { + clientDir = path.resolve(rootDirname, settings.ClientDir); + } + + await fastify.register(fastifyStatic, { + prefix: "/", + root: clientDir, + decorateReply: false, + }); + } +}; + +export default routes; From be75c9c89f66facc1df9bff7d2c5fcee32f0a4e4 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Tue, 5 Nov 2024 11:56:20 -0500 Subject: [PATCH 16/39] missing change --- components/clp-package-utils/clp_package_utils/general.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/components/clp-package-utils/clp_package_utils/general.py b/components/clp-package-utils/clp_package_utils/general.py index 6e827c1e5..f42542ebc 100644 --- a/components/clp-package-utils/clp_package_utils/general.py +++ b/components/clp-package-utils/clp_package_utils/general.py @@ -252,7 +252,7 @@ def generate_container_config( container_clp_config.archive_output.directory, ) - container_clp_config.stream_output.directory = pathlib.Path("/") / "mnt" / "ir-output" + container_clp_config.stream_output.directory = pathlib.Path("/") / "mnt" / "stream-output" if not is_path_already_mounted( clp_home, CONTAINER_CLP_HOME, From 383781a66ee95aebafa4abfb8057c1b7cf8a743b Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Tue, 5 Nov 2024 12:57:42 -0500 Subject: [PATCH 17/39] polishing --- .../scripts/native/decompress.py | 2 +- .../scheduler/query/query_scheduler.py | 4 +- .../server/src/routes/static.js | 112 +++++++++--------- 3 files changed, 59 insertions(+), 59 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index 997060ed5..a2f862fd2 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -81,7 +81,7 @@ def submit_and_monitor_extraction_job_in_db( job_config: QueryJobConfig, ) -> int: """ - Submits an extraction job to the scheduler and waits until the job finishes. + Submits a stream extraction job to the scheduler and waits until it finishes. :param db_config: :param job_type: :param job_config: diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 985ef65b9..6b4715970 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -424,10 +424,10 @@ def check_if_archive_exists( query = f"""SELECT 1 FROM {CLP_METADATA_TABLE_PREFIX}archives WHERE - id = '{archive_id}' + id = %s """ with contextlib.closing(db_conn.cursor(dictionary=True)) as cursor: - cursor.execute(query) + cursor.execute(query, (archive_id,)) if cursor.fetchone(): return True diff --git a/components/log-viewer-webui/server/src/routes/static.js b/components/log-viewer-webui/server/src/routes/static.js index 3971dd0ae..25e0870f1 100644 --- a/components/log-viewer-webui/server/src/routes/static.js +++ b/components/log-viewer-webui/server/src/routes/static.js @@ -1,56 +1,56 @@ -import path from "node:path"; -import process from "node:process"; -import {fileURLToPath} from "node:url"; - -import {fastifyStatic} from "@fastify/static"; - -import settings from "../../settings.json" with {type: "json"}; - - -/** - * Creates static files serving routes. - * - * @param {import("fastify").FastifyInstance} fastify - * @param {import("fastify").FastifyPluginOptions} options - */ -const routes = async (fastify, options) => { - const filename = fileURLToPath(import.meta.url); - const dirname = path.dirname(filename); - const rootDirname = path.resolve(dirname, "../.."); - - let StreamFilesDir = settings.StreamFilesDir; - if (false === path.isAbsolute(StreamFilesDir)) { - StreamFilesDir = path.resolve(rootDirname, StreamFilesDir); - } - await fastify.register(fastifyStatic, { - prefix: "/ir", - root: StreamFilesDir, - }); - - let logViewerDir = settings.LogViewerDir; - if (false === path.isAbsolute(logViewerDir)) { - logViewerDir = path.resolve(rootDirname, logViewerDir); - } - await fastify.register(fastifyStatic, { - prefix: "/log-viewer", - root: logViewerDir, - decorateReply: false, - }); - - if ("production" === process.env.NODE_ENV) { - // In the development environment, we expect the client to use a separate webserver that - // supports live reloading. - let clientDir = settings.ClientDir; - if (false === path.isAbsolute(clientDir)) { - clientDir = path.resolve(rootDirname, settings.ClientDir); - } - - await fastify.register(fastifyStatic, { - prefix: "/", - root: clientDir, - decorateReply: false, - }); - } -}; - -export default routes; +import path from "node:path"; +import process from "node:process"; +import {fileURLToPath} from "node:url"; + +import {fastifyStatic} from "@fastify/static"; + +import settings from "../../settings.json" with {type: "json"}; + + +/** + * Creates static files serving routes. + * + * @param {import("fastify").FastifyInstance} fastify + * @param {import("fastify").FastifyPluginOptions} options + */ +const routes = async (fastify, options) => { + const filename = fileURLToPath(import.meta.url); + const dirname = path.dirname(filename); + const rootDirname = path.resolve(dirname, "../.."); + + let {StreamFilesDir} = settings; + if (false === path.isAbsolute(StreamFilesDir)) { + StreamFilesDir = path.resolve(rootDirname, StreamFilesDir); + } + await fastify.register(fastifyStatic, { + prefix: "/ir", + root: StreamFilesDir, + }); + + let logViewerDir = settings.LogViewerDir; + if (false === path.isAbsolute(logViewerDir)) { + logViewerDir = path.resolve(rootDirname, logViewerDir); + } + await fastify.register(fastifyStatic, { + prefix: "/log-viewer", + root: logViewerDir, + decorateReply: false, + }); + + if ("production" === process.env.NODE_ENV) { + // In the development environment, we expect the client to use a separate webserver that + // supports live reloading. + let clientDir = settings.ClientDir; + if (false === path.isAbsolute(clientDir)) { + clientDir = path.resolve(rootDirname, settings.ClientDir); + } + + await fastify.register(fastifyStatic, { + prefix: "/", + root: clientDir, + decorateReply: false, + }); + } +}; + +export default routes; From 9518074799bf816f02800424f8cd014b60fbde54 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Tue, 5 Nov 2024 16:14:43 -0500 Subject: [PATCH 18/39] Update comments --- .../scheduler/query/query_scheduler.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 6b4715970..9ac11d647 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -645,13 +645,11 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as failed") continue - # TODO: update this comment - # NOTE: The following two if blocks should not be reordered since if we first check - # whether *an* IR file has been extracted for the requested file split, it doesn't - # mean that *all* IR files have has been extracted for the file split (since the - # extraction job may still be in progress). Thus, we must first check whether the - # file split is in the process of being extracted, and then check whether it's - # already been extracted. + # NOTE: The following two if blocks should not be reordered. The method that checks + # whether a stream file has been extracted doesn't guarantee that *all* stream + # files to be extracted for the target (since the extraction job may still be in + # progress). Thus, we must first check whether the target is in the process of + # being extracted, and then check whether it's already been extracted. # Check if the target is currently being extracted; if so, add the job ID to the # list of jobs waiting for it. @@ -672,7 +670,7 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as running") continue - # Check if the target has already been extracted + # Check if a stream file in the target has already been extracted if is_target_extracted( results_cache_uri, stream_collection_name, target_id, job_type ): From ebe6395833e2bf7c506982a9868bceae1cf7119f Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Tue, 5 Nov 2024 17:03:25 -0500 Subject: [PATCH 19/39] fixes --- .../job_orchestration/scheduler/query/query_scheduler.py | 4 ++-- components/log-viewer-webui/server/src/routes/static.js | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 9ac11d647..e3b8ec733 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -387,8 +387,8 @@ def is_target_extracted( target_key = "orig_file_id" with pymongo.MongoClient(results_cache_uri) as results_cache_client: - ir_collection = results_cache_client.get_default_database()[stream_collection_name] - results_count = ir_collection.count_documents({target_key: target_id}) + stream_collection = results_cache_client.get_default_database()[stream_collection_name] + results_count = stream_collection.count_documents({target_key: target_id}) return 0 != results_count diff --git a/components/log-viewer-webui/server/src/routes/static.js b/components/log-viewer-webui/server/src/routes/static.js index 25e0870f1..33715eb57 100644 --- a/components/log-viewer-webui/server/src/routes/static.js +++ b/components/log-viewer-webui/server/src/routes/static.js @@ -18,13 +18,13 @@ const routes = async (fastify, options) => { const dirname = path.dirname(filename); const rootDirname = path.resolve(dirname, "../.."); - let {StreamFilesDir} = settings; - if (false === path.isAbsolute(StreamFilesDir)) { - StreamFilesDir = path.resolve(rootDirname, StreamFilesDir); + let {streamFilesDir} = settings; + if (false === path.isAbsolute(streamFilesDir)) { + streamFilesDir = path.resolve(rootDirname, streamFilesDir); } await fastify.register(fastifyStatic, { prefix: "/ir", - root: StreamFilesDir, + root: streamFilesDir, }); let logViewerDir = settings.LogViewerDir; From 7b79dffcc484adff1789beaec203045f30cc75d1 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Thu, 7 Nov 2024 15:00:23 -0500 Subject: [PATCH 20/39] fixes --- components/log-viewer-webui/server/src/routes/static.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/components/log-viewer-webui/server/src/routes/static.js b/components/log-viewer-webui/server/src/routes/static.js index 33715eb57..194f8decd 100644 --- a/components/log-viewer-webui/server/src/routes/static.js +++ b/components/log-viewer-webui/server/src/routes/static.js @@ -18,7 +18,7 @@ const routes = async (fastify, options) => { const dirname = path.dirname(filename); const rootDirname = path.resolve(dirname, "../.."); - let {streamFilesDir} = settings; + let streamFilesDir = settings.StreamFilesDir; if (false === path.isAbsolute(streamFilesDir)) { streamFilesDir = path.resolve(rootDirname, streamFilesDir); } From 7498c69e6b4b1a148d14f2a48968cb19d5caa6d6 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Thu, 7 Nov 2024 17:20:58 -0500 Subject: [PATCH 21/39] fixes2 --- components/log-viewer-webui/server/.env | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/components/log-viewer-webui/server/.env b/components/log-viewer-webui/server/.env index b66dc997b..dd88b78ed 100644 --- a/components/log-viewer-webui/server/.env +++ b/components/log-viewer-webui/server/.env @@ -1,5 +1,5 @@ CLIENT_DIR=../client/dist -IR_DATA_DIR=../../../build/clp-package/var/data/ir +STREAM_DATA_DIR=../../../build/clp-package/var/data/stream LOG_VIEWER_DIR=../yscope-log-viewer/dist HOST=localhost From cc9b5c222406721846362901baf00afe4fcb2051 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 8 Nov 2024 14:36:53 -0500 Subject: [PATCH 22/39] update webui part --- components/log-viewer-webui/client/src/ui/QueryStatus.jsx | 2 +- components/log-viewer-webui/server/src/routes/static.js | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/components/log-viewer-webui/client/src/ui/QueryStatus.jsx b/components/log-viewer-webui/client/src/ui/QueryStatus.jsx index c1ad9eb9e..7f9bc91cf 100644 --- a/components/log-viewer-webui/client/src/ui/QueryStatus.jsx +++ b/components/log-viewer-webui/client/src/ui/QueryStatus.jsx @@ -51,7 +51,7 @@ const QueryStatus = () => { setQueryState(QUERY_LOADING_STATES.LOADING); const innerLogEventNum = logEventIdx - data.begin_msg_ix + 1; - window.location = `/log-viewer/index.html?filePath=/ir/${data.path}` + + window.location = `/log-viewer/index.html?filePath=/stream/${data.path}` + `#logEventNum=${innerLogEventNum}`; }) .catch((e) => { diff --git a/components/log-viewer-webui/server/src/routes/static.js b/components/log-viewer-webui/server/src/routes/static.js index 194f8decd..fbf91cff9 100644 --- a/components/log-viewer-webui/server/src/routes/static.js +++ b/components/log-viewer-webui/server/src/routes/static.js @@ -23,7 +23,7 @@ const routes = async (fastify, options) => { streamFilesDir = path.resolve(rootDirname, streamFilesDir); } await fastify.register(fastifyStatic, { - prefix: "/ir", + prefix: "/stream", root: streamFilesDir, }); From 3c30054d264b8f708115a148b278b5591a632f1a Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 14:10:53 -0500 Subject: [PATCH 23/39] Apply suggestions from code review Co-authored-by: kirkrodrigues <2454684+kirkrodrigues@users.noreply.github.com> --- .../clp_package_utils/scripts/decompress.py | 7 ++++--- .../clp_package_utils/scripts/native/decompress.py | 12 ++++++------ .../executor/query/extract_stream_task.py | 4 ++-- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/decompress.py index ec5280e41..903107c32 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/decompress.py @@ -201,7 +201,8 @@ def handle_extract_stream_cmd( extract_cmd.append("--target-chunk-size") extract_cmd.append(str(parsed_args.target_chunk_size)) else: - logger.exception(f"Unexpected command: {job_command}") + logger.error(f"Unexpected command: {job_command}") + return -1 cmd = container_start_cmd + extract_cmd @@ -253,7 +254,7 @@ def main(argv): group.add_argument("--orig-file-id", type=str, help="Original file's ID.") group.add_argument("--orig-file-path", type=str, help="Original file's path.") - # Json extraction command parser + # JSON extraction command parser json_extraction_parser = command_args_parser.add_parser(EXTRACT_JSON_CMD) json_extraction_parser.add_argument("archive_id", type=str, help="Archive ID") json_extraction_parser.add_argument( @@ -265,7 +266,7 @@ def main(argv): command = parsed_args.command if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) - elif command in [EXTRACT_IR_CMD, EXTRACT_JSON_CMD]: + elif command in (EXTRACT_IR_CMD, EXTRACT_JSON_CMD): return handle_extract_stream_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index a2f862fd2..e455894d2 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -129,9 +129,9 @@ def handle_extract_stream_cmd( orig_file_id = parsed_args.orig_file_id else: orig_file_path = parsed_args.orig_file_path - orig_file_id = get_orig_file_id(clp_config.database, parsed_args.orig_file_path) + orig_file_id = get_orig_file_id(clp_config.database, orig_file_path) if orig_file_id is None: - logger.error(f"Cannot find orig_file_id corresponding to {orig_file_path}") + logger.error(f"Cannot find orig_file_id corresponding to '{orig_file_path}'.") return -1 extraction_config = ExtractIrJobConfig( orig_file_id=orig_file_id, @@ -144,7 +144,7 @@ def handle_extract_stream_cmd( archive_id=parsed_args.archive_id, target_chunk_size=parsed_args.target_chunk_size ) else: - logger.exception(f"Unsupported stream extraction command: {command}") + logger.error(f"Unsupported stream extraction command: {command}") return -1 try: @@ -157,7 +157,7 @@ def handle_extract_stream_cmd( ) ) except asyncio.CancelledError: - logger.error("stream extraction cancelled.") + logger.error("Stream extraction cancelled.") return -1 @@ -295,7 +295,7 @@ def main(argv): group.add_argument("--orig-file-id", type=str, help="Original file's ID.") group.add_argument("--orig-file-path", type=str, help="Original file's path.") - # Json extraction command parser + # JSON extraction command parser json_extraction_parser = command_args_parser.add_parser(EXTRACT_JSON_CMD) json_extraction_parser.add_argument("archive_id", type=str, help="Archive ID") json_extraction_parser.add_argument( @@ -307,7 +307,7 @@ def main(argv): command = parsed_args.command if EXTRACT_FILE_CMD == command: return handle_extract_file_cmd(parsed_args, clp_home, default_config_file_path) - elif command in [EXTRACT_IR_CMD, EXTRACT_JSON_CMD]: + elif command in (EXTRACT_IR_CMD, EXTRACT_JSON_CMD): return handle_extract_stream_cmd(parsed_args, clp_home, default_config_file_path) else: logger.exception(f"Unexpected command: {command}") diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py index 46d88dabd..8fb2a07fd 100644 --- a/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py +++ b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py @@ -31,7 +31,7 @@ def make_command( stream_collection: str, ) -> Optional[List[str]]: if StorageEngine.CLP == storage_engine: - logger.info("Start IR extraction") + logger.info("Starting IR extraction") extract_ir_config = ExtractIrJobConfig.parse_obj(job_config_obj) if not extract_ir_config.file_split_id: logger.error("file_split_id not supplied") @@ -49,7 +49,7 @@ def make_command( command.append("--target-size") command.append(str(extract_ir_config.target_uncompressed_size)) elif StorageEngine.CLP_S == storage_engine: - logger.info("Start Json extraction") + logger.info("Starting JSON extraction") extract_json_config = ExtractJsonJobConfig.parse_obj(job_config_obj) command = [ str(clp_home / "bin" / "clp-s"), From 90e55d597c5d5bc5b77ef4b5ce60d93eb6c763c8 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 14:50:35 -0500 Subject: [PATCH 24/39] Address code review suggestions --- .../clp_package_utils/scripts/native/decompress.py | 8 ++++---- .../job_orchestration/scheduler/query/query_scheduler.py | 6 +++--- components/log-viewer-webui/client/src/ui/QueryStatus.jsx | 2 +- components/log-viewer-webui/server/.env | 2 +- components/log-viewer-webui/server/settings.json | 2 +- components/log-viewer-webui/server/src/DbManager.js | 8 ++++---- components/log-viewer-webui/server/src/app.js | 2 +- components/log-viewer-webui/server/src/routes/static.js | 2 +- components/package-template/src/etc/clp-config.yml | 2 +- 9 files changed, 17 insertions(+), 17 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py index e455894d2..d4217d66d 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py +++ b/components/clp-package-utils/clp_package_utils/scripts/native/decompress.py @@ -120,7 +120,7 @@ def handle_extract_stream_cmd( command = parsed_args.command - extraction_config: QueryJobConfig + job_config: QueryJobConfig job_type: QueryJobType if EXTRACT_IR_CMD == command: job_type = QueryJobType.EXTRACT_IR @@ -133,14 +133,14 @@ def handle_extract_stream_cmd( if orig_file_id is None: logger.error(f"Cannot find orig_file_id corresponding to '{orig_file_path}'.") return -1 - extraction_config = ExtractIrJobConfig( + job_config = ExtractIrJobConfig( orig_file_id=orig_file_id, msg_ix=parsed_args.msg_ix, target_uncompressed_size=parsed_args.target_uncompressed_size, ) elif EXTRACT_JSON_CMD == command: job_type = QueryJobType.EXTRACT_JSON - extraction_config = ExtractJsonJobConfig( + job_config = ExtractJsonJobConfig( archive_id=parsed_args.archive_id, target_chunk_size=parsed_args.target_chunk_size ) else: @@ -153,7 +153,7 @@ def handle_extract_stream_cmd( submit_and_monitor_extraction_job_in_db, clp_config.database, job_type, - extraction_config, + job_config, ) ) except asyncio.CancelledError: diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index e3b8ec733..a177d1432 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -690,13 +690,13 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as succeeded") continue - nex_stream_extraction_job = create_stream_extraction_job( + next_stream_extraction_job = create_stream_extraction_job( job_id, job_config, target_id, job_type ) dispatch_job_and_update_db( db_conn, - nex_stream_extraction_job, + next_stream_extraction_job, [archive_id], clp_metadata_db_conn_params, results_cache_uri, @@ -704,7 +704,7 @@ def handle_pending_query_jobs( ) mark_job_waiting_for_target(target_id, job_id, job_type) - active_jobs[job_id] = nex_stream_extraction_job + active_jobs[job_id] = next_stream_extraction_job logger.info(f"Dispatched stream extraction job {job_id} on archive: {archive_id}") else: diff --git a/components/log-viewer-webui/client/src/ui/QueryStatus.jsx b/components/log-viewer-webui/client/src/ui/QueryStatus.jsx index 7f9bc91cf..c1bb639a6 100644 --- a/components/log-viewer-webui/client/src/ui/QueryStatus.jsx +++ b/components/log-viewer-webui/client/src/ui/QueryStatus.jsx @@ -51,7 +51,7 @@ const QueryStatus = () => { setQueryState(QUERY_LOADING_STATES.LOADING); const innerLogEventNum = logEventIdx - data.begin_msg_ix + 1; - window.location = `/log-viewer/index.html?filePath=/stream/${data.path}` + + window.location = `/log-viewer/index.html?filePath=/streams/${data.path}` + `#logEventNum=${innerLogEventNum}`; }) .catch((e) => { diff --git a/components/log-viewer-webui/server/.env b/components/log-viewer-webui/server/.env index dd88b78ed..95e3054a8 100644 --- a/components/log-viewer-webui/server/.env +++ b/components/log-viewer-webui/server/.env @@ -1,5 +1,5 @@ CLIENT_DIR=../client/dist -STREAM_DATA_DIR=../../../build/clp-package/var/data/stream +STREAMS_DATA_DIR=../../../build/clp-package/var/data/streams LOG_VIEWER_DIR=../yscope-log-viewer/dist HOST=localhost diff --git a/components/log-viewer-webui/server/settings.json b/components/log-viewer-webui/server/settings.json index a5f7cb1e7..163f9a9e2 100644 --- a/components/log-viewer-webui/server/settings.json +++ b/components/log-viewer-webui/server/settings.json @@ -9,6 +9,6 @@ "MongoDbStreamFilesCollectionName": "stream-files", "ClientDir": "../client/dist", - "StreamFilesDir": "../../../build/clp-package/var/data/stream", + "StreamFilesDir": "../../../build/clp-package/var/data/streams", "LogViewerDir": "../yscope-log-viewer/dist" } diff --git a/components/log-viewer-webui/server/src/DbManager.js b/components/log-viewer-webui/server/src/DbManager.js index 1da06dd34..79e097280 100644 --- a/components/log-viewer-webui/server/src/DbManager.js +++ b/components/log-viewer-webui/server/src/DbManager.js @@ -84,7 +84,7 @@ class DbManager { /** * @type {import("mongodb").Collection} */ - #StreamFilesCollection; + #streamFilesCollection; #queryJobsTableName; @@ -138,7 +138,7 @@ class DbManager { * @return {Promise} A promise that resolves to the extracted IR file's metadata. */ async getExtractedIrFileMetadata (origFileId, logEventIdx) { - return await this.#StreamFilesCollection.findOne({ + return await this.#streamFilesCollection.findOne({ orig_file_id: origFileId, begin_msg_ix: {$lte: logEventIdx}, end_msg_ix: {$gt: logEventIdx}, @@ -187,8 +187,8 @@ class DbManager { if (err) { throw err; } - this.#StreamFilesCollection = - this.#fastify.mongo.db.collection(config.StreamFilesCollectionName); + this.#streamFilesCollection = + this.#fastify.mongo.db.collection(config.streamFilesCollectionName); }); } diff --git a/components/log-viewer-webui/server/src/app.js b/components/log-viewer-webui/server/src/app.js index ce81ecfd8..5d351fd39 100644 --- a/components/log-viewer-webui/server/src/app.js +++ b/components/log-viewer-webui/server/src/app.js @@ -37,7 +37,7 @@ const app = async ({ mongoConfig: { database: settings.MongoDbName, host: settings.MongoDbHost, - StreamFilesCollectionName: settings.MongoDbStreamFilesCollectionName, + streamFilesCollectionName: settings.MongoDbStreamFilesCollectionName, port: settings.MongoDbPort, }, }); diff --git a/components/log-viewer-webui/server/src/routes/static.js b/components/log-viewer-webui/server/src/routes/static.js index fbf91cff9..6118c2855 100644 --- a/components/log-viewer-webui/server/src/routes/static.js +++ b/components/log-viewer-webui/server/src/routes/static.js @@ -23,7 +23,7 @@ const routes = async (fastify, options) => { streamFilesDir = path.resolve(rootDirname, streamFilesDir); } await fastify.register(fastifyStatic, { - prefix: "/stream", + prefix: "/streams", root: streamFilesDir, }); diff --git a/components/package-template/src/etc/clp-config.yml b/components/package-template/src/etc/clp-config.yml index 875fdb11e..6163abc79 100644 --- a/components/package-template/src/etc/clp-config.yml +++ b/components/package-template/src/etc/clp-config.yml @@ -84,7 +84,7 @@ # ## Where CLP IR files should be output #stream_output: -# directory: "var/data/stream" +# directory: "var/data/streams" # # # How large each IR file should be before being split into a new IR file # target_uncompressed_size: 134217728 # 128 MB From 63a18d75aa9efc9938d9c0ba9fa481360c167bcc Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 14:55:37 -0500 Subject: [PATCH 25/39] Address more code review suggestions --- .../clp_package_utils/scripts/start_clp.py | 2 +- .../executor/query/extract_stream_task.py | 20 +++++++++---------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/components/clp-package-utils/clp_package_utils/scripts/start_clp.py b/components/clp-package-utils/clp_package_utils/scripts/start_clp.py index aea1ec19e..a25756fd9 100755 --- a/components/clp-package-utils/clp_package_utils/scripts/start_clp.py +++ b/components/clp-package-utils/clp_package_utils/scripts/start_clp.py @@ -663,7 +663,7 @@ def start_query_worker( query_worker_mount = [mounts.stream_output_dir] query_worker_env = { "CLP_STREAM_OUTPUT_DIR": container_clp_config.stream_output.directory, - "CLP_STREAM_COLLECTION": clp_config.results_cache.stream_collection_name, + "CLP_STREAM_COLLECTION_NAME": clp_config.results_cache.stream_collection_name, } generic_start_worker( diff --git a/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py index 8fb2a07fd..9e99842ab 100644 --- a/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py +++ b/components/job-orchestration/job_orchestration/executor/query/extract_stream_task.py @@ -26,13 +26,13 @@ def make_command( archives_dir: Path, archive_id: str, stream_output_dir: Path, - job_config_obj: dict, + job_config: dict, results_cache_uri: str, - stream_collection: str, + stream_collection_name: str, ) -> Optional[List[str]]: if StorageEngine.CLP == storage_engine: logger.info("Starting IR extraction") - extract_ir_config = ExtractIrJobConfig.parse_obj(job_config_obj) + extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) if not extract_ir_config.file_split_id: logger.error("file_split_id not supplied") return None @@ -43,14 +43,14 @@ def make_command( extract_ir_config.file_split_id, str(stream_output_dir), results_cache_uri, - stream_collection, + stream_collection_name, ] if extract_ir_config.target_uncompressed_size is not None: command.append("--target-size") command.append(str(extract_ir_config.target_uncompressed_size)) elif StorageEngine.CLP_S == storage_engine: logger.info("Starting JSON extraction") - extract_json_config = ExtractJsonJobConfig.parse_obj(job_config_obj) + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) command = [ str(clp_home / "bin" / "clp-s"), "x", @@ -62,7 +62,7 @@ def make_command( "--mongodb-uri", results_cache_uri, "--mongodb-collection", - stream_collection, + stream_collection_name, ] if extract_json_config.target_chunk_size is not None: command.append("--ordered-chunk-size") @@ -79,7 +79,7 @@ def extract_stream( self: Task, job_id: str, task_id: int, - job_config_obj: dict, + job_config: dict, archive_id: str, clp_metadata_db_conn_params: dict, results_cache_uri: str, @@ -102,7 +102,7 @@ def extract_stream( archive_directory = Path(os.getenv("CLP_ARCHIVE_OUTPUT_DIR")) clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") stream_output_dir = Path(os.getenv("CLP_STREAM_OUTPUT_DIR")) - stream_collection = os.getenv("CLP_STREAM_COLLECTION") + stream_collection_name = os.getenv("CLP_STREAM_COLLECTION_NAME") task_command = make_command( storage_engine=clp_storage_engine, @@ -110,9 +110,9 @@ def extract_stream( archives_dir=archive_directory, archive_id=archive_id, stream_output_dir=stream_output_dir, - job_config_obj=job_config_obj, + job_config=job_config, results_cache_uri=results_cache_uri, - stream_collection=stream_collection, + stream_collection_name=stream_collection_name, ) if not task_command: return report_command_creation_failure( From b900553b6d7599ef549627678c244a88e84fdaba Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 15:04:26 -0500 Subject: [PATCH 26/39] fix for previous commit --- .../job_orchestration/scheduler/query/query_scheduler.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index a177d1432..9b2f6634b 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -441,7 +441,7 @@ def get_task_group_for_job( clp_metadata_db_conn_params: Dict[str, any], results_cache_uri: str, ): - job_config_obj = job.get_config().dict() + job_config = job.get_config().dict() job_type = job.get_type() if QueryJobType.SEARCH_OR_AGGREGATION == job_type: return celery.group( @@ -449,7 +449,7 @@ def get_task_group_for_job( job_id=job.id, archive_id=archive_ids[i], task_id=task_ids[i], - job_config_obj=job_config_obj, + job_config=job_config, clp_metadata_db_conn_params=clp_metadata_db_conn_params, results_cache_uri=results_cache_uri, ) @@ -461,7 +461,7 @@ def get_task_group_for_job( job_id=job.id, archive_id=archive_ids[i], task_id=task_ids[i], - job_config_obj=job_config_obj, + job_config=job_config, clp_metadata_db_conn_params=clp_metadata_db_conn_params, results_cache_uri=results_cache_uri, ) From 6b5e60f19205152a8667f6bead407c9a261830cc Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 15:06:20 -0500 Subject: [PATCH 27/39] Apply suggestions from code review Apply missing suggestions Co-authored-by: kirkrodrigues <2454684+kirkrodrigues@users.noreply.github.com> --- .../job_orchestration/scheduler/query/query_scheduler.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 9b2f6634b..f41c5b9f1 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -71,6 +71,7 @@ # Dictionary that maps IDs of file splits being extracted to IDs of jobs waiting for them active_file_split_ir_extractions: Dict[str, List[str]] = {} + # Dictionary that maps IDs of clp-s archives being extracted to IDs of jobs waiting for them active_archive_json_extractions: Dict[str, List[str]] = {} reducer_connection_queue: Optional[asyncio.Queue] = None @@ -455,7 +456,7 @@ def get_task_group_for_job( ) for i in range(len(archive_ids)) ) - elif job_type in [QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR]: + elif job_type in (QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR): return celery.group( extract_stream.s( job_id=job.id, @@ -627,7 +628,7 @@ def handle_pending_query_jobs( pending_search_jobs.append(new_search_job) active_jobs[job_id] = new_search_job - elif job_type in [QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON]: + elif job_type in (QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON): archive_id, target_id = get_archive_and_target_ids_for_stream_extraction( db_conn, job_config, job_type ) From ff15840ce1afc85a86f7c12beb0ecce32afa0925 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 15:52:11 -0500 Subject: [PATCH 28/39] fix for previous commit (again) --- .../job_orchestration/executor/query/fs_search_task.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/components/job-orchestration/job_orchestration/executor/query/fs_search_task.py b/components/job-orchestration/job_orchestration/executor/query/fs_search_task.py index 162056220..598bfdcfc 100644 --- a/components/job-orchestration/job_orchestration/executor/query/fs_search_task.py +++ b/components/job-orchestration/job_orchestration/executor/query/fs_search_task.py @@ -98,7 +98,7 @@ def search( self: Task, job_id: str, task_id: int, - job_config_obj: dict, + job_config: dict, archive_id: str, clp_metadata_db_conn_params: dict, results_cache_uri: str, @@ -120,7 +120,7 @@ def search( clp_home = Path(os.getenv("CLP_HOME")) archive_directory = Path(os.getenv("CLP_ARCHIVE_OUTPUT_DIR")) clp_storage_engine = os.getenv("CLP_STORAGE_ENGINE") - search_config = SearchJobConfig.parse_obj(job_config_obj) + search_config = SearchJobConfig.parse_obj(job_config) task_command = make_command( storage_engine=clp_storage_engine, From 549d3c43f69d3995436daee1782835a966e57603 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 15:59:26 -0500 Subject: [PATCH 29/39] First refactor --- .../scheduler/query/query_scheduler.py | 214 +++++++++++------- 1 file changed, 130 insertions(+), 84 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index f41c5b9f1..7d924a581 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -15,6 +15,7 @@ from __future__ import annotations +from abc import ABC, abstractmethod import argparse import asyncio import contextlib @@ -76,6 +77,123 @@ active_archive_json_extractions: Dict[str, List[str]] = {} reducer_connection_queue: Optional[asyncio.Queue] = None +class StreamExtractionHandle(ABC): + @abstractmethod + def get_archive_and_target_ids_for_stream_extraction( + self, db_conn, job_config: Dict[str, Any], job_type: QueryJobType + ) -> Tuple[Optional[str], Optional[str]]: ... + + @abstractmethod + def is_stream_extraction_target_active( + self, target_id: str, job_type: QueryJobType + ) -> bool: ... + + @abstractmethod + def mark_job_waiting_for_target( + self, target_id: str, job_id: str, job_type: QueryJobType + ) -> None: ... + + @abstractmethod + def is_target_extracted( + self, results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType + ) -> bool: ... + + @abstractmethod + def create_stream_extraction_job( + self, job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType + ) -> QueryJob: ... + + +class IrExtractionHandle(StreamExtractionHandle): + def get_archive_and_target_ids_for_stream_extraction( + self, db_conn, job_config: Dict[str, Any], job_type: QueryJobType + ) -> Tuple[Optional[str], Optional[str]]: + extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) + return get_archive_and_file_split_ids_for_ir_extraction(db_conn, extract_ir_config) + + def is_stream_extraction_target_active( + self, target_id: str, job_type: QueryJobType + ) -> bool: + return target_id in active_file_split_ir_extractions + + def mark_job_waiting_for_target( + self, target_id: str, job_id: str, job_type: QueryJobType + ) -> None: + global active_file_split_ir_extractions + if target_id not in active_file_split_ir_extractions: + active_file_split_ir_extractions[target_id] = [] + active_file_split_ir_extractions[target_id].append(job_id) + + def is_target_extracted( + self, results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType + ) -> bool: + target_key = "file_split_id" + with pymongo.MongoClient(results_cache_uri) as results_cache_client: + stream_collection = results_cache_client.get_default_database()[stream_collection_name] + results_count = stream_collection.count_documents({target_key: target_id}) + return 0 != results_count + + def create_stream_extraction_job( + self, job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType + ) -> QueryJob: + extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) + extract_ir_config.file_split_id = target_id + new_stream_extraction_job = ExtractIrJob( + id=job_id, + extract_ir_config=extract_ir_config, + state=InternalJobState.WAITING_FOR_DISPATCH, + ) + logger.info(f"Created ir extraction job {job_id} on file_split: {target_id}") + return new_stream_extraction_job + + +class JsonExtractionHandle(StreamExtractionHandle): + def get_archive_and_target_ids_for_stream_extraction( + self, db_conn, job_config: Dict[str, Any], job_type: QueryJobType + ) -> Tuple[Optional[str], Optional[str]]: + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) + archive_id = extract_json_config.archive_id + if check_if_archive_exists(db_conn, extract_json_config.archive_id): + return archive_id, archive_id + else: + logger.error(f"archive {archive_id} does not exist") + + return None, None + + def is_stream_extraction_target_active( + self, target_id: str, job_type: QueryJobType + ) -> bool: + return target_id in active_archive_json_extractions + + def mark_job_waiting_for_target( + self, target_id: str, job_id: str, job_type: QueryJobType + ) -> None: + global active_archive_json_extractions + if target_id not in active_archive_json_extractions: + active_archive_json_extractions[target_id] = [] + active_archive_json_extractions[target_id].append(job_id) + + def is_target_extracted( + self, results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType + ) -> bool: + target_key = "orig_file_id" + with pymongo.MongoClient(results_cache_uri) as results_cache_client: + stream_collection = results_cache_client.get_default_database()[stream_collection_name] + results_count = stream_collection.count_documents({target_key: target_id}) + return 0 != results_count + + def create_stream_extraction_job( + self, job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType + ) -> QueryJob: + extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) + new_stream_extraction_job = ExtractJsonJob( + id=job_id, + extract_json_config=extract_json_config, + state=InternalJobState.WAITING_FOR_DISPATCH, + ) + logger.info(f"Created json extraction job {job_id} on archive: {target_id}") + return new_stream_extraction_job + def cancel_job_except_reducer(job: SearchJob): """ @@ -289,23 +407,6 @@ def get_archives_for_search( return archives_for_search -def get_archive_and_target_ids_for_stream_extraction( - db_conn, job_config: Dict[str, Any], job_type: QueryJobType -) -> Tuple[Optional[str], Optional[str]]: - if QueryJobType.EXTRACT_IR == job_type: - extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) - return get_archive_and_file_split_ids_for_ir_extraction(db_conn, extract_ir_config) - - extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) - archive_id = extract_json_config.archive_id - if check_if_archive_exists(db_conn, extract_json_config.archive_id): - return archive_id, archive_id - else: - logger.error(f"archive {archive_id} does not exist") - - return None, None - - def get_archive_and_file_split_ids_for_ir_extraction( db_conn, extract_ir_config: ExtractIrJobConfig, @@ -357,67 +458,6 @@ def get_archive_and_file_split_ids( return results -def is_stream_extraction_target_active(target_id: str, job_type: QueryJobType) -> bool: - if QueryJobType.EXTRACT_IR == job_type: - return target_id in active_file_split_ir_extractions - return target_id in active_archive_json_extractions - - -def mark_job_waiting_for_target(target_id: str, job_id: str, job_type: QueryJobType) -> None: - global active_file_split_ir_extractions - global active_archive_json_extractions - - active_extraction_lists: Dict[str, List[str]] - if QueryJobType.EXTRACT_IR == job_type: - active_extraction_lists = active_file_split_ir_extractions - else: - active_extraction_lists = active_archive_json_extractions - - if target_id not in active_extraction_lists: - active_extraction_lists[target_id] = [] - active_extraction_lists[target_id].append(job_id) - - -def is_target_extracted( - results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType -) -> bool: - target_key: str - if QueryJobType.EXTRACT_IR == job_type: - target_key = "file_split_id" - else: - target_key = "orig_file_id" - - with pymongo.MongoClient(results_cache_uri) as results_cache_client: - stream_collection = results_cache_client.get_default_database()[stream_collection_name] - results_count = stream_collection.count_documents({target_key: target_id}) - return 0 != results_count - - -def create_stream_extraction_job( - job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType -) -> QueryJob: - new_stream_extraction_job: QueryJob - new_job_state = InternalJobState.WAITING_FOR_DISPATCH - if QueryJobType.EXTRACT_IR == job_type: - extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) - extract_ir_config.file_split_id = target_id - new_stream_extraction_job = ExtractIrJob( - id=job_id, - extract_ir_config=extract_ir_config, - state=new_job_state, - ) - logger.info(f"Created ir extraction job {job_id} on file_split: {target_id}") - else: - extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) - new_stream_extraction_job = ExtractJsonJob( - id=job_id, - extract_json_config=extract_json_config, - state=new_job_state, - ) - logger.info(f"Created json extraction job {job_id} on archive: {target_id}") - return new_stream_extraction_job - - def check_if_archive_exists( db_conn, archive_id: str, @@ -629,7 +669,13 @@ def handle_pending_query_jobs( active_jobs[job_id] = new_search_job elif job_type in (QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON): - archive_id, target_id = get_archive_and_target_ids_for_stream_extraction( + job_handle: StreamExtractionHandle + if QueryJobType.EXTRACT_IR == job_type: + job_handle = IrExtractionHandle() + else: + job_handle = JsonExtractionHandle() + + archive_id, target_id = job_handle.get_archive_and_target_ids_for_stream_extraction( db_conn, job_config, job_type ) if not target_id: @@ -654,8 +700,8 @@ def handle_pending_query_jobs( # Check if the target is currently being extracted; if so, add the job ID to the # list of jobs waiting for it. - if is_stream_extraction_target_active(target_id, job_type): - mark_job_waiting_for_target(target_id, job_id, job_type) + if job_handle.is_stream_extraction_target_active(target_id, job_type): + job_handle.mark_job_waiting_for_target(target_id, job_id, job_type) logger.info( f"target {target_id} is being extracted, so mark job {job_id} as running" ) @@ -672,7 +718,7 @@ def handle_pending_query_jobs( continue # Check if a stream file in the target has already been extracted - if is_target_extracted( + if job_handle.is_target_extracted( results_cache_uri, stream_collection_name, target_id, job_type ): logger.info( @@ -691,7 +737,7 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as succeeded") continue - next_stream_extraction_job = create_stream_extraction_job( + next_stream_extraction_job = job_handle.create_stream_extraction_job( job_id, job_config, target_id, job_type ) @@ -704,7 +750,7 @@ def handle_pending_query_jobs( 1, ) - mark_job_waiting_for_target(target_id, job_id, job_type) + job_handle.mark_job_waiting_for_target(target_id, job_id, job_type) active_jobs[job_id] = next_stream_extraction_job logger.info(f"Dispatched stream extraction job {job_id} on archive: {archive_id}") From 972cb2b548b9182164d6524ea208d1e4254d192b Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 16:41:29 -0500 Subject: [PATCH 30/39] Further refactor --- .../scheduler/query/query_scheduler.py | 209 +++++++++--------- 1 file changed, 100 insertions(+), 109 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 7d924a581..63dc33352 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -15,7 +15,6 @@ from __future__ import annotations -from abc import ABC, abstractmethod import argparse import asyncio import contextlib @@ -24,6 +23,7 @@ import os import pathlib import sys +from abc import ABC, abstractmethod from pathlib import Path from typing import Any, Dict, List, Optional, Tuple @@ -77,122 +77,117 @@ active_archive_json_extractions: Dict[str, List[str]] = {} reducer_connection_queue: Optional[asyncio.Queue] = None + class StreamExtractionHandle(ABC): + def __init__(self, job_id: str): + self.job_id = job_id + + @abstractmethod + def get_stream_id(self) -> Optional[str]: ... + @abstractmethod - def get_archive_and_target_ids_for_stream_extraction( - self, db_conn, job_config: Dict[str, Any], job_type: QueryJobType - ) -> Tuple[Optional[str], Optional[str]]: ... + def get_archive_for_stream_extraction(self, db_conn) -> Optional[str]: ... @abstractmethod - def is_stream_extraction_target_active( - self, target_id: str, job_type: QueryJobType - ) -> bool: ... + def is_stream_extraction_active(self) -> bool: ... @abstractmethod - def mark_job_waiting_for_target( - self, target_id: str, job_id: str, job_type: QueryJobType - ) -> None: ... + def mark_job_as_waiting(self) -> None: ... @abstractmethod - def is_target_extracted( - self, results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType - ) -> bool: ... + def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: ... @abstractmethod - def create_stream_extraction_job( - self, job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType - ) -> QueryJob: ... + def create_stream_extraction_job(self) -> QueryJob: ... class IrExtractionHandle(StreamExtractionHandle): - def get_archive_and_target_ids_for_stream_extraction( - self, db_conn, job_config: Dict[str, Any], job_type: QueryJobType - ) -> Tuple[Optional[str], Optional[str]]: - extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) - return get_archive_and_file_split_ids_for_ir_extraction(db_conn, extract_ir_config) - - def is_stream_extraction_target_active( - self, target_id: str, job_type: QueryJobType - ) -> bool: - return target_id in active_file_split_ir_extractions - - def mark_job_waiting_for_target( - self, target_id: str, job_id: str, job_type: QueryJobType - ) -> None: + def __init__(self, job_id: str, job_config: Dict[str, Any]): + super().__init__(job_id) + self.job_config = ExtractIrJobConfig.parse_obj(job_config) + self.file_split_id = None + + def get_stream_id(self) -> Optional[str]: + return self.file_split_id + + def get_archive_for_stream_extraction(self, db_conn) -> Optional[str]: + archive_id, self.file_split_id = get_archive_and_file_split_ids_for_ir_extraction( + db_conn, self.job_config + ) + self.job_config.file_split_id = self.file_split_id + return archive_id + + def is_stream_extraction_active(self) -> bool: + return self.file_split_id in active_file_split_ir_extractions + + def mark_job_as_waiting(self) -> None: global active_file_split_ir_extractions - if target_id not in active_file_split_ir_extractions: - active_file_split_ir_extractions[target_id] = [] - active_file_split_ir_extractions[target_id].append(job_id) - - def is_target_extracted( - self, results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType - ) -> bool: - target_key = "file_split_id" - with pymongo.MongoClient(results_cache_uri) as results_cache_client: - stream_collection = results_cache_client.get_default_database()[stream_collection_name] - results_count = stream_collection.count_documents({target_key: target_id}) - return 0 != results_count - - def create_stream_extraction_job( - self, job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType - ) -> QueryJob: - extract_ir_config = ExtractIrJobConfig.parse_obj(job_config) - extract_ir_config.file_split_id = target_id - new_stream_extraction_job = ExtractIrJob( - id=job_id, - extract_ir_config=extract_ir_config, + file_split_id = self.file_split_id + if file_split_id not in active_file_split_ir_extractions: + active_file_split_ir_extractions[file_split_id] = [] + active_file_split_ir_extractions[file_split_id].append(self.job_id) + + def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: + return does_document_exist( + results_cache_uri, stream_collection_name, "file_split_id", self.file_split_id + ) + + def create_stream_extraction_job(self) -> QueryJob: + job = ExtractIrJob( + id=self.job_id, + extract_ir_config=self.job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) - logger.info(f"Created ir extraction job {job_id} on file_split: {target_id}") - return new_stream_extraction_job + logger.info(f"Created ir extraction job {self.job_id} on file_split: {self.file_split_id}") + return job class JsonExtractionHandle(StreamExtractionHandle): - def get_archive_and_target_ids_for_stream_extraction( - self, db_conn, job_config: Dict[str, Any], job_type: QueryJobType - ) -> Tuple[Optional[str], Optional[str]]: - extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) - archive_id = extract_json_config.archive_id - if check_if_archive_exists(db_conn, extract_json_config.archive_id): - return archive_id, archive_id - else: - logger.error(f"archive {archive_id} does not exist") + def __init__(self, job_id: str, job_config: Dict[str, Any]): + super().__init__(job_id) + self.job_config = ExtractJsonJobConfig.parse_obj(job_config) + self.archive_id = None - return None, None + def get_stream_id(self) -> Optional[str]: + return self.archive_id - def is_stream_extraction_target_active( - self, target_id: str, job_type: QueryJobType - ) -> bool: - return target_id in active_archive_json_extractions + def get_archive_for_stream_extraction(self, db_conn) -> Optional[str]: + self.archive_id = self.job_config.archive_id + if not check_if_archive_exists(db_conn, self.archive_id): + logger.error(f"archive {self.archive_id} does not exist") + return None - def mark_job_waiting_for_target( - self, target_id: str, job_id: str, job_type: QueryJobType - ) -> None: + return self.archive_id + + def is_stream_extraction_active(self) -> bool: + return self.archive_id in active_archive_json_extractions + + def mark_job_as_waiting(self) -> None: global active_archive_json_extractions - if target_id not in active_archive_json_extractions: - active_archive_json_extractions[target_id] = [] - active_archive_json_extractions[target_id].append(job_id) - - def is_target_extracted( - self, results_cache_uri: str, stream_collection_name: str, target_id: str, job_type: QueryJobType - ) -> bool: - target_key = "orig_file_id" - with pymongo.MongoClient(results_cache_uri) as results_cache_client: - stream_collection = results_cache_client.get_default_database()[stream_collection_name] - results_count = stream_collection.count_documents({target_key: target_id}) - return 0 != results_count - - def create_stream_extraction_job( - self, job_id: str, job_config: Dict[str, Any], target_id: str, job_type: QueryJobType - ) -> QueryJob: - extract_json_config = ExtractJsonJobConfig.parse_obj(job_config) - new_stream_extraction_job = ExtractJsonJob( - id=job_id, - extract_json_config=extract_json_config, + archive_id = self.archive_id + if archive_id not in active_archive_json_extractions: + active_archive_json_extractions[archive_id] = [] + active_archive_json_extractions[archive_id].append(self.job_id) + + def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: + return does_document_exist( + results_cache_uri, stream_collection_name, "orig_file_id", self.archive_id + ) + + def create_stream_extraction_job(self) -> QueryJob: + job = ExtractJsonJob( + id=self.job_id, + extract_json_config=self.job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) - logger.info(f"Created json extraction job {job_id} on archive: {target_id}") - return new_stream_extraction_job + logger.info(f"Created json extraction job {self.job_id} on archive: {self.archive_id}") + return job + + +def does_document_exist(mongodb_uri, collection_name, field, value): + with pymongo.MongoClient(mongodb_uri) as mongo_client: + stream_collection = mongo_client.get_default_database()[collection_name] + return 0 != stream_collection.count_documents({field: value}) def cancel_job_except_reducer(job: SearchJob): @@ -671,14 +666,12 @@ def handle_pending_query_jobs( elif job_type in (QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON): job_handle: StreamExtractionHandle if QueryJobType.EXTRACT_IR == job_type: - job_handle = IrExtractionHandle() + job_handle = IrExtractionHandle(job_id, job_config) else: - job_handle = JsonExtractionHandle() + job_handle = JsonExtractionHandle(job_id, job_config) - archive_id, target_id = job_handle.get_archive_and_target_ids_for_stream_extraction( - db_conn, job_config, job_type - ) - if not target_id: + archive_id = job_handle.get_archive_for_stream_extraction(db_conn) + if not archive_id: if not set_job_or_task_status( db_conn, QUERY_JOBS_TABLE_NAME, @@ -700,10 +693,11 @@ def handle_pending_query_jobs( # Check if the target is currently being extracted; if so, add the job ID to the # list of jobs waiting for it. - if job_handle.is_stream_extraction_target_active(target_id, job_type): - job_handle.mark_job_waiting_for_target(target_id, job_id, job_type) + if job_handle.is_stream_extraction_active(): + job_handle.mark_job_as_waiting() logger.info( - f"target {target_id} is being extracted, so mark job {job_id} as running" + f"target {job_handle.get_stream_id()} is being extracted, " + f"so mark job {job_id} as running" ) if not set_job_or_task_status( db_conn, @@ -718,11 +712,10 @@ def handle_pending_query_jobs( continue # Check if a stream file in the target has already been extracted - if job_handle.is_target_extracted( - results_cache_uri, stream_collection_name, target_id, job_type - ): + if job_handle.is_stream_extracted(results_cache_uri, stream_collection_name): logger.info( - f"target {target_id} already extracted, so mark job {job_id} as done" + f"target {job_handle.get_stream_id()} already extracted, " + f"so mark job {job_id} as done" ) if not set_job_or_task_status( db_conn, @@ -737,9 +730,7 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as succeeded") continue - next_stream_extraction_job = job_handle.create_stream_extraction_job( - job_id, job_config, target_id, job_type - ) + next_stream_extraction_job = job_handle.create_stream_extraction_job() dispatch_job_and_update_db( db_conn, @@ -750,7 +741,7 @@ def handle_pending_query_jobs( 1, ) - job_handle.mark_job_waiting_for_target(target_id, job_id, job_type) + job_handle.mark_job_as_waiting() active_jobs[job_id] = next_stream_extraction_job logger.info(f"Dispatched stream extraction job {job_id} on archive: {archive_id}") From 5ec988f86f4782b4710ae452e295be1930bf1d17 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 16:57:01 -0500 Subject: [PATCH 31/39] Update comments --- .../scheduler/query/query_scheduler.py | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 63dc33352..317af4f34 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -685,11 +685,20 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as failed") continue - # NOTE: The following two if blocks should not be reordered. The method that checks - # whether a stream file has been extracted doesn't guarantee that *all* stream - # files to be extracted for the target (since the extraction job may still be in - # progress). Thus, we must first check whether the target is in the process of - # being extracted, and then check whether it's already been extracted. + # NOTE: The following two if blocks of `is_stream_extraction_active` and + # `is_stream_extracted` should not be reordered. + # We must: + # 1. First, check if the stream is in the process of being extracted + # (`is_stream_extraction_active`). + # 2. Then, check if the stream has already been extracted + # (`is_stream_extracted`). + # + # This order ensures correctness because `is_stream_extracted` returns True if + # any chunk of the stream has been extracted, but it does not guarantee that *all* + # chunks are extracted. If `is_stream_extracted` is checked first and the job is + # marked as successful based on its result, it is possible that the extraction job + # is still in progress, meaning the specific chunk requested by the web UI might + # not yet be ready. # Check if the target is currently being extracted; if so, add the job ID to the # list of jobs waiting for it. From e65799e0aeb31214c6264d8a5c9525ef7e0a6860 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 17:00:03 -0500 Subject: [PATCH 32/39] small polishing --- .../scheduler/query/query_scheduler.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 317af4f34..0efd4f47a 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -133,13 +133,12 @@ def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: st ) def create_stream_extraction_job(self) -> QueryJob: - job = ExtractIrJob( + logger.info(f"Creating ir extraction job {self.job_id} on file_split: {self.file_split_id}") + return ExtractIrJob( id=self.job_id, extract_ir_config=self.job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) - logger.info(f"Created ir extraction job {self.job_id} on file_split: {self.file_split_id}") - return job class JsonExtractionHandle(StreamExtractionHandle): @@ -175,13 +174,12 @@ def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: st ) def create_stream_extraction_job(self) -> QueryJob: - job = ExtractJsonJob( + logger.info(f"Creating json extraction job {self.job_id} on archive: {self.archive_id}") + return ExtractJsonJob( id=self.job_id, extract_json_config=self.job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) - logger.info(f"Created json extraction job {self.job_id} on archive: {self.archive_id}") - return job def does_document_exist(mongodb_uri, collection_name, field, value): From fb8e2a6bf8d2981514c0c20601e152ec2699efce Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Fri, 15 Nov 2024 18:29:52 -0500 Subject: [PATCH 33/39] Further polishing --- .../scheduler/query/query_scheduler.py | 49 ++++++++----------- 1 file changed, 20 insertions(+), 29 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 0efd4f47a..5de494506 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -81,13 +81,11 @@ class StreamExtractionHandle(ABC): def __init__(self, job_id: str): self.job_id = job_id + self.archive_id: Optional[str] = None @abstractmethod def get_stream_id(self) -> Optional[str]: ... - @abstractmethod - def get_archive_for_stream_extraction(self, db_conn) -> Optional[str]: ... - @abstractmethod def is_stream_extraction_active(self) -> bool: ... @@ -102,20 +100,19 @@ def create_stream_extraction_job(self) -> QueryJob: ... class IrExtractionHandle(StreamExtractionHandle): - def __init__(self, job_id: str, job_config: Dict[str, Any]): + def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): super().__init__(job_id) self.job_config = ExtractIrJobConfig.parse_obj(job_config) - self.file_split_id = None - - def get_stream_id(self) -> Optional[str]: - return self.file_split_id - - def get_archive_for_stream_extraction(self, db_conn) -> Optional[str]: - archive_id, self.file_split_id = get_archive_and_file_split_ids_for_ir_extraction( + self.archive_id, self.file_split_id = get_archive_and_file_split_ids_for_ir_extraction( db_conn, self.job_config ) + if self.archive_id is None: + raise ValueError("Job parameters does not resolve to an existing archive") + self.job_config.file_split_id = self.file_split_id - return archive_id + + def get_stream_id(self) -> Optional[str]: + return self.file_split_id def is_stream_extraction_active(self) -> bool: return self.file_split_id in active_file_split_ir_extractions @@ -142,20 +139,14 @@ def create_stream_extraction_job(self) -> QueryJob: class JsonExtractionHandle(StreamExtractionHandle): - def __init__(self, job_id: str, job_config: Dict[str, Any]): + def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): super().__init__(job_id) self.job_config = ExtractJsonJobConfig.parse_obj(job_config) - self.archive_id = None - - def get_stream_id(self) -> Optional[str]: - return self.archive_id - - def get_archive_for_stream_extraction(self, db_conn) -> Optional[str]: self.archive_id = self.job_config.archive_id if not check_if_archive_exists(db_conn, self.archive_id): - logger.error(f"archive {self.archive_id} does not exist") - return None + raise ValueError(f"archive {self.archive_id} does not exist") + def get_stream_id(self) -> Optional[str]: return self.archive_id def is_stream_extraction_active(self) -> bool: @@ -663,13 +654,13 @@ def handle_pending_query_jobs( elif job_type in (QueryJobType.EXTRACT_IR, QueryJobType.EXTRACT_JSON): job_handle: StreamExtractionHandle - if QueryJobType.EXTRACT_IR == job_type: - job_handle = IrExtractionHandle(job_id, job_config) - else: - job_handle = JsonExtractionHandle(job_id, job_config) - - archive_id = job_handle.get_archive_for_stream_extraction(db_conn) - if not archive_id: + try: + if QueryJobType.EXTRACT_IR == job_type: + job_handle = IrExtractionHandle(job_id, job_config, db_conn) + else: + job_handle = JsonExtractionHandle(job_id, job_config, db_conn) + except Exception: + logger.exception("Failed to initialize extraction job") if not set_job_or_task_status( db_conn, QUERY_JOBS_TABLE_NAME, @@ -738,7 +729,7 @@ def handle_pending_query_jobs( continue next_stream_extraction_job = job_handle.create_stream_extraction_job() - + archive_id = job_handle.archive_id dispatch_job_and_update_db( db_conn, next_stream_extraction_job, From 314e008ccec6d01c4eb76b158795fd83477bcd78 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Sat, 16 Nov 2024 11:16:53 -0500 Subject: [PATCH 34/39] Use proper encapsulation for class data member variables --- .../scheduler/query/query_scheduler.py | 79 ++++++++++--------- 1 file changed, 41 insertions(+), 38 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 5de494506..39de8512d 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -80,8 +80,11 @@ class StreamExtractionHandle(ABC): def __init__(self, job_id: str): - self.job_id = job_id - self.archive_id: Optional[str] = None + self._job_id = job_id + self._archive_id: Optional[str] = None + + def get_archive_id(self) -> Optional[str]: + return self._archive_id @abstractmethod def get_stream_id(self) -> Optional[str]: ... @@ -90,10 +93,10 @@ def get_stream_id(self) -> Optional[str]: ... def is_stream_extraction_active(self) -> bool: ... @abstractmethod - def mark_job_as_waiting(self) -> None: ... + def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: ... @abstractmethod - def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: ... + def mark_job_as_waiting(self) -> None: ... @abstractmethod def create_stream_extraction_job(self) -> QueryJob: ... @@ -102,38 +105,38 @@ def create_stream_extraction_job(self) -> QueryJob: ... class IrExtractionHandle(StreamExtractionHandle): def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): super().__init__(job_id) - self.job_config = ExtractIrJobConfig.parse_obj(job_config) - self.archive_id, self.file_split_id = get_archive_and_file_split_ids_for_ir_extraction( - db_conn, self.job_config + self._job_config = ExtractIrJobConfig.parse_obj(job_config) + self._archive_id, self._file_split_id = get_archive_and_file_split_ids_for_ir_extraction( + db_conn, self._job_config ) - if self.archive_id is None: + if self._archive_id is None: raise ValueError("Job parameters does not resolve to an existing archive") - self.job_config.file_split_id = self.file_split_id + self._job_config.file_split_id = self._file_split_id def get_stream_id(self) -> Optional[str]: - return self.file_split_id + return self._file_split_id def is_stream_extraction_active(self) -> bool: - return self.file_split_id in active_file_split_ir_extractions + return self._file_split_id in active_file_split_ir_extractions + + def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: + return does_document_exist( + results_cache_uri, stream_collection_name, "file_split_id", self._file_split_id + ) def mark_job_as_waiting(self) -> None: global active_file_split_ir_extractions - file_split_id = self.file_split_id + file_split_id = self._file_split_id if file_split_id not in active_file_split_ir_extractions: active_file_split_ir_extractions[file_split_id] = [] - active_file_split_ir_extractions[file_split_id].append(self.job_id) - - def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: - return does_document_exist( - results_cache_uri, stream_collection_name, "file_split_id", self.file_split_id - ) + active_file_split_ir_extractions[file_split_id].append(self._job_id) def create_stream_extraction_job(self) -> QueryJob: - logger.info(f"Creating ir extraction job {self.job_id} on file_split: {self.file_split_id}") + logger.info(f"Creating ir extraction job {self._job_id} on file_split: {self._file_split_id}") return ExtractIrJob( - id=self.job_id, - extract_ir_config=self.job_config, + id=self._job_id, + extract_ir_config=self._job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) @@ -141,34 +144,34 @@ def create_stream_extraction_job(self) -> QueryJob: class JsonExtractionHandle(StreamExtractionHandle): def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): super().__init__(job_id) - self.job_config = ExtractJsonJobConfig.parse_obj(job_config) - self.archive_id = self.job_config.archive_id - if not check_if_archive_exists(db_conn, self.archive_id): - raise ValueError(f"archive {self.archive_id} does not exist") + self._job_config = ExtractJsonJobConfig.parse_obj(job_config) + self._archive_id = self._job_config.archive_id + if not check_if_archive_exists(db_conn, self._archive_id): + raise ValueError(f"archive {self._archive_id} does not exist") def get_stream_id(self) -> Optional[str]: - return self.archive_id + return self._archive_id def is_stream_extraction_active(self) -> bool: - return self.archive_id in active_archive_json_extractions + return self._archive_id in active_archive_json_extractions + + def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: + return does_document_exist( + results_cache_uri, stream_collection_name, "orig_file_id", self._archive_id + ) def mark_job_as_waiting(self) -> None: global active_archive_json_extractions - archive_id = self.archive_id + archive_id = self._archive_id if archive_id not in active_archive_json_extractions: active_archive_json_extractions[archive_id] = [] - active_archive_json_extractions[archive_id].append(self.job_id) - - def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: - return does_document_exist( - results_cache_uri, stream_collection_name, "orig_file_id", self.archive_id - ) + active_archive_json_extractions[archive_id].append(self._job_id) def create_stream_extraction_job(self) -> QueryJob: - logger.info(f"Creating json extraction job {self.job_id} on archive: {self.archive_id}") + logger.info(f"Creating json extraction job {self._job_id} on archive: {self._archive_id}") return ExtractJsonJob( - id=self.job_id, - extract_json_config=self.job_config, + id=self._job_id, + extract_json_config=self._job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) @@ -729,7 +732,7 @@ def handle_pending_query_jobs( continue next_stream_extraction_job = job_handle.create_stream_extraction_job() - archive_id = job_handle.archive_id + archive_id = job_handle.get_archive_id() dispatch_job_and_update_db( db_conn, next_stream_extraction_job, From 3d86291bff4d48e19732d3d34dd95948274cb35a Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Sat, 16 Nov 2024 11:17:55 -0500 Subject: [PATCH 35/39] Linter --- .../job_orchestration/scheduler/query/query_scheduler.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 39de8512d..4309187be 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -133,7 +133,9 @@ def mark_job_as_waiting(self) -> None: active_file_split_ir_extractions[file_split_id].append(self._job_id) def create_stream_extraction_job(self) -> QueryJob: - logger.info(f"Creating ir extraction job {self._job_id} on file_split: {self._file_split_id}") + logger.info( + f"Creating ir extraction job {self._job_id} on file_split: {self._file_split_id}" + ) return ExtractIrJob( id=self._job_id, extract_ir_config=self._job_config, From 4b848123213a3488c9e1bf8196689d774ebc3364 Mon Sep 17 00:00:00 2001 From: haiqi96 <14502009+haiqi96@users.noreply.github.com> Date: Sun, 17 Nov 2024 17:46:23 -0500 Subject: [PATCH 36/39] Apply suggestions from code review Co-authored-by: kirkrodrigues <2454684+kirkrodrigues@users.noreply.github.com> --- .../scheduler/query/query_scheduler.py | 59 ++++++++++--------- 1 file changed, 30 insertions(+), 29 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index 4309187be..e5c0dc522 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -75,6 +75,7 @@ # Dictionary that maps IDs of clp-s archives being extracted to IDs of jobs waiting for them active_archive_json_extractions: Dict[str, List[str]] = {} + reducer_connection_queue: Optional[asyncio.Queue] = None @@ -110,7 +111,7 @@ def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): db_conn, self._job_config ) if self._archive_id is None: - raise ValueError("Job parameters does not resolve to an existing archive") + raise ValueError("Job parameters don't resolve to an existing archive") self._job_config.file_split_id = self._file_split_id @@ -134,7 +135,7 @@ def mark_job_as_waiting(self) -> None: def create_stream_extraction_job(self) -> QueryJob: logger.info( - f"Creating ir extraction job {self._job_id} on file_split: {self._file_split_id}" + f"Creating IR extraction job {self._job_id} for file_split: {self._file_split_id}" ) return ExtractIrJob( id=self._job_id, @@ -149,7 +150,7 @@ def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): self._job_config = ExtractJsonJobConfig.parse_obj(job_config) self._archive_id = self._job_config.archive_id if not check_if_archive_exists(db_conn, self._archive_id): - raise ValueError(f"archive {self._archive_id} does not exist") + raise ValueError(f"Archive {self._archive_id} doesn't exist") def get_stream_id(self) -> Optional[str]: return self._archive_id @@ -451,7 +452,6 @@ def check_if_archive_exists( db_conn, archive_id: str, ) -> bool: - query = f"""SELECT 1 FROM {CLP_METADATA_TABLE_PREFIX}archives WHERE id = %s @@ -665,7 +665,7 @@ def handle_pending_query_jobs( else: job_handle = JsonExtractionHandle(job_id, job_config, db_conn) except Exception: - logger.exception("Failed to initialize extraction job") + logger.exception("Failed to initialize extraction job handle") if not set_job_or_task_status( db_conn, QUERY_JOBS_TABLE_NAME, @@ -679,28 +679,29 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as failed") continue - # NOTE: The following two if blocks of `is_stream_extraction_active` and + # NOTE: The following two if blocks for `is_stream_extraction_active` and # `is_stream_extracted` should not be reordered. - # We must: - # 1. First, check if the stream is in the process of being extracted - # (`is_stream_extraction_active`). - # 2. Then, check if the stream has already been extracted - # (`is_stream_extracted`). # - # This order ensures correctness because `is_stream_extracted` returns True if - # any chunk of the stream has been extracted, but it does not guarantee that *all* - # chunks are extracted. If `is_stream_extracted` is checked first and the job is - # marked as successful based on its result, it is possible that the extraction job - # is still in progress, meaning the specific chunk requested by the web UI might - # not yet be ready. - - # Check if the target is currently being extracted; if so, add the job ID to the - # list of jobs waiting for it. + # The logic below works as follows: + # 1. It checks if a stream is already being extracted + # (`is_stream_extraction_active`) and if so, it marks the new job as waiting for + # the old job to finish. + # 2. Otherwise, it checks if a stream has already been extracted + # (`is_stream_extracted`) and if so, it marks the new job as complete. + # 3. Otherwise, it creates a new stream extraction job. + # + # `is_stream_extracted` only checks if a single stream has been extracted rather + # than whether all required streams have been extracted. This means that we can't + # use it to check if the old job is complete; instead, we need to employ the + # aforementioned logic. + + # Check if the required streams are currently being extracted; if so, add the job ID + # to the list of jobs waiting for it. if job_handle.is_stream_extraction_active(): job_handle.mark_job_as_waiting() logger.info( - f"target {job_handle.get_stream_id()} is being extracted, " - f"so mark job {job_id} as running" + f"Stream {job_handle.get_stream_id()} is already being extracted," + f" so mark job {job_id} as running." ) if not set_job_or_task_status( db_conn, @@ -714,11 +715,11 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as running") continue - # Check if a stream file in the target has already been extracted + # Check if a required stream file has already been extracted if job_handle.is_stream_extracted(results_cache_uri, stream_collection_name): logger.info( - f"target {job_handle.get_stream_id()} already extracted, " - f"so mark job {job_id} as done" + f"Stream {job_handle.get_stream_id()} already extracted," + f" so mark job {job_id} as succeeded." ) if not set_job_or_task_status( db_conn, @@ -746,7 +747,7 @@ def handle_pending_query_jobs( job_handle.mark_job_as_waiting() active_jobs[job_id] = next_stream_extraction_job - logger.info(f"Dispatched stream extraction job {job_id} on archive: {archive_id}") + logger.info(f"Dispatched stream extraction job {job_id} for archive: {archive_id}") else: # NOTE: We're skipping the job for this iteration, but its status will remain @@ -927,13 +928,13 @@ async def handle_finished_stream_extraction_job( task_id = task_result.task_id if not QueryJobStatus.SUCCEEDED == task_result.status: logger.error( - f"extraction task job-{job_id}-task-{task_id} failed. " + f"Extraction task job-{job_id}-task-{task_id} failed. " f"Check {task_result.error_log_path} for details." ) new_job_status = QueryJobStatus.FAILED else: logger.info( - f"extraction task job-{job_id}-task-{task_id} succeeded in " + f"Extraction task job-{job_id}-task-{task_id} succeeded in " f"{task_result.duration} second(s)." ) @@ -1011,7 +1012,7 @@ async def check_job_status_and_update_db(db_conn_pool, results_cache_uri): await handle_finished_search_job( db_conn, search_job, returned_results, results_cache_uri ) - elif job_type in [QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR]: + elif job_type in (QueryJobType.EXTRACT_JSON, QueryJobType.EXTRACT_IR): await handle_finished_stream_extraction_job(db_conn, job, returned_results) else: logger.error(f"Unexpected job type: {job_type}, skipping job {job_id}") From 7a3398f7ceb62489a69c910b9a8f8453733488c0 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Sun, 17 Nov 2024 18:22:45 -0500 Subject: [PATCH 37/39] Address code review --- .../scheduler/query/query_scheduler.py | 56 ++++++++++--------- 1 file changed, 29 insertions(+), 27 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index e5c0dc522..d25a41f4e 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -88,7 +88,7 @@ def get_archive_id(self) -> Optional[str]: return self._archive_id @abstractmethod - def get_stream_id(self) -> Optional[str]: ... + def get_stream_id(self) -> str: ... @abstractmethod def is_stream_extraction_active(self) -> bool: ... @@ -106,40 +106,40 @@ def create_stream_extraction_job(self) -> QueryJob: ... class IrExtractionHandle(StreamExtractionHandle): def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): super().__init__(job_id) - self._job_config = ExtractIrJobConfig.parse_obj(job_config) - self._archive_id, self._file_split_id = get_archive_and_file_split_ids_for_ir_extraction( - db_conn, self._job_config + self.__job_config = ExtractIrJobConfig.parse_obj(job_config) + self._archive_id, self.__file_split_id = get_archive_and_file_split_ids_for_ir_extraction( + db_conn, self.__job_config ) if self._archive_id is None: raise ValueError("Job parameters don't resolve to an existing archive") - self._job_config.file_split_id = self._file_split_id + self.__job_config.file_split_id = self.__file_split_id - def get_stream_id(self) -> Optional[str]: - return self._file_split_id + def get_stream_id(self) -> str: + return self.__file_split_id def is_stream_extraction_active(self) -> bool: - return self._file_split_id in active_file_split_ir_extractions + return self.__file_split_id in active_file_split_ir_extractions def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: - return does_document_exist( - results_cache_uri, stream_collection_name, "file_split_id", self._file_split_id + return document_exist( + results_cache_uri, stream_collection_name, "file_split_id", self.__file_split_id ) def mark_job_as_waiting(self) -> None: global active_file_split_ir_extractions - file_split_id = self._file_split_id + file_split_id = self.__file_split_id if file_split_id not in active_file_split_ir_extractions: active_file_split_ir_extractions[file_split_id] = [] active_file_split_ir_extractions[file_split_id].append(self._job_id) def create_stream_extraction_job(self) -> QueryJob: logger.info( - f"Creating IR extraction job {self._job_id} for file_split: {self._file_split_id}" + f"Creating IR extraction job {self._job_id} for file_split: {self.__file_split_id}" ) return ExtractIrJob( id=self._job_id, - extract_ir_config=self._job_config, + extract_ir_config=self.__job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) @@ -147,19 +147,19 @@ def create_stream_extraction_job(self) -> QueryJob: class JsonExtractionHandle(StreamExtractionHandle): def __init__(self, job_id: str, job_config: Dict[str, Any], db_conn): super().__init__(job_id) - self._job_config = ExtractJsonJobConfig.parse_obj(job_config) - self._archive_id = self._job_config.archive_id - if not check_if_archive_exists(db_conn, self._archive_id): + self.__job_config = ExtractJsonJobConfig.parse_obj(job_config) + self._archive_id = self.__job_config.archive_id + if not archive_exists(db_conn, self._archive_id): raise ValueError(f"Archive {self._archive_id} doesn't exist") - def get_stream_id(self) -> Optional[str]: + def get_stream_id(self) -> str: return self._archive_id def is_stream_extraction_active(self) -> bool: return self._archive_id in active_archive_json_extractions def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: - return does_document_exist( + return document_exist( results_cache_uri, stream_collection_name, "orig_file_id", self._archive_id ) @@ -174,15 +174,15 @@ def create_stream_extraction_job(self) -> QueryJob: logger.info(f"Creating json extraction job {self._job_id} on archive: {self._archive_id}") return ExtractJsonJob( id=self._job_id, - extract_json_config=self._job_config, + extract_json_config=self.__job_config, state=InternalJobState.WAITING_FOR_DISPATCH, ) -def does_document_exist(mongodb_uri, collection_name, field, value): +def document_exist(mongodb_uri, collection_name, field, value): with pymongo.MongoClient(mongodb_uri) as mongo_client: - stream_collection = mongo_client.get_default_database()[collection_name] - return 0 != stream_collection.count_documents({field: value}) + collection = mongo_client.get_default_database()[collection_name] + return 0 != collection.count_documents({field: value}) def cancel_job_except_reducer(job: SearchJob): @@ -448,7 +448,8 @@ def get_archive_and_file_split_ids( return results -def check_if_archive_exists( +@exception_default_value(default=[]) +def archive_exists( db_conn, archive_id: str, ) -> bool: @@ -664,7 +665,7 @@ def handle_pending_query_jobs( job_handle = IrExtractionHandle(job_id, job_config, db_conn) else: job_handle = JsonExtractionHandle(job_id, job_config, db_conn) - except Exception: + except ValueError: logger.exception("Failed to initialize extraction job handle") if not set_job_or_task_status( db_conn, @@ -734,11 +735,11 @@ def handle_pending_query_jobs( logger.error(f"Failed to set job {job_id} as succeeded") continue - next_stream_extraction_job = job_handle.create_stream_extraction_job() + new_stream_extraction_job = job_handle.create_stream_extraction_job() archive_id = job_handle.get_archive_id() dispatch_job_and_update_db( db_conn, - next_stream_extraction_job, + new_stream_extraction_job, [archive_id], clp_metadata_db_conn_params, results_cache_uri, @@ -746,7 +747,7 @@ def handle_pending_query_jobs( ) job_handle.mark_job_as_waiting() - active_jobs[job_id] = next_stream_extraction_job + active_jobs[job_id] = new_stream_extraction_job logger.info(f"Dispatched stream extraction job {job_id} for archive: {archive_id}") else: @@ -960,6 +961,7 @@ async def handle_finished_stream_extraction_job( extract_json_config: ExtractJsonJobConfig = job.get_config() waiting_jobs = active_archive_json_extractions.pop(extract_json_config.archive_id) + waiting_jobs.remove(job_id) for waiting_job in waiting_jobs: logger.info(f"Setting status to {new_job_status.to_str()} for waiting jobs: {waiting_job}.") set_job_or_task_status( From 1d1599e1d7aac0900834ee5b6655233e2d3b8425 Mon Sep 17 00:00:00 2001 From: Haiqi Xu <14502009+haiqi96@users.noreply.github.com> Date: Sun, 17 Nov 2024 22:35:58 -0500 Subject: [PATCH 38/39] Fix silly mistakes --- .../job_orchestration/scheduler/query/query_scheduler.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py index d25a41f4e..a9c60f380 100644 --- a/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py +++ b/components/job-orchestration/job_orchestration/scheduler/query/query_scheduler.py @@ -122,7 +122,7 @@ def is_stream_extraction_active(self) -> bool: return self.__file_split_id in active_file_split_ir_extractions def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: - return document_exist( + return document_exists( results_cache_uri, stream_collection_name, "file_split_id", self.__file_split_id ) @@ -159,7 +159,7 @@ def is_stream_extraction_active(self) -> bool: return self._archive_id in active_archive_json_extractions def is_stream_extracted(self, results_cache_uri: str, stream_collection_name: str) -> bool: - return document_exist( + return document_exists( results_cache_uri, stream_collection_name, "orig_file_id", self._archive_id ) @@ -179,7 +179,7 @@ def create_stream_extraction_job(self) -> QueryJob: ) -def document_exist(mongodb_uri, collection_name, field, value): +def document_exists(mongodb_uri, collection_name, field, value): with pymongo.MongoClient(mongodb_uri) as mongo_client: collection = mongo_client.get_default_database()[collection_name] return 0 != collection.count_documents({field: value}) @@ -448,7 +448,7 @@ def get_archive_and_file_split_ids( return results -@exception_default_value(default=[]) +@exception_default_value(default=False) def archive_exists( db_conn, archive_id: str, From a5eb21796ab4c76fc8a96c78d150ca2eea3ab752 Mon Sep 17 00:00:00 2001 From: Kirk Rodrigues <2454684+kirkrodrigues@users.noreply.github.com> Date: Mon, 18 Nov 2024 08:11:46 -0500 Subject: [PATCH 39/39] Update comment. --- components/package-template/src/etc/clp-config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/components/package-template/src/etc/clp-config.yml b/components/package-template/src/etc/clp-config.yml index 6163abc79..15747fe42 100644 --- a/components/package-template/src/etc/clp-config.yml +++ b/components/package-template/src/etc/clp-config.yml @@ -82,7 +82,7 @@ # # How much data CLP should try to fit into each segment within an archive # target_segment_size: 268435456 # 256 MB # -## Where CLP IR files should be output +## Where CLP stream files (e.g., IR streams) should be output #stream_output: # directory: "var/data/streams" #