From 5e7416f5325be578f30dd44d52c778d1f8a8fcd0 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Fri, 18 Aug 2023 11:07:02 -0700 Subject: [PATCH 01/13] add backend functionality for merlin status --- merlin/common/tasks.py | 212 ++++++++++++++++++++++---- merlin/config/celeryconfig.py | 1 + merlin/exceptions/__init__.py | 13 +- merlin/study/dag.py | 16 +- merlin/study/step.py | 278 ++++++++++++++++++++++++++++------ merlin/study/study.py | 30 +++- merlin/utils.py | 107 ++++++++++++- 7 files changed, 568 insertions(+), 89 deletions(-) diff --git a/merlin/common/tasks.py b/merlin/common/tasks.py index b24e0af3e..d91ffd29c 100644 --- a/merlin/common/tasks.py +++ b/merlin/common/tasks.py @@ -31,14 +31,16 @@ """Test tasks.""" from __future__ import absolute_import, unicode_literals +import json import logging import os -from typing import Any, Dict, Optional +from typing import Any, Dict, List, Optional # Need to disable an overwrite warning here since celery has an exception that we need that directly # overwrites a python built-in exception from celery import chain, chord, group, shared_task, signature from celery.exceptions import MaxRetriesExceededError, OperationalError, TimeoutError # pylint: disable=W0622 +from filelock import FileLock, Timeout from merlin.common.abstracts.enums import ReturnCode from merlin.common.sample_index import uniform_directories @@ -47,7 +49,7 @@ from merlin.exceptions import HardFailException, InvalidChainException, RestartException, RetryException from merlin.router import stop_workers from merlin.spec.expansion import parameter_substitutions_for_cmd, parameter_substitutions_for_sample -from merlin.study.step import Step +from merlin.utils import dict_deep_merge retry_exceptions = ( @@ -58,6 +60,7 @@ OperationalError, RetryException, RestartException, + FileNotFoundError, ) LOG = logging.getLogger(__name__) @@ -87,9 +90,11 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq Example kwargs dict: {"adapter_config": {'type':'local'}, - "next_in_chain": } # merlin_step will be added to the current chord - # with next_in_chain as an argument + "next_in_chain": } # merlin_step will be added to the current chord + # with next_in_chain as an argument """ + from merlin.study.step import Step # pylint: disable=C0415 + step: Optional[Step] = None LOG.debug(f"args is {len(args)} long") @@ -109,13 +114,17 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq step_dir: str = step.get_workspace() LOG.debug(f"merlin_step: step_name '{step_name}' step_dir '{step_dir}'") finished_filename: str = os.path.join(step_dir, "MERLIN_FINISHED") + # if we've already finished this task, skip it result: ReturnCode if os.path.exists(finished_filename): LOG.info(f"Skipping step '{step_name}' in '{step_dir}'.") result = ReturnCode.OK else: + LOG.info(f"Executing step '{step_name}' in '{step_dir}'...") result = step.execute(config) + step.mstep.mark_end(result) + if result == ReturnCode.OK: LOG.info(f"Step '{step_name}' in '{step_dir}' finished successfully.") # touch a file indicating we're done with this step @@ -129,6 +138,7 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq LOG.info( f"Step '{step_name}' in '{step_dir}' is being restarted ({self.request.retries + 1}/{self.max_retries})..." ) + step.mstep.mark_restart() self.retry(countdown=step.retry_delay) except MaxRetriesExceededError: LOG.warning( @@ -136,12 +146,15 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq but has already reached its retry limit ({self.max_retries}). Continuing with workflow.""" ) result = ReturnCode.SOFT_FAIL + # Need to call mark_end again since we switched from RESTART to SOFT_FAIL + step.mstep.mark_end(result, max_retries=True) elif result == ReturnCode.RETRY: step.restart = False try: LOG.info( f"Step '{step_name}' in '{step_dir}' is being retried ({self.request.retries + 1}/{self.max_retries})..." ) + step.mstep.mark_restart() self.retry(countdown=step.retry_delay) except MaxRetriesExceededError: LOG.warning( @@ -149,6 +162,8 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq but has already reached its retry limit ({self.max_retries}). Continuing with workflow.""" ) result = ReturnCode.SOFT_FAIL + # Need to call mark_end again since we switched from RETRY to SOFT_FAIL + step.mstep.mark_end(result, max_retries=True) elif result == ReturnCode.SOFT_FAIL: LOG.warning(f"*** Step '{step_name}' in '{step_dir}' soft failed. Continuing with workflow.") elif result == ReturnCode.HARD_FAIL: @@ -159,7 +174,6 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq shutdown = shutdown_workers.s([step_queue]) shutdown.set(queue=step_queue) shutdown.apply_async(countdown=STOP_COUNTDOWN) - raise HardFailException elif result == ReturnCode.STOP_WORKERS: LOG.warning(f"*** Shutting down all workers in {STOP_COUNTDOWN} secs!") @@ -168,6 +182,7 @@ def merlin_step(self, *args: Any, **kwargs: Any) -> Optional[ReturnCode]: # noq shutdown.apply_async(countdown=STOP_COUNTDOWN) else: LOG.warning(f"**** Step '{step_name}' in '{step_dir}' had unhandled exit code {result}. Continuing with workflow.") + # queue off the next task in a chain while adding it to the current chord so that the chordfinisher actually # waits for the next task in the chain if next_in_chain is not None: @@ -194,7 +209,7 @@ def is_chain_expandable(chain_, labels): """ - array_of_bools = [step.needs_merlin_expansion(labels) for step in chain_] + array_of_bools = [step.check_if_expansion_needed(labels) for step in chain_] needs_expansion = all(array_of_bools) @@ -260,14 +275,16 @@ def add_merlin_expanded_chain_to_chord( # pylint: disable=R0913,R0914 :param adapter_config: The adapter config. :param min_sample_id: offset to use for the sample_index. """ + num_samples = len(samples) # Use the index to get a path to each sample - LOG.debug(f"recursing with {len(samples)} samples {samples}") + LOG.debug(f"recursing with {num_samples} samples {samples}") if sample_index.is_grandparent_of_leaf or sample_index.is_parent_of_leaf: all_chains = [] - LOG.debug(f"gathering up {len(samples)} relative paths") + LOG.debug(f"gathering up {num_samples} relative paths") relative_paths = [ - os.path.dirname(sample_index.get_path_to_sample(sample_id + min_sample_id)) for sample_id in range(len(samples)) + os.path.dirname(sample_index.get_path_to_sample(sample_id + min_sample_id)) for sample_id in range(num_samples) ] + top_lvl_workspace = chain_[0].get_workspace() LOG.debug(f"recursing grandparent with relative paths {relative_paths}") for step in chain_: # Make a list of new task objects with modified cmd and workspace @@ -288,13 +305,24 @@ def add_merlin_expanded_chain_to_chord( # pylint: disable=R0913,R0914 ), ), adapter_config=adapter_config, + top_lvl_workspace=top_lvl_workspace, ) new_step.set(queue=step.get_task_queue()) new_chain.append(new_step) all_chains.append(new_chain) + + condense_sig = condense_status_files.s( + sample_index=sample_index, + workspace=top_lvl_workspace, + condensed_workspace=chain_[0].mstep.condensed_workspace, + ).set( + queue=chain_[0].get_task_queue(), + ) + LOG.debug("adding chain to chord") - add_chains_to_chord(self, all_chains) + chain_1d = get_1d_chain(all_chains) + launch_chain(self, chain_1d, condense_sig=condense_sig) LOG.debug("chain added to chord") else: # recurse down the sample_index hierarchy @@ -340,24 +368,52 @@ def add_simple_chain_to_chord(self, task_type, chain_, adapter_config): new_steps = [task_type.s(step, adapter_config=adapter_config).set(queue=step.get_task_queue())] all_chains.append(new_steps) - add_chains_to_chord(self, all_chains) + chain_1d = get_1d_chain(all_chains) + launch_chain(self, chain_1d) -def add_chains_to_chord(self, all_chains): +def launch_chain(self: "Task", chain_1d: List["Signature"], condense_sig: "Signature" = None): # noqa: F821 """ - Adds chains to the current chord. - :param self: The current task whose chord we will add the chains' tasks to. - :param all_chains: Two-dimensional list of chains [chain_length][number_of_chains] + Given a 1D chain, appropriately launch the signatures it contains. + If this is a local run, launch the signatures instantly. + Otherwise, there's two cases: + a. The chain is dealing with samples (i.e. we'll need to condense status files) + so create a new chord and add it to the current chord + b. The chain is NOT dealing with samples so we can just add the signatures to the current chord + + :param `self`: The current task + :param `chain_1d`: A 1-dimensional list of signatures to launch + :param `condense_sig`: A signature for condensing the status files. None if condensing isn't needed. """ - - if len(all_chains) == 1: - # enqueue the steps as a single parallel group - LOG.debug(f"launching group with {signature(all_chains[0][0])}") - for sig in all_chains[0]: - if self.request.is_eager: + # If there's nothing in the chain then we won't have to launch anything so check that first + if chain_1d: + # Case 1: local run; launch signatures instantly + if self.request.is_eager: + for sig in chain_1d: sig.delay() + # Case 2: non-local run; signatures need to be added to the current chord + else: + # Case a: we're dealing with a sample hierarchy and need to condense status files when we're done executing tasks + if condense_sig: + # This chord makes it so we'll process all tasks in chain_1d, then condense the status files when they're done + sample_chord = chord(chain_1d, condense_sig) + self.add_to_chord(sample_chord, lazy=False) + # Case b: no condensing is needed so just add all the signatures to the chord else: - self.add_to_chord(sig, lazy=False) + for sig in chain_1d: + self.add_to_chord(sig, lazy=False) + + +def get_1d_chain(all_chains: List[List["Signature"]]) -> List["Signature"]: # noqa: F821 + """ + Convert a 2D list of chains into a 1D list. + :param all_chains: Two-dimensional list of chains [chain_length][number_of_chains] + :returns: A one-dimensional list representing a chain of tasks + """ + chain_steps = [] + if len(all_chains) == 1: + # Steps will be enqueued in a single parallel group + chain_steps = all_chains[0] if len(all_chains) > 1: # in this case, we need to make a chain. @@ -368,7 +424,6 @@ def add_chains_to_chord(self, all_chains): # during execution of a task belonging to that chord, # so we set up a chain by passing the child member of a chain in as an # argument to the signature of the parent member of a chain. - chain_steps = [] length = len(all_chains[0]) for i in range(length): # Do the following in reverse order because the replace method @@ -385,12 +440,111 @@ def add_chains_to_chord(self, all_chains): all_chains[j][i] = all_chains[j][i].replace(kwargs=new_kwargs) chain_steps.append(all_chains[0][i]) - for sig in chain_steps: - LOG.debug(f"launching chain {signature(sig)}") - if self.request.is_eager: - sig.delay() - else: - self.add_to_chord(sig, lazy=False) + return chain_steps + + +def gather_statuses(sample_index: "SampleIndex", workspace: str, condensed_workspace: str) -> Dict: # noqa: F821 + """ + Traverse the sample index and gather all of the statuses into one. + + :param `sample_index`: A SampleIndex object to track this specific sample hierarchy + :param `workspace`: The full workspace path to the step we're condensing for + :param `condensed_workspace`: A shortened version of `workspace` that's saved in the status files + :returns: A dict of condensed statuses + """ + condensed_statuses = {} + for path, _ in sample_index.traverse(conditional=lambda c: c.is_parent_of_leaf): + # Read in the status data + sample_workspace = f"{workspace}/{path}" + status_filepath = f"{sample_workspace}/MERLIN_STATUS.json" + lock = FileLock(f"{sample_workspace}/status.lock") # pylint: disable=E0110 + try: + # The status files will need locks when reading to avoid race conditions + with lock.acquire(timeout=10): + with open(status_filepath, "r") as status_file: + status = json.load(status_file) + + # This for loop is just to get the step name that we don't have; it's really not even looping + for step_name, _ in status.items(): + try: + # Make sure the status for this sample workspace is in a finished state (not initialized or running) + if status[step_name][f"{condensed_workspace}/{path}"]["status"] not in ("INITIALIZED", "RUNNING"): + # Add the status data to the statuses we'll write to the condensed file and remove this status file + dict_deep_merge(condensed_statuses, status) + os.remove(status_filepath) + except KeyError: + LOG.warning(f"Key error when reading from {sample_workspace}") + except Timeout: + # Raising this celery timeout instead will trigger a restart for this task + raise TimeoutError # pylint: disable=W0707 + except FileNotFoundError: + LOG.warning(f"Could not find {status_filepath} while trying to condense. Restarting this task...") + raise FileNotFoundError # pylint: disable=W0707 + + return condensed_statuses + + +@shared_task( + bind=True, + autoretry_for=retry_exceptions, + retry_backoff=True, + priority=get_priority(Priority.LOW), +) +def condense_status_files(self, *args: Any, **kwargs: Any) -> ReturnCode: # pylint: disable=R0914,W0613 + """ + After a section of the sample tree has finished, condense the status files. + + kwargs should look like so: + kwargs = { + "sample_index": SampleIndex Object, + "workspace": str representing the step's workspace + } + """ + # Get the sample index object that we'll use for condensing + sample_index = kwargs.pop("sample_index", None) + if not sample_index: + LOG.warning("Sample index not found. Cannot condense status files.") + return None + + # Get the full step (or step/parameter) workspace + workspace = kwargs.pop("workspace", None) + if not workspace: + LOG.warning("Workspace not found. Cannot condense status files.") + return None + + # Get a condensed version of the workspace + condensed_workspace = kwargs.pop("condensed_workspace", None) + if not condensed_workspace: + LOG.warning("Condensed workspace not provided. Cannot condense status files.") + return None + + # Read in all the statuses from this sample index + condensed_statuses = gather_statuses(sample_index, workspace, condensed_workspace) + + # If there are statuses to write to the condensed status file then write them + if condensed_statuses: + condensed_status_filepath = f"{workspace}/MERLIN_STATUS.json" + condensed_lock_file = f"{workspace}/status.lock" + lock = FileLock(condensed_lock_file) # pylint: disable=E0110 + try: + # Lock the file to avoid race conditions + with lock.acquire(timeout=20): + # If the condensed file already exists, grab the statuses from it + if os.path.exists(condensed_status_filepath): + with open(condensed_status_filepath, "r") as condensed_status_file: + existing_condensed_statuses = json.load(condensed_status_file) + # Merging the statuses we're condensing into the already existing statuses + # because it's faster at scale than vice versa + dict_deep_merge(existing_condensed_statuses, condensed_statuses) + condensed_statuses = existing_condensed_statuses + + # Write the condensed statuses to the condensed status file + with open(condensed_status_filepath, "w") as condensed_status_file: + json.dump(condensed_statuses, condensed_status_file) + except Timeout: + # Raising this celery timeout instead will trigger a restart for this task + raise TimeoutError # pylint: disable=W0707 + return ReturnCode.OK diff --git a/merlin/config/celeryconfig.py b/merlin/config/celeryconfig.py index 73605768e..5e998765a 100644 --- a/merlin/config/celeryconfig.py +++ b/merlin/config/celeryconfig.py @@ -56,4 +56,5 @@ "worker_log_color": True, "worker_log_format": FORMATS["DEFAULT"], "worker_task_log_format": FORMATS["WORKER"], + "worker_cancel_long_running_tasks_on_connection_loss": True, } diff --git a/merlin/exceptions/__init__.py b/merlin/exceptions/__init__.py index b176378a9..8fa9a21f2 100644 --- a/merlin/exceptions/__init__.py +++ b/merlin/exceptions/__init__.py @@ -34,7 +34,7 @@ # Pylint complains that these exceptions are no different from Exception # but we don't care, we just need new names for exceptions here -# pylint: disable=W0246 +# pylint: disable=W0235 __all__ = ( "RetryException", @@ -42,6 +42,7 @@ "HardFailException", "InvalidChainException", "RestartException", + "DeepMergeException," ) @@ -92,3 +93,13 @@ class RestartException(Exception): def __init__(self): super().__init__() + + +class DeepMergeException(Exception): + """ + Exception to signal that there's a conflict when trying + to merge two dicts together + """ + + def __init__(self, message): + super().__init__(message) diff --git a/merlin/study/dag.py b/merlin/study/dag.py index d2315bf7f..06de5070a 100644 --- a/merlin/study/dag.py +++ b/merlin/study/dag.py @@ -44,11 +44,15 @@ class DAG: independent chains of tasks. """ - def __init__(self, maestro_adjacency_table, maestro_values, labels): + def __init__( + self, maestro_adjacency_table, maestro_values, column_labels, study_name, parameter_info + ): # pylint: disable=R0913 """ :param `maestro_adjacency_table`: An ordered dict showing adjacency of nodes. Comes from a maestrowf ExecutionGraph. :param `maestro_values`: An ordered dict of the values at each node. Comes from a maestrowf ExecutionGraph. - :param `labels`: A list of labels provided in the spec file. + :param `column_labels`: A list of column labels provided in the spec file. + :param `study_name`: The name of the study + :param `parameter_info`: A dict containing information about parameters in the study """ # We used to store the entire maestro ExecutionGraph here but now it's # unpacked so we're only storing the 2 attributes from it that we use: @@ -56,9 +60,11 @@ def __init__(self, maestro_adjacency_table, maestro_values, labels): # to work for Celery. self.maestro_adjacency_table = maestro_adjacency_table self.maestro_values = maestro_values + self.column_labels = column_labels + self.study_name = study_name + self.parameter_info = parameter_info self.backwards_adjacency = {} self.calc_backwards_adjacency() - self.labels = labels def step(self, task_name): """Return a Step object for the given task name @@ -66,7 +72,7 @@ def step(self, task_name): :param `task_name`: The task name. :return: A Merlin Step object. """ - return Step(self.maestro_values[task_name]) + return Step(self.maestro_values[task_name], self.study_name, self.parameter_info) def calc_depth(self, node, depths, current_depth=0): """Calculate the depth of the given node and its children. @@ -176,7 +182,7 @@ def compatible_merlin_expansion(self, task1, task2): """ step1 = self.step(task1) step2 = self.step(task2) - return step1.needs_merlin_expansion(self.labels) == step2.needs_merlin_expansion(self.labels) + return step1.check_if_expansion_needed(self.column_labels) == step2.check_if_expansion_needed(self.column_labels) def find_independent_chains(self, list_of_groups_of_chains): """ diff --git a/merlin/study/step.py b/merlin/study/step.py index 686b5afa2..3a0290b89 100644 --- a/merlin/study/step.py +++ b/merlin/study/step.py @@ -29,43 +29,227 @@ ############################################################################### """This module represents all of the logic that goes into a step""" +import json import logging +import os import re from contextlib import suppress from copy import deepcopy -from datetime import datetime +from typing import Dict, Optional, Tuple +from celery import current_task +from filelock import FileLock from maestrowf.abstracts.enums import State from maestrowf.datastructures.core.executiongraph import _StepRecord from maestrowf.datastructures.core.study import StudyStep from merlin.common.abstracts.enums import ReturnCode from merlin.study.script_adapter import MerlinScriptAdapter +from merlin.study.status import read_status +from merlin.utils import needs_merlin_expansion LOG = logging.getLogger(__name__) +def get_current_worker(): + """Get the worker on the current running task from celery""" + worker = re.search(r"@.+\.", current_task.request.hostname).group() + worker = worker[1 : len(worker) - 1] + return worker + + +def get_current_queue(): + """Get the queue on the current running task from celery""" + from merlin.config.configfile import CONFIG # pylint: disable=C0415 + + queue = current_task.request.delivery_info["routing_key"] + queue = queue.replace(CONFIG.celery.queue_tag, "") + return queue + + class MerlinStepRecord(_StepRecord): """ - This classs is a wrapper for the Maestro _StepRecord to remove - a re-submit message. + This class is a wrapper for the Maestro _StepRecord to remove + a re-submit message and handle status updates. """ - def __init__(self, workspace, step, **kwargs): - _StepRecord.__init__(self, workspace, step, **kwargs) + def __init__(self, workspace: str, maestro_step: StudyStep, merlin_step: "Step", **kwargs): + """ + :param `workspace`: The output workspace for this step + :param `maestro_step`: The StudyStep object associated with this step + :param `merlin_step`: The Step object associated with this step + """ + _StepRecord.__init__(self, workspace, maestro_step, status=State.INITIALIZED, **kwargs) + self.merlin_step = merlin_step - def mark_submitted(self): - """Mark the submission time of the record.""" - LOG.debug("Marking %s as submitted (PENDING) -- previously %s", self.name, self.status) - self.status = State.PENDING - if not self._submit_time: - self._submit_time = datetime.now() + @property + def condensed_workspace(self) -> str: + """ + Put together a smaller version of the workspace path to display. + :returns: A condensed workspace name + """ + timestamp_regex = r"\d{8}-\d{6}/" + match = re.search(rf"{self.merlin_step.study_name}_{timestamp_regex}", self.workspace.value) + + # If we got a match from the regex (which we should always get) then use it to condense the workspace + if match: + condensed_workspace = self.workspace.value.split(match.group())[1] + # Otherwise manually condense (which could have issues if step names/parameters/study names are equivalent) else: - LOG.debug( - "Merlin: Cannot set the submission time of '%s' because it has already been set.", - self.name, - ) + step_name = self.merlin_step.name_no_params() + end_of_path = self.workspace.value.rsplit(step_name, 1)[1] + condensed_workspace = f"{step_name}{end_of_path}" + + return condensed_workspace + + def _execute(self, adapter: "ScriptAdapter", script: str) -> Tuple["SubmissionRecord", int]: # noqa: F821 + """ + Overwrites _StepRecord's _execute method from Maestro since self.to_be_scheduled is + always true here. Also, if we didn't overwrite this we wouldn't be able to call + self.mark_running() for status updates. + + :param `adapter`: The script adapter to submit jobs to + :param `script`: The script to send to the script adapter + :returns: A tuple of a return code and the jobid from the execution of `script` + """ + self.mark_running() + srecord = adapter.submit(self.step, script, self.workspace.value) + + retcode = srecord.submission_code + jobid = srecord.job_identifier + return retcode, jobid + + def mark_running(self): + """Mark the start time of the record and update the status file.""" + super().mark_running() + self._update_status_file() + + def mark_end(self, state: ReturnCode, max_retries: bool = False): + """ + Mark the end time of the record with associated termination state + and update the status file. + + :param `state`: A merlin ReturnCode object representing the end state of a task + :param `max_retries`: A bool representing whether we hit the max number of retries or not + """ + # Dictionary to keep track of associated variables for each return code + state_mapper = { + ReturnCode.OK: { + "maestro state": State.FINISHED, + "result": "MERLIN_SUCCESS", + }, + ReturnCode.DRY_OK: { + "maestro state": State.DRYRUN, + "result": "MERLIN_DRY_SUCCESS", + }, + ReturnCode.RETRY: { + "maestro state": State.FINISHED, + "result": "MERLIN_RETRY", + }, + ReturnCode.RESTART: { + "maestro state": State.FINISHED, + "result": "MERLIN_RESTART", + }, + ReturnCode.SOFT_FAIL: { + "maestro state": State.FAILED, + "result": "MERLIN_SOFT_FAIL", + }, + ReturnCode.HARD_FAIL: { + "maestro state": State.FAILED, + "result": "MERLIN_HARD_FAIL", + }, + ReturnCode.STOP_WORKERS: { + "maestro state": State.CANCELLED, + "result": "MERLIN_STOP_WORKERS", + }, + "UNKNOWN": { + "maestro state": State.UNKNOWN, + "result": "MERLIN_UNRECOGNIZED", + }, + } + + # Check if the state provided is valid + if state not in state_mapper: + state = "UNKNOWN" + + # Call to super().mark_end() will mark end time and update self.status for us + super().mark_end(state_mapper[state]["maestro state"]) + step_result = state_mapper[state]["result"] + + # Append a "max retries reached" message to the step result if necessary + if state == ReturnCode.SOFT_FAIL and max_retries: + step_result += " (MAX RETRIES REACHED)" + + # Update the status file + self._update_status_file(result=step_result) + + def mark_restart(self): + """Increment the number of restarts we've had for this step and update the status file""" + if self.restart_limit == 0 or self._num_restarts < self.restart_limit: + self._num_restarts += 1 + self._update_status_file() + + def setup_workspace(self): + """Initialize the record's workspace and status file.""" + super().setup_workspace() + self._update_status_file() + + def _update_status_file( + self, + result: Optional[str] = None, + task_server: Optional[str] = "celery", + ): + """ + Puts together a dictionary full of status info and creates a signature + for the update_status celery task. This signature is ran here as well. + + :param `result`: Optional parameter only applied when we've finished running + this step. String representation of a ReturnCode value. + :param `task_server`: Optional parameter to define the task server we're using. + """ + + # This dict is used for converting an enum value to a string for readability + state_translator: Dict[State, str] = { + State.INITIALIZED: "INITIALIZED", + State.RUNNING: "RUNNING", + State.FINISHED: "FINISHED", + State.CANCELLED: "CANCELLED", + State.DRYRUN: "DRY_RUN", + State.FAILED: "FAILED", + State.UNKNOWN: "UNKNOWN", + } + + status_filepath = f"{self.workspace.value}/MERLIN_STATUS.json" + + # If the status file already exists then we can just add to it + if os.path.exists(status_filepath): + lock = FileLock(f"{self.workspace.value}/status.lock") # pylint: disable=E0110 + status_info = read_status(status_filepath, lock) + else: + # Inititalize the status_info dict we'll be dumping to the status file + status_info = {self.name: {}} + + # Add celery specific info + if task_server == "celery": + from merlin.celery import app # pylint: disable=C0415 + + # If the tasks are always eager, this is a local run and we won't have workers running + if not app.conf.task_always_eager: + status_info[self.name]["task_queue"] = get_current_queue() + status_info[self.name]["worker_name"] = get_current_worker() + + # Put together a dict of status info + status_info[self.name][self.condensed_workspace] = { + "status": state_translator[self.status], + "return_code": result, + "elapsed_time": self.elapsed_time, + "run_time": self.run_time, + "restarts": self.restarts, + } + + with open(status_filepath, "w") as status_file: + json.dump(status_info, status_file) class Step: @@ -74,11 +258,15 @@ class Step: executed by calling execute. """ - def __init__(self, maestro_step_record): + def __init__(self, maestro_step_record, study_name, parameter_info): """ :param maestro_step_record: The StepRecord object. + :param `study_name`: The name of the study + :param `parameter_info`: A dict containing information about parameters in the study """ self.mstep = maestro_step_record + self.study_name = study_name + self.parameter_info = parameter_info self.__restart = False def get_cmd(self): @@ -125,7 +313,7 @@ def clone_changing_workspace_and_cmd(self, new_cmd=None, cmd_replacement_pairs=N study_step.name = step_dict["_name"] study_step.description = step_dict["description"] study_step.run = step_dict["run"] - return Step(MerlinStepRecord(new_workspace, study_step)) + return Step(MerlinStepRecord(new_workspace, study_step, self), self.study_name, self.parameter_info) def get_task_queue(self): """Retrieve the task queue for the Step.""" @@ -179,36 +367,12 @@ def restart(self, val): """ self.__restart = val - def needs_merlin_expansion(self, labels): + def check_if_expansion_needed(self, labels): """ :return : True if the cmd has any of the default keywords or spec specified sample column labels. """ - needs_expansion = False - - cmd = self.get_cmd() - for label in labels + [ - "MERLIN_SAMPLE_ID", - "MERLIN_SAMPLE_PATH", - "merlin_sample_id", - "merlin_sample_path", - ]: - if f"$({label})" in cmd: - needs_expansion = True - - # The restart may need expansion while the cmd does not. - restart_cmd = self.get_restart_cmd() - if not needs_expansion and restart_cmd: - for label in labels + [ - "MERLIN_SAMPLE_ID", - "MERLIN_SAMPLE_PATH", - "merlin_sample_id", - "merlin_sample_path", - ]: - if f"$({label})" in restart_cmd: - needs_expansion = True - - return needs_expansion + return needs_merlin_expansion(self.get_cmd(), self.get_restart_cmd(), labels) def get_workspace(self): """ @@ -222,6 +386,29 @@ def name(self): """ return self.mstep.step.__dict__["_name"] + def name_no_params(self): + """ + Get the original name of the step without any parameters/samples in the name. + :returns: A string representing the name of the step + """ + # Get the name with everything still in it + name = self.name() + + # Remove the parameter labels from the name + for label in self.parameter_info["labels"]: + name = name.replace(f"{label}", "") + + # Remove possible leftover characters after condensing the name + while name.endswith(".") or name.endswith("_"): + if name.endswith("."): + split_char = "." + else: + split_char = "_" + split_name = name.rsplit(split_char, 1) + name = "".join(split_name) + + return name + def execute(self, adapter_config): """ Execute the step. @@ -255,8 +442,6 @@ def execute(self, adapter_config): self.mstep.setup_workspace() self.mstep.generate_script(adapter) - step_name = self.name() - step_dir = self.get_workspace() # dry run: sets up a workspace without executing any tasks. Each step's # workspace directory is created, and each step's command script is @@ -265,7 +450,6 @@ def execute(self, adapter_config): if adapter_config["dry_run"] is True: return ReturnCode.DRY_OK - LOG.info(f"Executing step '{step_name}' in '{step_dir}'...") # TODO: once maestrowf is updated so that execute returns a # submissionrecord, then we need to return the record.return_code here # at that point, we can drop the use of MerlinScriptAdapter above, and @@ -276,4 +460,4 @@ def execute(self, adapter_config): if self.restart and self.get_restart_cmd(): return ReturnCode(self.mstep.restart(adapter)) - return ReturnCode(self.mstep.execute(adapter)) + return ReturnCode(self.mstep.execute(adapter)) \ No newline at end of file diff --git a/merlin/study/study.py b/merlin/study/study.py index 8eaf306ca..445e6925d 100644 --- a/merlin/study/study.py +++ b/merlin/study/study.py @@ -55,10 +55,11 @@ LOG = logging.getLogger(__name__) -# TODO: see if there's any way to split this class up (pylint doesn't like how many attributes there are) +# TODO: see if there's any way to split this class up +# (pylint doesn't like how many attributes and public methods there are) # - Might be able to create an object to store files and handle file modifications # - If we don't want to create entirely new classes we could try grouping args into dicts -class MerlinStudy: # pylint: disable=R0902 +class MerlinStudy: # pylint: disable=R0902,R0904 """ Represents a Merlin study run on a specification. Used for 'merlin run'. @@ -543,11 +544,14 @@ def load_dag(self): # Generate the DAG _, maestro_dag = study.stage() - labels = [] + column_labels = [] if self.expanded_spec.merlin["samples"]: - labels = self.expanded_spec.merlin["samples"]["column_labels"] + column_labels = self.expanded_spec.merlin["samples"]["column_labels"] + parameter_info = { + "labels": self.parameter_labels, + } # To avoid pickling issues with _pass_detect_cycle from maestro, we unpack the dag here - self.dag = DAG(maestro_dag.adjacency_table, maestro_dag.values, labels) + self.dag = DAG(maestro_dag.adjacency_table, maestro_dag.values, column_labels, study.name, parameter_info) def get_adapter_config(self, override_type=None): """Builds and returns the adapter configuration dictionary""" @@ -571,3 +575,19 @@ def get_adapter_config(self, override_type=None): LOG.debug(f"Adapter config = {adapter_config}") return adapter_config + + @property + def parameter_labels(self): + """ + Get the parameter labels for this study. + :returns: A list of parameter labels used in this study + """ + parameters = self.expanded_spec.get_parameters() + metadata = parameters.get_metadata() + + param_labels = [] + for parameter_info in metadata.values(): + for parameter_label in parameter_info["labels"].values(): + param_labels.append(parameter_label) + + return param_labels diff --git a/merlin/utils.py b/merlin/utils.py index a2e4966d0..1e4b5e199 100644 --- a/merlin/utils.py +++ b/merlin/utils.py @@ -39,14 +39,16 @@ import subprocess from contextlib import contextmanager from copy import deepcopy -from datetime import timedelta +from datetime import datetime, timedelta from types import SimpleNamespace -from typing import Union +from typing import List, Optional, Union import numpy as np import psutil import yaml +from merlin.exceptions import DeepMergeException + try: import cPickle as pickle @@ -288,6 +290,38 @@ def determine_protocol(fname): return protocol +def verify_filepath(filepath: str) -> str: + """ + Verify that the filepath argument is a valid + file. + + :param [str] `filepath`: the path of a file + + :return: the verified absolute filepath with expanded environment variables. + :rtype: str + """ + filepath = os.path.abspath(os.path.expandvars(os.path.expanduser(filepath))) + if not os.path.isfile(filepath): + raise ValueError(f"'{filepath}' is not a valid filepath") + return filepath + + +def verify_dirpath(dirpath: str) -> str: + """ + Verify that the dirpath argument is a valid + directory. + + :param [str] `dirpath`: the path of a directory + + :return: returns the absolute path with expanded environment vars for a given dirpath. + :rtype: str + """ + dirpath: str = os.path.abspath(os.path.expandvars(os.path.expanduser(dirpath))) + if not os.path.isdir(dirpath): + raise ValueError(f"'{dirpath}' is not a valid directory path") + return dirpath + + @contextmanager def cd(path): # pylint: disable=C0103 """ @@ -497,6 +531,59 @@ def contains_shell_ref(string): return False +def needs_merlin_expansion( + cmd: str, restart_cmd: str, labels: List[str], include_sample_keywords: Optional[bool] = True +) -> bool: + """ + Check if the cmd or restart cmd provided have variables that need expansion. + + :param `cmd`: The command inside a study step to check for expansion + :param `restart_cmd`: The restart command inside a study step to check for expansion + :param `labels`: A list of labels to check for inside `cmd` and `restart_cmd` + :return : True if the cmd has any of the default keywords or spec + specified sample column labels. False otherwise. + """ + sample_keywords = ["MERLIN_SAMPLE_ID", "MERLIN_SAMPLE_PATH", "merlin_sample_id", "merlin_sample_path"] + if include_sample_keywords: + labels += sample_keywords + + for label in labels: + if f"$({label})" in cmd: + return True + # The restart may need expansion while the cmd does not. + if restart_cmd and f"$({label})" in restart_cmd: + return True + + # If we got through all the labels and no expansion was needed then these commands don't need expansion + return False + + +def dict_deep_merge(dict_a, dict_b, path=None): + """ + This function recursively merges dict_b into dict_a. The built-in + merge of dictionaries in python (dict(dict_a) | dict(dict_b)) does not do a + deep merge so this function is necessary. This will only merge in new keys, + it will NOT update existing ones. + Credit to this stack overflow post: https://stackoverflow.com/a/7205107. + + :param `dict_a`: A dict that we'll merge dict_b into + :param `dict_b`: A dict that we want to merge into dict_a + :param `path`: The path down the dictionary tree that we're currently at + """ + if path is None: + path = [] + for key in dict_b: + if key in dict_a: + if isinstance(dict_a[key], dict) and isinstance(dict_b[key], dict): + dict_deep_merge(dict_a[key], dict_b[key], path + [str(key)]) + elif dict_a[key] == dict_b[key]: + pass # same leaf value + else: + raise DeepMergeException(f"Conflict at {'.'.join(path + [str(key)])}") + else: + dict_a[key] = dict_b[key] + + # Time utilities def convert_to_timedelta(timestr: Union[str, int]) -> timedelta: """Convert a timestring to a timedelta object. @@ -560,3 +647,19 @@ def convert_timestring(timestring: Union[str, int], format_method: str = "HMS") tdelta = convert_to_timedelta(timestring) LOG.debug(f"Timedelta object is: {tdelta}") return repr_timedelta(tdelta, method=format_method) + + +def ws_time_to_dt(ws_time: str) -> datetime: + """ + Converts a workspace timestring to a datetime object. + + :param `ws_time`: A workspace timestring in the format YYYYMMDD-HHMMSS + :returns: A datetime object created from the workspace timestring + """ + year = int(ws_time[:4]) + month = int(ws_time[4:6]) + day = int(ws_time[6:8]) + hour = int(ws_time[9:11]) + minute = int(ws_time[11:13]) + second = int(ws_time[13:]) + return datetime(year, month, day, hour=hour, minute=minute, second=second) \ No newline at end of file From 0192610d2bcb4e50051e30634ed2754ddb891a4d Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Fri, 18 Aug 2023 11:08:22 -0700 Subject: [PATCH 02/13] add frontend functionality for merlin status --- merlin/common/dumper.py | 151 +++++++++++ merlin/display.py | 240 ++++++++++++++--- merlin/main.py | 122 +++++---- merlin/spec/specification.py | 39 ++- merlin/study/celeryadapter.py | 4 +- merlin/study/status.py | 484 ++++++++++++++++++++++++++++++++++ 6 files changed, 950 insertions(+), 90 deletions(-) create mode 100644 merlin/common/dumper.py create mode 100644 merlin/study/status.py diff --git a/merlin/common/dumper.py b/merlin/common/dumper.py new file mode 100644 index 000000000..178153a03 --- /dev/null +++ b/merlin/common/dumper.py @@ -0,0 +1,151 @@ +############################################################################### +# Copyright (c) 2022, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.0 +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### +"""This file is meant to help dump information to files""" + +import csv +import json +import logging +import os +from typing import Dict, List + + +LOG = logging.getLogger(__name__) + + +# TODO When we add more public methods we can get rid of this pylint disable +class Dumper: # pylint: disable=R0903 + """ + The dumper class is intended to help write information to files. + Currently, the supported file types to dump to are csv and json. + + Example csv usage: + dumper = Dumper("populations.csv") + # Eugene, OR has a population of 175096 + # Livermore, CA has a population of 86803 + population_data = { + "City": ["Eugene", "Livermore"], + "State": ["OR", "CA"], + "Population": [175096, 86803] + } + dumper.write(population_data, "w") + |---> Output will be written to populations.csv + + Example json usage: + dumper = Dumper("populations.json") + population_data = { + "OR": {"Eugene": 175096, "Portland": 641162}, + "CA": {"Livermore": 86803, "San Francisco": 815201} + } + dumper.write(population_data, "w") + |---> Output will be written to populations.json + """ + + def __init__(self, file_name): + """ + Initialize the class and ensure the file is of a supported type. + :param `file_name`: The name of the file to dump to eventually + """ + supported_types = ["csv", "json"] + + valid_file = False + for stype in supported_types: + if file_name.endswith(stype): + valid_file = True + self.file_type = stype + + if not valid_file: + raise ValueError(f"Invalid file type for {file_name}. Supported file types are: {supported_types}.") + + self.file_name = file_name + + def write(self, info_to_write: Dict, fmode: str): + """ + Write information to an outfile. + :param `info_to_write`: The information you want to write to the output file + :param `fmode`: The file write mode ("w", "a", etc.) + """ + if self.file_type == "csv": + self._csv_write(info_to_write, fmode) + elif self.file_type == "json": + self._json_write(info_to_write, fmode) + + def _csv_write(self, csv_to_dump: Dict[str, List], fmode: str): + """ + Write information to a csv file. + :param `csv_to_dump`: The information to write to the csv file. + Dict keys will be the column headers and values will be the column values. + :param `fmode`: The file write mode ("w", "a", etc.) + """ + # If we have statuses to write, create a csv writer object and write to the csv file + with open(self.file_name, fmode) as outfile: + csv_writer = csv.writer(outfile) + if fmode == "w": + csv_writer.writerow(csv_to_dump.keys()) + csv_writer.writerows(zip(*csv_to_dump.values())) + + def _json_write(self, json_to_dump: Dict[str, Dict], fmode: str): + """ + Write information to a json file. + :param `json_to_dump`: The information to write to the json file. + :param `fmode`: The file write mode ("w", "a", etc.) + """ + # Appending to json requires file mode to be r+ for json.load + if fmode == "a": + fmode = "r+" + + with open(self.file_name, fmode) as outfile: + # If we're appending, read in the existing file data + if fmode == "r+": + file_data = json.load(outfile) + json_to_dump.update(file_data) + outfile.seek(0) + # Write to the outfile + json.dump(json_to_dump, outfile) + + +def dump_handler(dump_file: str, dump_info: Dict): + """ + Help handle the process of creating a Dumper object and writing + to an output file. + + :param `dump_file`: A filepath to the file we're dumping to + :param `dump_info`: A dict of information that we'll be dumping to `dump_file` + """ + # Create a dumper object to help us write to dump_file + dumper = Dumper(dump_file) + + # Get the correct file write mode and log message + fmode = "a" if os.path.exists(dump_file) else "w" + write_type = "Writing" if fmode == "w" else "Appending" + LOG.info(f"{write_type} to {dump_file}...") + + # Write the output + dumper.write(dump_info, fmode) + LOG.info(f"{write_type} complete.") diff --git a/merlin/display.py b/merlin/display.py index bf60bb50d..915773f18 100644 --- a/merlin/display.py +++ b/merlin/display.py @@ -31,34 +31,42 @@ """ Manages formatting for displaying information to the console. """ +import logging import pprint +import shutil import subprocess import time import traceback +from datetime import datetime from multiprocessing import Pipe, Process +from typing import Dict from kombu import Connection from tabulate import tabulate from merlin.ascii_art import banner_small -from merlin.config import broker, results_backend -from merlin.config.configfile import default_config_info -# TODO: make these color blind compliant -# (see https://mikemol.github.io/technique/colorblind/2018/02/11/color-safe-palette.html) +LOG = logging.getLogger("merlin") +DEFAULT_LOG_LEVEL = "INFO" + +# Colors here are chosen based on the Bang Wong color palette (https://www.nature.com/articles/nmeth.1618) +# Another useful link for comparing colors: +# https://davidmathlogic.com/colorblind/#%2356B4E9-%230072B2-%23009E73-%23D55E00-%23F0E442-%23E69F00-%23666666 ANSI_COLORS = { "RESET": "\033[0m", - "GREY": "\033[90m", - "RED": "\033[91m", - "GREEN": "\033[92m", - "YELLOW": "\033[93m", - "BLUE": "\033[94m", - "MAGENTA": "\033[95m", - "CYAN": "\033[96m", - "WHITE": "\033[97m", + "GREY": "\033[38;2;102;102;102m", + "LIGHT_BLUE": "\033[38;2;86;180;233m", + "BLUE": "\033[38;2;0;114;178m", + "GREEN": "\033[38;2;0;158;115m", + "YELLOW": "\033[38;2;240;228;66m", + "ORANGE": "\033[38;2;230;159;0m", + "RED": "\033[38;2;213;94;0m", } +# Inverse of ANSI_COLORS (useful for debugging) +COLOR_TRANSLATOR = {v: k for k, v in ANSI_COLORS.items()} + class ConnProcess(Process): """ @@ -75,7 +83,7 @@ def run(self): try: Process.run(self) self._cconn.send(None) - except Exception as e: # pylint: disable=W0718,C0103 + except Exception as e: # pylint: disable=C0103,W0703 trace_back = traceback.format_exc() self._cconn.send((e, trace_back)) # raise e # You can still rise this exception if you need to @@ -111,6 +119,8 @@ def check_server_access(sconf): def _examine_connection(server, sconf, excpts): + from merlin.config import broker, results_backend # pylint: disable=C0415 + connect_timeout = 60 try: ssl_conf = None @@ -132,7 +142,7 @@ def _examine_connection(server, sconf, excpts): if conn_check.exception: error, _ = conn_check.exception raise error - except Exception as e: # pylint: disable=W0718,C0103 + except Exception as e: # pylint: disable=C0103,W0703 print(f"{server} connection: Error") excpts[server] = e else: @@ -143,6 +153,9 @@ def display_config_info(): """ Prints useful configuration information to the console. """ + from merlin.config import broker, results_backend # pylint: disable=C0415 + from merlin.config.configfile import default_config_info # pylint: disable=C0415 + print("Merlin Configuration") print("-" * 25) print("") @@ -154,7 +167,7 @@ def display_config_info(): conf["broker server"] = broker.get_connection_string(include_password=False) sconf["broker server"] = broker.get_connection_string() conf["broker ssl"] = broker.get_ssl_config() - except Exception as e: # pylint: disable=W0718,C0103 + except Exception as e: # pylint: disable=C0103,W0703 conf["broker server"] = "Broker server error." excpts["broker server"] = e @@ -162,7 +175,7 @@ def display_config_info(): conf["results server"] = results_backend.get_connection_string(include_password=False) sconf["results server"] = results_backend.get_connection_string() conf["results ssl"] = results_backend.get_ssl_config() - except Exception as e: # pylint: disable=W0718,C0103 + except Exception as e: # pylint: disable=C0103,W0703 conf["results server"] = "No results server configured or error." excpts["results server"] = e @@ -221,22 +234,185 @@ def print_info(args): # pylint: disable=W0613 print("") -def tabulate_info(info, headers=None, color=None): - """ - Display info in a table. Colorize the table if you'd like. - Intended for use for functions outside of this file so they don't - need to import tabulate. - :param `info`: The info you want to tabulate. - :param `headers`: A string or list stating what you'd like the headers to be. - Options: "firstrow", "keys", or List[str] - :param `color`: An ANSI color. +def _display_summary(state_info: Dict[str, str], cb_help: bool): """ - # Adds the color at the start of the print - if color: - print(color, end="") + Given a dict of state info for a step, print a summary of the task states. - # \033[0m resets color to white - if headers: - print(tabulate(info, headers=headers), ANSI_COLORS["RESET"]) + :param `state_info`: A dictionary of information related to task states for a step + :param `cb_help`: True if colorblind assistance (using symbols) is needed. False otherwise. + """ + # Build a summary list of task info + print("\nSUMMARY:") + summary = [] + for key, val in state_info.items(): + label = key + # Add colorblind symbols if needed + if cb_help and "fill" in val: + label = f"{key} {val['fill']}" + # Color the label + if "color" in val: + label = f"{val['color']}{label}{ANSI_COLORS['RESET']}" + + # Grab the value associated with the label + value = None + if "count" in val and val["count"] > 0: + value = val["count"] + elif "total" in val: + value = val["total"] + elif "name" in val: + value = val["name"] + # Add the label and value as an entry to the summary + if value: + summary.append([label, value]) + + # Display the summary + print(tabulate(summary)) + print() + + +def display_status_summary( # pylint: disable=R0912 + status_obj: "Status", non_workspace_keys: set, test_mode=False # noqa: F821 +) -> Dict: + """ + Displays a high level overview of the status of a study. This includes + progress bars for each step and a summary of the number of initialized, + running, finished, cancelled, dry ran, failed, and unknown tasks. + + :param `status_obj`: A Status object + :param `non_workspace_keys`: A set of keys in requested_statuses that are not workspace keys. + This will be set("parameters", "task_queue", "worker_name) + :param `test_mode`: If True, don't print anything and just return a dict of all the state info for each step + :returns: A dict that's empty usually. If ran in test_mode it will be a dict of state_info for every step. + """ + all_state_info = {} + if not test_mode: + print(f"{ANSI_COLORS['YELLOW']}Status for {status_obj.workspace} as of {datetime.now()}:{ANSI_COLORS['RESET']}") + terminal_size = shutil.get_terminal_size() + progress_bar_width = terminal_size.columns // 4 + + for sstep in status_obj.step_tracker["started_steps"]: + # This dict will keep track of the number of tasks at each status + state_info = { + "FINISHED": {"count": 0, "color": ANSI_COLORS["GREEN"], "fill": "█"}, + "CANCELLED": {"count": 0, "color": ANSI_COLORS["YELLOW"], "fill": "/"}, + "FAILED": {"count": 0, "color": ANSI_COLORS["RED"], "fill": "⣿"}, + "UNKNOWN": {"count": 0, "color": ANSI_COLORS["GREY"], "fill": "?"}, + "INITIALIZED": {"count": 0, "color": ANSI_COLORS["LIGHT_BLUE"]}, + "RUNNING": {"count": 0, "color": ANSI_COLORS["BLUE"]}, + "DRY_RUN": {"count": 0, "color": ANSI_COLORS["ORANGE"], "fill": "\\"}, + "TOTAL_TASKS": {"total": status_obj.tasks_per_step[sstep]}, + } + + num_completed_tasks = 0 + # Loop through each entry for the step (if there's no parameters there will just be one entry) + for real_step_name in status_obj.real_step_name_map[sstep]: + # Grab the statuses for this step + overall_step_info = status_obj.requested_statuses[real_step_name] + + # If this was a non-local run we should have a task queue and worker name to add to state_info + if "task_queue" in overall_step_info: + state_info["TASK_QUEUE"] = {"name": overall_step_info["task_queue"]} + if "worker_name" in overall_step_info: + state_info["WORKER_NAME"] = {"name": overall_step_info["worker_name"]} + + # Loop through all workspaces for this step (if there's no samples for this step it'll just be one path) + for sub_step_workspace, task_status_info in overall_step_info.items(): + # We've already handled the non-workspace keys that we need so ignore them here + if sub_step_workspace in non_workspace_keys: + continue + + state_info[task_status_info["status"]]["count"] += 1 + # Increment the number of completed tasks (not running or initialized) + if task_status_info["status"] not in ("INITIALIZED", "RUNNING"): + num_completed_tasks += 1 + + if test_mode: + all_state_info[sstep] = state_info + else: + # Display the progress bar and summary for the step + print(f"\n{sstep}\n") + display_progress_bar( + num_completed_tasks, + status_obj.tasks_per_step[sstep], + state_info=state_info, + suffix="Complete", + length=progress_bar_width, + cb_help=status_obj.args.cb_help, + ) + _display_summary(state_info, status_obj.args.cb_help) + print("-" * (terminal_size.columns // 2)) + + # For each unstarted step, print an empty progress bar + for ustep in status_obj.step_tracker["unstarted_steps"]: + if test_mode: + all_state_info[ustep] = "UNSTARTED" + else: + print(f"\n{ustep}\n") + display_progress_bar(0, 100, suffix="Complete", length=progress_bar_width) + print(f"\n{ustep} has not started yet.\n") + print("-" * (terminal_size.columns // 2)) + + return all_state_info + + +# Credit to this stack overflow post: https://stackoverflow.com/a/34325723 +def display_progress_bar( # pylint: disable=R0913,R0914 + current, + total, + state_info=None, + prefix="", + suffix="", + decimals=1, + length=80, + fill="█", + print_end="\n", + color=None, + cb_help=False, +): + """ + Prints a progress bar based on current and total. + + :param `current`: current number (Int) + :param `total`: total number (Int) + :param `state_info`: information about the state of tasks (Dict) (overrides color) + :param `prefix`: prefix string (Str) + :param `suffix`: suffix string (Str) + :param `decimals`: positive number of decimals in percent complete (Int) + :param `length`: character length of bar (Int) + :param `fill`: bar fill character (Str) + :param `print_end`: end character (e.g. "\r", "\r\n") (Str) + :param `color`: color of the progress bar (ANSI Str) (overridden by state_info) + :param `cb_help`: true if color blind help is needed; false otherwise (Bool) + """ + # Set the color of the bar + if color and color in ANSI_COLORS: + fill = f"{color}{fill}{ANSI_COLORS['RESET']}" + + # Get the percentage done and the total fill length of the bar + percent = ("{0:." + str(decimals) + "f}").format(100 * (current / float(total))) + total_filled_length = int(length * current // total) + + # Print a progress bar based on state of the study + if state_info: + print(f"\r{prefix} |", end="") + for key, val in state_info.items(): + # Only fill bar with completed tasks + if key in ("INITIALIZED", "RUNNING", "TASK_QUEUE", "WORKER_NAME", "TOTAL_TASKS"): + continue + + # Get the length to fill for this specific state + partial_filled_length = int(length * val["count"] // total) + + if partial_filled_length > 0: + if cb_help: + fill = val["fill"] + progress_bar = fill * partial_filled_length + print(f"{val['color']}{progress_bar}", end="") + + # The remaining bar represents the number of tasks still incomplete + remaining_bar = "-" * (length - total_filled_length) + print(f'{ANSI_COLORS["RESET"]}{remaining_bar}| {percent}% {suffix}', end=print_end) + # Print a normal progress bar else: - print(tabulate(info), ANSI_COLORS["RESET"]) + progress_bar = fill * total_filled_length + "-" * (length - total_filled_length) + print(f"\r{prefix} |{progress_bar}| {percent}% {suffix}", end=print_end) diff --git a/merlin/main.py b/merlin/main.py index d3fe4b80a..57f53f099 100644 --- a/merlin/main.py +++ b/merlin/main.py @@ -55,8 +55,9 @@ from merlin.server.server_commands import config_server, init_server, restart_server, start_server, status_server, stop_server from merlin.spec.expansion import RESERVED, get_spec_with_expansion from merlin.spec.specification import MerlinSpec +from merlin.study.status import Status from merlin.study.study import MerlinStudy -from merlin.utils import ARRAY_FILE_FORMATS +from merlin.utils import ARRAY_FILE_FORMATS, verify_dirpath, verify_filepath LOG = logging.getLogger("merlin") @@ -73,38 +74,6 @@ def error(self, message): sys.exit(2) -def verify_filepath(filepath: str) -> str: - """ - Verify that the filepath argument is a valid - file. - - :param [str] `filepath`: the path of a file - - :return: the verified absolute filepath with expanded environment variables. - :rtype: str - """ - filepath = os.path.abspath(os.path.expandvars(os.path.expanduser(filepath))) - if not os.path.isfile(filepath): - raise ValueError(f"'{filepath}' is not a valid filepath") - return filepath - - -def verify_dirpath(dirpath: str) -> str: - """ - Verify that the dirpath argument is a valid - directory. - - :param [str] `dirpath`: the path of a directory - - :return: returns the absolute path with expanded environment vars for a given dirpath. - :rtype: str - """ - dirpath: str = os.path.abspath(os.path.expandvars(os.path.expanduser(dirpath))) - if not os.path.isdir(dirpath): - raise ValueError(f"'{dirpath}' is not a valid directory path") - return dirpath - - def parse_override_vars( variables_list: Optional[List[str]], ) -> Optional[Dict[str, Union[str, int]]]: @@ -249,17 +218,56 @@ def purge_tasks(args): def query_status(args): """ - CLI command for querying queue status. + CLI command for querying status of studies. + Based on the parsed CLI args, construct either a Status object or a DetailedStatus object + and display the appropriate output. + Object mapping is as follows: + merlin status -> Status object ; merlin detailed-status -> DetailedStatus object - :param 'args': parsed CLI arguments + :param `args`: parsed CLI arguments """ print(banner_small) - spec, _ = get_merlin_spec_with_override(args) - ret = router.query_status(args.task_server, spec, args.steps) - for name, jobs, consumers in ret: - print(f"{name:30} - Workers: {consumers:10} - Queued Tasks: {jobs:10}") - if args.csv is not None: - router.dump_status(ret, args.csv) + + # Ensure task server is valid + if args.task_server != "celery": + raise ValueError("Currently the only supported task server is celery.") + + # Make sure dump is valid if provided + if args.dump and (not args.dump.endswith(".csv") and not args.dump.endswith(".json")): + raise ValueError("The --dump option takes a filename that must end with .csv or .json") + + # Establish whether the argument provided by the user was a spec file or a study directory + spec_display = False + try: + file_or_ws = verify_filepath(args.spec_or_workspace) + spec_display = True + except ValueError: + try: + file_or_ws = verify_dirpath(args.spec_or_workspace) + except ValueError: + LOG.error(f"The file or directory path {args.spec_or_workspace} does not exist.") + return None + + # If we're loading status based on a spec, load in the spec provided + if spec_display: + args.specification = file_or_ws + spec_provided, _ = get_merlin_spec_with_override(args) + args.spec_provided = spec_provided + + # Get either a Status object or DetailedStatus object + # if args.detailed: + # status_obj = DetailedStatus(args, spec_display, file_or_ws) + # else: + # status_obj = Status(args, spec_display, file_or_ws) + status_obj = Status(args, spec_display, file_or_ws) # The above can be uncommented when we add DetailedStatus + + # Handle output appropriately + if args.dump: + status_obj.dump() + else: + status_obj.display() + + return None def query_workers(args): @@ -885,29 +893,32 @@ def generate_diagnostic_parsers(subparsers: ArgumentParser) -> None: Merlin job. """ # merlin status - status: ArgumentParser = subparsers.add_parser( + status_cmd: ArgumentParser = subparsers.add_parser( "status", - help="List server stats (name, number of tasks to do, \ - number of connected workers) for a workflow spec.", + help="Display a summary of the status of a study.", ) - status.set_defaults(func=query_status) - status.add_argument("specification", type=str, help="Path to a Merlin YAML spec file") - status.add_argument( - "--steps", - nargs="+", - type=str, - dest="steps", - default=["all"], - help="The specific steps in the YAML file you want to query", + status_cmd.set_defaults(func=query_status, detailed=False) + status_cmd.add_argument("spec_or_workspace", type=str, help="Path to a Merlin YAML spec file or a launched Merlin study") + status_cmd.add_argument( + "--cb-help", action="store_true", help="Colorblind help; uses different symbols to represent different statuses" + ) + status_cmd.add_argument( + "--dump", type=str, help="Dump the status to a file. Provide the filename (must be .csv or .json).", default=None + ) + status_cmd.add_argument( + "--no-prompts", + action="store_true", + help="Ignore any prompts provided. This will default to the latest study \ + if you provide a spec file rather than a study workspace.", ) - status.add_argument( + status_cmd.add_argument( "--task_server", type=str, default="celery", help="Task server type.\ Default: %(default)s", ) - status.add_argument( + status_cmd.add_argument( "--vars", action="store", dest="variables", @@ -917,7 +928,6 @@ def generate_diagnostic_parsers(subparsers: ArgumentParser) -> None: help="Specify desired Merlin variable values to override those found in the specification. Space-delimited. " "Example: '--vars LEARN=path/to/new_learn.py EPOCHS=3'", ) - status.add_argument("--csv", type=str, help="csv file to dump status report to", default=None) # merlin info info: ArgumentParser = subparsers.add_parser( diff --git a/merlin/spec/specification.py b/merlin/spec/specification.py index 1cb26d512..0cc5cbe12 100644 --- a/merlin/spec/specification.py +++ b/merlin/spec/specification.py @@ -40,12 +40,13 @@ from copy import deepcopy from datetime import timedelta from io import StringIO +from typing import Dict import yaml from maestrowf.specification import YAMLSpecification from merlin.spec import all_keys, defaults -from merlin.utils import repr_timedelta +from merlin.utils import load_array_file, needs_merlin_expansion, repr_timedelta LOG = logging.getLogger(__name__) @@ -623,3 +624,39 @@ def get_worker_names(self): for worker in self.merlin["resources"]["workers"]: result.append(worker) return result + + def get_tasks_per_step(self) -> Dict[str, int]: + """ + Get the number of tasks needed to complete each step, formatted as a dictionary. + :returns: A dict where the keys are the step names and the values are the number of tasks required for that step + """ + # Get the number of samples used + samples = [] + if self.merlin["samples"] and self.merlin["samples"]["file"]: + samples = load_array_file(self.merlin["samples"]["file"]) + num_samples = len(samples) + + # Get the column labels, the parameter labels, the number of parameters, and the steps in the study + if num_samples > 0: + column_labels = self.merlin["samples"]["column_labels"] + parameter_labels = list(self.get_parameters().labels.keys()) + num_params = self.get_parameters().length + study_steps = self.get_study_steps() + + tasks_per_step = {} + for step in study_steps: + cmd = step.__dict__["run"]["cmd"] + restart_cmd = step.__dict__["run"]["restart"] + + # Default number of tasks for a step is 1 + tasks_per_step[step.name] = 1 + + # If this step uses parameters, we'll at least have a num_params number of tasks to complete + if needs_merlin_expansion(cmd, restart_cmd, parameter_labels, include_sample_keywords=False): + tasks_per_step[step.name] = num_params + + # If merlin expansion is needed with column labels, this step uses samples + if num_samples > 0 and needs_merlin_expansion(cmd, restart_cmd, column_labels): + tasks_per_step[step.name] *= num_samples + + return tasks_per_step diff --git a/merlin/study/celeryadapter.py b/merlin/study/celeryadapter.py index f9c04b1e3..cb2963f40 100644 --- a/merlin/study/celeryadapter.py +++ b/merlin/study/celeryadapter.py @@ -38,6 +38,8 @@ import time from contextlib import suppress +from tabulate import tabulate + from merlin.study.batch import batch_check_parallel, batch_worker_launch from merlin.utils import apply_list_of_regex, check_machines, get_procs, get_yaml_var, is_running @@ -256,7 +258,7 @@ def query_celery_workers(spec_worker_names, queues, workers_regex): # Display the output table LOG.info("Found these connected workers:") - tabulate_info(table, headers=["Workers", "Queues"]) + print(tabulate(table, headers=["Workers", "Queues"])) print() diff --git a/merlin/study/status.py b/merlin/study/status.py new file mode 100644 index 000000000..ab8b8af35 --- /dev/null +++ b/merlin/study/status.py @@ -0,0 +1,484 @@ +############################################################################### +# Copyright (c) 2023, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.0 +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### +"""This module handles all the functionality of getting the statuses of studies""" +import json +import logging +import os +import re +from argparse import Namespace +from datetime import datetime +from glob import glob +from typing import Dict, List, Optional, Tuple + +from filelock import FileLock, Timeout +from tabulate import tabulate + +from merlin.common.dumper import dump_handler +from merlin.display import ANSI_COLORS, display_status_summary +from merlin.spec.expansion import get_spec_with_expansion +from merlin.utils import dict_deep_merge, verify_dirpath, ws_time_to_dt + + +LOG = logging.getLogger(__name__) +VALID_STATUS_FILTERS = ("INITIALIZED", "RUNNING", "FINISHED", "FAILED", "CANCELLED", "DRY_RUN", "UNKNOWN") +VALID_RETURN_CODES = ("SUCCESS", "SOFT_FAIL", "HARD_FAIL", "STOP_WORKERS", "RESTART", "RETRY", "DRY_SUCCESS", "UNRECOGNIZED") +VALID_EXIT_FILTERS = ("E", "EXIT") +ALL_VALID_FILTERS = VALID_STATUS_FILTERS + VALID_RETURN_CODES + VALID_EXIT_FILTERS + ("MAX_TASKS",) +NON_WORKSPACE_KEYS = set(["task_queue", "worker_name"]) + + +class Status: + """ + This class handles everything to do with status besides displaying it. + Display functionality is handled in display.py. + """ + + def __init__(self, args: Namespace, spec_display: bool, file_or_ws: str): + # Save the args to this class instance and check if the steps filter was given + self.args = args + + # Load in the workspace path and spec object + if spec_display: + self.workspace, self.spec = self._load_from_spec(file_or_ws) + else: + self.workspace = file_or_ws + self.spec = self._load_from_workspace() + + # Verify the filter args (this will only do something for DetailedStatus) + self._verify_filter_args() + + # Create a step tracker that will tell us which steps we'll display that have started/not started + self.step_tracker = self.get_steps_to_display() + + # Create a tasks per step mapping in order to give accurate totals for each step + self.tasks_per_step = self.spec.get_tasks_per_step() + + # This attribute will store a map between the overall step name and the real step names + # that are created with parameters (e.g. step name is hello and uses a "GREET: hello" parameter + # so the real step name is hello_GREET.hello) + self.real_step_name_map = {} + + # Variable to store the statuses that the user wants + self.requested_statuses = {} + self.load_requested_statuses() + + def _verify_filter_args(self): + """ + This is an abstract method since we'll need to verify filter args for DetailedStatus + but not for Status. + """ + + def _get_latest_study(self, studies: List[str]) -> str: + """ + Given a list of studies, get the latest one. + + :param `studies`: A list of studies to sort through + :returns: The latest study in the list provided + """ + # We can assume the newest study is the last one to be added to the list of potential studies + newest_study = studies[-1] + newest_timestring = newest_study[-15:] + newest_study_date = ws_time_to_dt(newest_timestring) + + # Check that the newest study somehow isn't the last entry + for study in studies: + temp_timestring = study[-15:] + date_to_check = ws_time_to_dt(temp_timestring) + if date_to_check > newest_study_date: + newest_study = study + newest_study_date = date_to_check + + return newest_study + + def _obtain_study(self, study_output_dir: str, num_studies: int, potential_studies: List[Tuple[int, str]]) -> str: + """ + Grab the study that the user wants to view the status of based on a list of potential studies provided. + + :param `study_output_dir`: A string representing the output path of a study; equivalent to $(OUTPUT_PATH) + :param `num_studies`: The number of potential studies we found + :param `potential_studies`: The list of potential studies we found; + Each entry is of the form (index, potential_study_name) + :returns: A directory path to the study that the user wants + to view the status of ("study_output_dir/selected_potential_study") + """ + study_to_check = f"{study_output_dir}/" + if num_studies == 0: + raise ValueError("Could not find any potential studies.") + if num_studies > 1: + # Get the latest study + if self.args.no_prompts: + LOG.info("Choosing the latest study...") + potential_studies = [study for _, study in potential_studies] + latest_study = self._get_latest_study(potential_studies) + LOG.info(f"Chose {latest_study}") + study_to_check += latest_study + # Ask the user which study to view + else: + print(f"Found {num_studies} potential studies:") + print(tabulate(potential_studies, headers=["Index", "Study Name"])) + prompt = "Which study would you like to view the status of? Use the index on the left: " + index = -1 + while index < 1 or index > num_studies: + try: + index = int(input(prompt)) + if index < 1 or index > num_studies: + raise ValueError + except ValueError: + print( + f"{ANSI_COLORS['RED']}Input must be an integer between 1 " + f"and {num_studies}.{ANSI_COLORS['RESET']}" + ) + prompt = "Enter a different index: " + study_to_check += potential_studies[index - 1][1] + else: + # Only one study was found so we'll just assume that's the one the user wants + study_to_check += potential_studies[0][1] + + return study_to_check + + def _load_from_spec(self, filepath: str) -> Tuple[str, "MerlinSpec"]: # noqa: F821 pylint: disable=R0914 + """ + Get the desired workspace from the user and load up it's yaml spec + for further processing. + + :param `filepath`: The filepath to a spec given by the user + :returns: The workspace of the study we'll check the status for and a MerlinSpec + object loaded in from the workspace's merlin_info subdirectory. + """ + # Get the output path of the study that was given to us + # Case where the output path is left out of the spec file + if self.args.spec_provided.output_path == "": + output_path = os.path.dirname(filepath) + # Case where output path is absolute + elif self.args.spec_provided.output_path.startswith("/"): + output_path = self.args.spec_provided.output_path + # Case where output path is relative to the specroot + else: + output_path = f"{os.path.dirname(filepath)}/{self.args.spec_provided.output_path}" + + study_output_dir = verify_dirpath(output_path) + + # Build a list of potential study output directories + study_output_subdirs = next(os.walk(study_output_dir))[1] + timestamp_regex = r"\d{8}-\d{6}" + potential_studies = [] + num_studies = 0 + for subdir in study_output_subdirs: + match = re.search(rf"{self.args.spec_provided.name}_{timestamp_regex}", subdir) + if match: + potential_studies.append((num_studies + 1, subdir)) + num_studies += 1 + + # Obtain the correct study to view the status of based on the list of potential studies we just built + study_to_check = self._obtain_study(study_output_dir, num_studies, potential_studies) + + # Verify the directory that the user selected is a merlin study output directory + if "merlin_info" not in next(os.walk(study_to_check))[1]: + raise ValueError( + f"The merlin_info subdirectory was not found. {study_to_check} may not be a Merlin study output directory." + ) + + # Grab the spec saved to the merlin info directory in case something + # in the current spec has changed since starting the study + expanded_spec_options = glob(f"{study_to_check}/merlin_info/*.expanded.yaml") + if len(expanded_spec_options) > 1: + raise ValueError(f"Multiple expanded spec options found in the {study_to_check}/merlin_info/ directory") + if len(expanded_spec_options) < 1: + raise ValueError(f"No expanded spec options found in the {study_to_check}/merlin_info/ directory") + actual_spec = get_spec_with_expansion(expanded_spec_options[0]) + + return study_to_check, actual_spec + + def _load_from_workspace(self) -> "MerlinSpec": # noqa: F821 + """ + Create a MerlinSpec object based on the spec file in the workspace. + + :returns: A MerlinSpec object loaded from the workspace provided by the user + """ + # Grab the spec file from the directory provided + info_dir = verify_dirpath(f"{self.workspace}/merlin_info") + spec_file = "" + for _, _, files in os.walk(info_dir): + for f in files: # pylint: disable=C0103 + if f.endswith(".expanded.yaml"): + spec_file = f"{info_dir}/{f}" + break + break + + # Make sure we got a spec file and load it in + if not spec_file: + LOG.error(f"Spec file not found in {info_dir}. Cannot display status.") + return None + spec = get_spec_with_expansion(spec_file) + + return spec + + def _create_step_tracker(self, steps_to_check: List[str]) -> Dict[str, List[str]]: + """ + Creates a dictionary of started and unstarted steps that we + will display the status for. + + :param `steps_to_check`: A list of steps to view the status of + :returns: A dictionary mapping of started and unstarted steps. Values are lists of step names. + """ + step_tracker = {"started_steps": [], "unstarted_steps": []} + started_steps = next(os.walk(self.workspace))[1] + started_steps.remove("merlin_info") + + for sstep in started_steps: + if sstep in steps_to_check: + step_tracker["started_steps"].append(sstep) + steps_to_check.remove(sstep) + step_tracker["unstarted_steps"] = steps_to_check + + return step_tracker + + def get_steps_to_display(self) -> Dict[str, List[str]]: + """ + Generates a list of steps to display the status for based on information + provided to the merlin status command by the user. + + :returns: A dictionary of started and unstarted steps for us to display the status of + """ + existing_steps = self.spec.get_study_step_names() + + LOG.debug(f"existing steps: {existing_steps}") + LOG.debug("Building step tracker based on existing steps...") + + # Filter the steps to display status for by started/unstarted + step_tracker = self._create_step_tracker(existing_steps) + + return step_tracker + + @property + def num_requested_statuses(self): + """ + Count the number of task statuses in a the requested_statuses dict. + We need to ignore non workspace keys when we count. + """ + num_statuses = 0 + for status_info in self.requested_statuses.values(): + num_statuses += len(status_info.keys() - NON_WORKSPACE_KEYS) + return num_statuses + + def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict[str, List[str]]: + """ + Given a step workspace and the name of the step, read in all the statuses + for the step and return them in a dict. + + :param `step_workspace`: The path to the step we're going to read statuses from + :param `started_step_name`: The name of the started step that we're getting statuses from + :returns: A dict of statuses for the given step + """ + step_statuses = {} + num_statuses_read = 0 + + # Traverse the step workspace and look for MERLIN_STATUS files + for root, _, files in os.walk(step_workspace): + if "MERLIN_STATUS.json" in files: + status_filepath = f"{root}/MERLIN_STATUS.json" + lock = FileLock(f"{root}/status.lock") # pylint: disable=E0110 + statuses_read = read_status(status_filepath, lock) + + # Count the number of statuses we just read + for step_name, status_info in statuses_read.items(): + if started_step_name not in self.real_step_name_map: + self.real_step_name_map[started_step_name] = [step_name] + else: + if step_name not in self.real_step_name_map[started_step_name]: + self.real_step_name_map[started_step_name].append(step_name) + num_statuses_read += len(status_info.keys() - NON_WORKSPACE_KEYS) + + # Merge the statuses we read with the dict tracking all statuses for this step + dict_deep_merge(step_statuses, statuses_read) + + # If we've read all the statuses then we're done + if num_statuses_read == self.tasks_per_step[started_step_name]: + break + # This shouldn't get hit + if num_statuses_read > self.tasks_per_step[started_step_name]: + LOG.error( + f"Read {num_statuses_read} statuses when there should " + f"only be {self.tasks_per_step[started_step_name]} tasks in total." + ) + break + + return step_statuses + + def load_requested_statuses(self): + """ + Populate the requested_statuses dict with the statuses from the study. + """ + LOG.info(f"Reading task statuses from {self.workspace}") + + # Read in all statuses from the started steps the user wants to see + for sstep in self.step_tracker["started_steps"]: + step_workspace = f"{self.workspace}/{sstep}" + step_statuses = self.get_step_statuses(step_workspace, sstep) + dict_deep_merge(self.requested_statuses, step_statuses) + + # Count how many statuses in total that we just read in + LOG.info(f"Read in {self.num_requested_statuses} statuses.") + + def display(self, test_mode=False) -> Dict: + """ + Displays the high level summary of the status. + + :param `test_mode`: If true, run this in testing mode and don't print any output + :returns: A dict that will be empty if test_mode is False. Otherwise, the dict will + contain the status info that would be displayed. + """ + return display_status_summary(self, NON_WORKSPACE_KEYS, test_mode=test_mode) + + def format_json_dump(self, date: datetime) -> Dict: + """ + Build the dict of statuses to dump to the json file. + + :param `date`: A timestamp for us to mark when this status occurred + :returns: A dictionary that's ready to dump to a json outfile + """ + # Statuses are already in json format so we'll just add a timestamp for the dump here + return {date: self.requested_statuses} + + def format_csv_dump(self, date: datetime) -> Dict: + """ + Add the timestamp to the statuses to write. + + :param `date`: A timestamp for us to mark when this status occurred + :returns: A dict equivalent of formatted statuses with a timestamp entry at the start of the dict. + """ + # Reformat the statuses to a new dict where the keys are the column labels and rows are the values + statuses_to_write = self.format_status_for_display() + + # Add date entries as the first column then update this dict with the statuses we just reformatted + statuses_with_timestamp = {"time_of_status": [date] * len(statuses_to_write["step_name"])} + statuses_with_timestamp.update(statuses_to_write) + + return statuses_with_timestamp + + def dump(self): + """ + Dump the status information to a file. + """ + # Get a timestamp for this dump + date = datetime.now().strftime("%Y-%m-%d %H:%M:%S") + + # Handle different file types + if self.args.dump.endswith(".csv"): + # Build the lists of information/labels we'll need + dump_info = self.format_csv_dump(date) + elif self.args.dump.endswith(".json"): + # Build the dict of info to dump to the json file + dump_info = self.format_json_dump(date) + + # Dump the information + dump_handler(self.args.dump, dump_info) + + def format_status_for_display(self) -> Dict: + """ + Reformat our statuses to display so they can use Maestro's status renderer layouts. + + :returns: A formatted dictionary where each key is a column and the values are the rows + of information to display for that column. + """ + reformatted_statuses = { + "step_name": [], + "step_workspace": [], + "status": [], + "return_code": [], + "elapsed_time": [], + "run_time": [], + "restarts": [], + "task_queue": [], + "worker_name": [], + } + + # Loop through all statuses + for step_name, overall_step_info in self.requested_statuses.items(): + # Get the number of statuses for this step so we know how many entries there should be + num_statuses = len(overall_step_info.keys() - NON_WORKSPACE_KEYS) + + # Loop through information for each step + for step_info_key, step_info_value in overall_step_info.items(): + # Format non-workspace keys (task_queue and worker_name) + if step_info_key in NON_WORKSPACE_KEYS: + # Set the val_to_add value based on if a value exists for the key + val_to_add = step_info_value if step_info_value else "-------" + # Add the val_to_add entry for each row + key_entries = [val_to_add] * num_statuses + reformatted_statuses[step_info_key].extend(key_entries) + + # Format workspace keys + else: + # Put the step name and workspace in each entry + reformatted_statuses["step_name"].append(step_name) + reformatted_statuses["step_workspace"].append(step_info_key) + + # Add the rest of the information for each task (status, return code, elapsed & run time, num restarts) + for key, val in step_info_value.items(): + reformatted_statuses[key].append(val) + + # For local runs, there will be no task queue or worker name so delete these entries + for celery_specific_key in ("task_queue", "worker_name"): + if not reformatted_statuses[celery_specific_key]: + del reformatted_statuses[celery_specific_key] + + return reformatted_statuses + + +class DetailedStatus(Status): + pass + + +def read_status(status_filepath: str, lock: FileLock, display_fnf_message: Optional[bool] = True) -> Dict: + """ + Locks the status file for reading and returns its contents. + + :param `status_filepath`: The path to the status file that we'll read from + :param `lock`: A FileLock object that we'll use to lock the file + :param `display_fnf_message`: If True, display the file not found warning. Otherwise don't. + :returns: A dict of the contents in the status file + """ + try: + # The status files will need locks when reading to avoid race conditions + with lock.acquire(timeout=10): + with open(status_filepath, "r") as status_file: + statuses_read = json.load(status_file) + # Handle timeouts + except Timeout: + LOG.warning(f"Timed out when trying to read status from {status_filepath}") + statuses_read = {} + # Handle FNF errors + except FileNotFoundError: + if display_fnf_message: + LOG.warning(f"Could not find {status_filepath}") + statuses_read = {} + + return statuses_read From 895a6bc6f282ac1496d353de3edc1537cc6ef451 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Fri, 18 Aug 2023 11:09:32 -0700 Subject: [PATCH 03/13] add tests for merlin status --- requirements/dev.txt | 1 + tests/integration/run_tests.py | 7 +- tests/unit/study/__init__.py | 29 ++ .../status_test_files/combine_status_files.py | 113 +++++ .../study/status_test_files/shared_tests.py | 309 ++++++++++++ .../status_test_files/status_test_spec.yaml | 79 +++ .../dummy.txt | 1 + .../cancel_step/MERLIN_STATUS.json | 13 + .../cancel_step/status.lock | 0 .../fail_step/MERLIN_STATUS.json | 13 + .../fail_step/status.lock | 0 .../MERLIN_STATUS.json | 13 + .../GREET.hello.LEAVE.goodbye/status.lock | 0 .../GREET.hola.LEAVE.adios/MERLIN_STATUS.json | 13 + .../GREET.hola.LEAVE.adios/status.lock | 0 .../just_samples/00/status.lock | 0 .../just_samples/01/status.lock | 0 .../just_samples/02/status.lock | 0 .../just_samples/03/status.lock | 0 .../just_samples/04/status.lock | 0 .../just_samples/MERLIN_STATUS.json | 41 ++ .../just_samples/status.lock | 0 .../merlin_info/samples.csv | 5 + .../status_test_spec.expanded.yaml | 112 +++++ .../merlin_info/status_test_spec.orig.yaml | 80 +++ .../merlin_info/status_test_spec.partial.yaml | 118 +++++ .../GREET.hello/00/status.lock | 0 .../GREET.hello/01/status.lock | 0 .../GREET.hello/02/status.lock | 0 .../GREET.hello/03/status.lock | 0 .../GREET.hello/04/status.lock | 0 .../GREET.hello/MERLIN_STATUS.json | 41 ++ .../GREET.hello/status.lock | 0 .../GREET.hola/00/status.lock | 0 .../GREET.hola/01/status.lock | 0 .../GREET.hola/02/status.lock | 0 .../GREET.hola/03/status.lock | 0 .../GREET.hola/04/status.lock | 0 .../GREET.hola/MERLIN_STATUS.json | 41 ++ .../params_and_samples/GREET.hola/status.lock | 0 .../status_test_variables.py | 471 ++++++++++++++++++ tests/unit/study/test_status.py | 274 ++++++++++ 42 files changed, 1770 insertions(+), 4 deletions(-) create mode 100644 tests/unit/study/__init__.py create mode 100644 tests/unit/study/status_test_files/combine_status_files.py create mode 100644 tests/unit/study/status_test_files/shared_tests.py create mode 100644 tests/unit/study/status_test_files/status_test_spec.yaml create mode 100644 tests/unit/study/status_test_files/status_test_study_20230713-000000/dummy.txt create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/00/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/01/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/02/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/03/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/04/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/samples.csv create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.expanded.yaml create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.orig.yaml create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.partial.yaml create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/00/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/01/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/02/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/03/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/04/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/00/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/01/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/02/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/03/status.lock create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/04/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json create mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/status.lock create mode 100644 tests/unit/study/status_test_files/status_test_variables.py create mode 100644 tests/unit/study/test_status.py diff --git a/requirements/dev.txt b/requirements/dev.txt index 9321694f8..895a89249 100644 --- a/requirements/dev.txt +++ b/requirements/dev.txt @@ -10,3 +10,4 @@ twine sphinx>=2.0.0 alabaster johnnydep +deepdiff diff --git a/tests/integration/run_tests.py b/tests/integration/run_tests.py index 82a4fdd92..61b6b99ed 100644 --- a/tests/integration/run_tests.py +++ b/tests/integration/run_tests.py @@ -39,10 +39,9 @@ from contextlib import suppress from subprocess import TimeoutExpired, run -# Pylint complains that we didn't install this module but it's defined locally so ignore -from test_definitions import OUTPUT_DIR, define_tests # pylint: disable=E0401 +from tabulate import tabulate -from merlin.display import tabulate_info +from tests.integration.test_definitions import OUTPUT_DIR, define_tests def get_definition_issues(test): @@ -320,7 +319,7 @@ def display_tests(tests): test_table = [(i + 1, test_names[i]) for i in range(len(test_names))] test_table.insert(0, ("ID", "Test Name")) print() - tabulate_info(test_table, headers="firstrow") + print(tabulate(test_table, headers="firstrow")) print() diff --git a/tests/unit/study/__init__.py b/tests/unit/study/__init__.py new file mode 100644 index 000000000..a48da8d86 --- /dev/null +++ b/tests/unit/study/__init__.py @@ -0,0 +1,29 @@ +############################################################################### +# Copyright (c) 2023, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.2. +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### \ No newline at end of file diff --git a/tests/unit/study/status_test_files/combine_status_files.py b/tests/unit/study/status_test_files/combine_status_files.py new file mode 100644 index 000000000..2795da2ab --- /dev/null +++ b/tests/unit/study/status_test_files/combine_status_files.py @@ -0,0 +1,113 @@ +############################################################################### +# Copyright (c) 2023, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.2. +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### +""" +Script to combine all status files from a study into one + +Usage: python combine_status_files.py .json [-d] +""" +import argparse +import json +import logging +import os +import sys + +from merlin.utils import dict_deep_merge, verify_dirpath, verify_filepath + + +LOG = logging.getLogger(__name__) + + +def combine_status_files(workspace, outfile, delete): + """ + Traverse the workspace looking for status files and condense their contents + into one outfile. + + :param `workspace`: The workspace to search for status files in + :param `outfile`: The output file that we're dumping statuses to + :param `delete`: If true, delete status files found in workspace after reading their contents + """ + condensed_statuses = {} + + # Traverse the step workspace and look for MERLIN_STATUS files + for root, _, files in os.walk(workspace): + # We only care about this level of the workspace if there's a MERLIN_STATUS file + if "MERLIN_STATUS.json" in files: + # Read in the status + status_filepath = f"{root}/MERLIN_STATUS.json" + with open(status_filepath, "r") as status_file: + status = json.load(status_file) + + # Merge the status with the other statuses (and delete the status file if necessary) + dict_deep_merge(condensed_statuses, status) + if delete: + os.remove(status_filepath) + + # Dump all of the statuses to the outfile + with open(outfile, "w") as condensed_status_file: + json.dump(condensed_statuses, condensed_status_file) + + +def main(): + """ + Build the argparse object, verify the arguments provided by the user, and then + call combine_status_files + """ + # Create argument parser and parse the args + parser = argparse.ArgumentParser(prog="combine-statuses", description="Combine the status files from a workspace") + parser.add_argument("workspace", action="store", help="the workspace containing status files") + parser.add_argument("outfile", action="store", help="a json filepath to dump the statuses to") + parser.add_argument( + "-d", "--delete", dest="delete", action="store_true", help="delete the status files after reading their information" + ) + args = parser.parse_args() + + # Verify workspace value provided and get absolute path + try: + workspace = verify_dirpath(args.workspace) + except ValueError: + LOG.error(f"The directory path {args.workspace} does not exist.") + sys.exit() + + # Verify outfile path + if not args.outfile.endswith(".json"): + LOG.error("The outfile must be json.") + sys.exit() + try: + outfile = verify_filepath(args.outfile) + except ValueError: + LOG.error(f"The file path {args.outfile} does not exist.") + sys.exit() + + # Combine the status files + combine_status_files(workspace, outfile, args.delete) + + +if __name__ == "__main__": + main() diff --git a/tests/unit/study/status_test_files/shared_tests.py b/tests/unit/study/status_test_files/shared_tests.py new file mode 100644 index 000000000..601fcd5d2 --- /dev/null +++ b/tests/unit/study/status_test_files/shared_tests.py @@ -0,0 +1,309 @@ +############################################################################### +# Copyright (c) 2023, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.2. +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### +""" +This module contains all shared tests needed for testing both +the Status object and the DetailedStatus object. +""" +import csv +import json +import os +from io import StringIO +from time import sleep +from typing import Dict, List, Tuple, Union +from unittest.mock import patch + +from deepdiff import DeepDiff +from tabulate import tabulate + +from merlin.display import ANSI_COLORS +from merlin.study.status import DetailedStatus, Status +from tests.unit.study.status_test_files import status_test_variables + + +def assert_correct_attribute_creation(status_obj: Union[Status, DetailedStatus]): + """ + Ensure that attributes of the Status/DetailedStatus class are initiated correctly. + This covers the get_steps_to_display, _create_step_tracker, spec.get_tasks_per_step, + get_requested_statuses, get_step_statuses, and num_requested_statuses methods. + + :param `status_obj`: The Status or DetailedStatus object that we're testing + """ + # Ensuring step_tracker was created properly + step_tracker_diff = DeepDiff(status_obj.step_tracker, status_test_variables.FULL_STEP_TRACKER, ignore_order=True) + assert step_tracker_diff == {} + + # Ensuring tasks_per_step was created properly + tasks_per_step_diff = DeepDiff(status_obj.tasks_per_step, status_test_variables.TASKS_PER_STEP, ignore_order=True) + assert tasks_per_step_diff == {} + + # Ensuring real_step_name_map was created properly + real_step_name_map_diff = DeepDiff( + status_obj.real_step_name_map, status_test_variables.REAL_STEP_NAME_MAP, ignore_order=True + ) + assert real_step_name_map_diff == {} + + # Ensuring requested_statuses was created properly + requested_statuses_diff = DeepDiff( + status_obj.requested_statuses, status_test_variables.ALL_REQUESTED_STATUSES, ignore_order=True + ) + assert requested_statuses_diff == {} + + # Ensuring num_requested_statuses is getting the correct amount of statuses + assert status_obj.num_requested_statuses == status_test_variables.NUM_ALL_REQUESTED_STATUSES + + +def run_study_selector_prompt_valid_input(status_obj: Union[Status, DetailedStatus]): + """ + This is testing the prompt that's displayed when multiple study output + directories are found. We use a patch context manager to send the value "2" to the input + when prompted. We'll also capture the output that's sent to stdout using StringIO + and a patch context manager. The stdout will have a table like so be displayed: + Index Study Name + ------- --------------------------------- + 1 status_test_study_20230713-000000 + 2 status_test_study_20230717-162921 + and since we're sending the value "2" to input, the result from this should be the + absolute path to the status_test_study_20230717-162921 directory. + + :prompt `status_obj`: The Status or DetailedStatus object that we're running this test for + """ + # If we don't switch this to False then the prompt will never be displayed + status_obj.args.no_prompts = False + + # When we call the _obtain_study_method, which will prompt the user to select a study, + # the input message will be captured in mock_input + with patch("builtins.input", side_effect=["2"]) as mock_input: + # Setup variables for our captured_output and the messages we expect to see in the output + captured_output = StringIO() + potential_studies = [(1, status_test_variables.DUMMY_WORKSPACE), (2, status_test_variables.VALID_WORKSPACE)] + tabulated_info = tabulate(potential_studies, headers=["Index", "Study Name"]) + studies_found_msg = f"Found 2 potential studies:\n{tabulated_info}" + prompt_msg = "Which study would you like to view the status of? Use the index on the left: " + + # When we call the _obtain_study method, which will prompt the user to select a study, + # the stdout will be captured in captured_output + with patch("sys.stdout", new=captured_output): + result = status_obj._obtain_study(status_test_variables.PATH_TO_TEST_FILES, 2, potential_studies) + + # We first check that the studies found message was in the captured output of the _obtain_study call + assert studies_found_msg in captured_output.getvalue() + # We then check that the input prompt was given one time with the provided prompt + mock_input.assert_called_once_with(prompt_msg) + # Finally, we check that the _obtain_study method returned the correct workspace + assert result == status_test_variables.VALID_WORKSPACE_PATH + + +def run_study_selector_prompt_invalid_input(status_obj: Union[Status, DetailedStatus]): + """ + This test is very similar to the run_study_selector_prompt_valid_input test above but here + we're testing for invalid inputs before a valid input is given. Invalid inputs for this test + are any inputs that are not the integers 1 or 2. We'll use a patch context manager to send all + of our invalid inputs in until the last one which will be valid (in order to get the function + to return). + + :prompt `status_obj`: The Status or DetailedStatus object that we're running this test for + """ + # If we don't switch this to False then the prompt will never be displayed + status_obj.args.no_prompts = False + + with patch("builtins.input", side_effect=["0", "-1", "3", "a", "1.5", "2"]) as mock_input: + # Setup variables for our captured_output and the messages we expect to see in the output + captured_output = StringIO() + potential_studies = [(1, status_test_variables.DUMMY_WORKSPACE), (2, status_test_variables.VALID_WORKSPACE)] + tabulated_info = tabulate(potential_studies, headers=["Index", "Study Name"]) + studies_found_msg = f"Found 2 potential studies:\n{tabulated_info}" + invalid_msg = f"{ANSI_COLORS['RED']}Input must be an integer between 1 and 2.{ANSI_COLORS['RESET']}" + invalid_prompt_msg = "Enter a different index: " + + # When we call the _obtain_study method, which will prompt the user to select a study, + # the stdout will be captured in captured_output + with patch("sys.stdout", new=captured_output): + result = status_obj._obtain_study(status_test_variables.PATH_TO_TEST_FILES, 2, potential_studies) + + # We first check that the studies found message was in the captured output of the _obtain_study call + captured_output_value = captured_output.getvalue() + assert studies_found_msg in captured_output_value + + # We then check that the input was called with the invalid prompt + mock_input.assert_called_with(invalid_prompt_msg) + + # There should be 5 instances of the invalid input message here (from '0', '-1', '3', 'a', '1.5') + count = 0 + while invalid_msg in captured_output_value: + count += 1 + captured_output_value = captured_output_value.replace(invalid_msg, "", 1) + assert count == 5 + + # Finally, we check that the _obtain_study method returned the correct workspace + assert result == status_test_variables.VALID_WORKSPACE_PATH + + +def run_json_dump_test(status_obj: Union[Status, DetailedStatus], expected_output: Dict): + """ + Test the json dump functionality. This tests both the write and append + dump functionalities. The file needs to exist already for an append so it's + better to keep these tests together. This covers the dump method. + + :param `status_obj`: A Status or DetailedStatus object that we're testing the dump functionality for + :param `expected_output`: The expected output from the dump that we'll compare against + """ + try: + # Test write dump functionality for json + status_obj.dump() + with open(status_obj.args.dump, "r") as json_df: + json_df_contents = json.load(json_df) + # There should only be one entry in the json dump file so this will only 'loop' once + for dump_entry in json_df_contents.values(): + json_dump_diff = DeepDiff(dump_entry, expected_output) + assert json_dump_diff == {} + + # Test append dump functionality for json + # If we don't sleep for 1 second here the program will run too fast and the timestamp for the append dump will be the same + # as the timestamp for the write dump, which causes the write dump entry to be overridden + sleep(1) + # Here, the file already exists from the previous test so it will automatically append to the file + status_obj.dump() + with open(status_obj.args.dump, "r") as json_df: + json_df_append_contents = json.load(json_df) + # There should be two entries here now, both with the same statuses just different timestamps + assert len(json_df_append_contents) == 2 + for dump_entry in json_df_append_contents.values(): + json_append_dump_diff = DeepDiff(dump_entry, expected_output) + assert json_append_dump_diff == {} + # Make sure we always remove the test file that's created from this dump + finally: + try: + os.remove(status_obj.args.dump) + except FileNotFoundError: + pass + + +def _format_csv_data(csv_dump_data: csv.DictReader) -> Dict[str, List[str]]: + """ + Helper function for testing the csv dump functionality to format csv data read in + from the dump file. + + :param `csv_dump_data`: The DictReader object that has the csv data from the dump file + :returns: A formatted dict where keys are fieldnames of the csv file and values are the columns for each field + """ + # Create a formatted dict to store the csv data in csv_dump_data + csv_dump_output = {field_name: [] for field_name in csv_dump_data.fieldnames} + for row in csv_dump_data: + for key, val in row.items(): + # TODO when we add entries for restarts we'll need to change this + if key == "restarts": + csv_dump_output[key].append(int(val)) + else: + csv_dump_output[key].append(val) + return csv_dump_output + + +def build_row_list(csv_formatted_dict: Dict[str, List[Union[str, int]]]) -> List[Tuple]: + """ + Given a dict where each key/val pair represents column label/column values, create a + list of rows. For example: + input: {"a": [1, 2, 3], "b": [4, 5, 6]} + output: [("a", "b"), (1, 4), (2, 5), (3, 6)] + + :param `csv_formatted_dict`: The dict of csv columns that we're converting to a list of rows + :returns: A list of rows created from the `csv_formatted_dict` + """ + column_labels = tuple(csv_formatted_dict.keys()) + row_list = list(zip(*csv_formatted_dict.values())) + row_list.insert(0, column_labels) + return row_list + + +def run_csv_dump_test(status_obj: Union[Status, DetailedStatus], expected_output: List[Tuple]): + """ + Test the csv dump functionality. This tests both the write and append + dump functionalities. The file needs to exist already for an append so it's + better to keep these tests together. This covers the format_status_for_display + and dump methods. + + :param `status_obj`: A Status or DetailedStatus object that we're testing the dump functionality for + :param `expected_output`: The expected output from the dump that we'll compare against + """ + try: + # Test write dump functionality for csv + status_obj.dump() + with open(status_obj.args.dump, "r") as csv_df: + csv_dump_data = csv.DictReader(csv_df) + # Make sure a timestamp field was created + assert "time_of_status" in csv_dump_data.fieldnames + + # Format the csv data that we just read in and create a set of timestamps + csv_dump_output = _format_csv_data(csv_dump_data) + timestamps = set(csv_dump_output["time_of_status"]) + + # We don't care if the timestamp matches, we only care that there should be exactly one timestamp here + del csv_dump_output["time_of_status"] + assert len(timestamps) == 1 + + # Check for differences (should be none) + csv_dump_output = build_row_list(csv_dump_output) + csv_dump_diff = DeepDiff(csv_dump_output, expected_output, ignore_order=True) + assert csv_dump_diff == {} + + # Test append dump functionality for csv + # If we don't sleep for 1 second here the program will run too fast and the timestamp for the append dump will be the same + # as the timestamp for the write dump, which makes it impossible to differentiate between different dump calls + sleep(1) + # Here, the file already exists from the previous test so it will automatically append to the file + status_obj.dump() + with open(status_obj.args.dump, "r") as csv_df: + csv_append_dump_data = csv.DictReader(csv_df) + # Make sure a timestamp field still exists + assert "time_of_status" in csv_append_dump_data.fieldnames + + # Format the csv data that we just read in and create a set of timestamps + csv_append_dump_output = _format_csv_data(csv_append_dump_data) + timestamps = set(csv_append_dump_output["time_of_status"]) + + # We don't care if the timestamp matches, we only care that there should be exactly two timestamps here now + del csv_append_dump_output["time_of_status"] + assert len(timestamps) == 2 + + # Since there are now two dumps, we need to double up the expected output too (except for the keys) + expected_keys = expected_output[0] + expected_output.remove(expected_keys) + expected_output.extend(expected_output) + expected_output.insert(0, expected_keys) + + # Check for differences (should be none) + csv_append_dump_output = build_row_list(csv_append_dump_output) + csv_append_dump_diff = DeepDiff(csv_append_dump_output, expected_output, ignore_order=True, report_repetition=True) + assert csv_append_dump_diff == {} + # Make sure we always remove the test file that's created from this dump + finally: + try: + os.remove(status_obj.args.dump) + except FileNotFoundError: + pass diff --git a/tests/unit/study/status_test_files/status_test_spec.yaml b/tests/unit/study/status_test_files/status_test_spec.yaml new file mode 100644 index 000000000..91b678476 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_spec.yaml @@ -0,0 +1,79 @@ +description: + name: status_test_study + description: a spec file to test the status command + +env: + variables: + N_SAMPLES: 5 + OUTPUT_PATH: . + +global.parameters: + GREET: + values : ["hello", "hola"] + label : GREET.%% + LEAVE: + values: ["goodbye", "adios"] + label: LEAVE.%% + +study: + - name: just_samples + description: no parameters, just samples + run: + cmd: | + echo "$(NUM). $(WORLD)!" + task_queue: just_samples_queue + + - name: just_parameters + description: no samples, just parameters + run: + cmd: | + echo "$(GREET)!" + restart: echo "$(LEAVE)." + task_queue: just_parameters_queue + + - name: params_and_samples + description: both parameters and samples + run: + cmd: | + echo "$(GREET), $(WORLD)!" + task_queue: both_queue + depends: [just_samples_*, just_parameters_*] + + - name: fail_step + description: purposefully fail a step + run: + cmd: | + exit $(MERLIN_SOFT_FAIL) + depends: [params_and_samples_*] + task_queue: fail_queue + + - name: cancel_step + description: purposefully cancel the study + run: + cmd: | + exit $(MERLIN_STOP_WORKERS) + depends: [fail_step] + task_queue: cancel_queue + + - name: unstarted_step + description: purposefully don't run this step + run: + cmd: | + echo "nobody wants to start me :(" + depends: [cancel_step] + task_queue: unstarted_queue + +merlin: + resources: + workers: + sample_worker: + args: -l INFO -Ofair --concurrency 36 --prefetch-multiplier 1 + steps: [just_samples, params_and_samples] + other_worker: + args: -l INFO -Ofair --concurrency 1 --prefetch-multiplier 1 + steps: [just_parameters, fail_step, cancel_step, unstarted_step] + samples: + generate: + cmd: python3 make_samples.py --filepath=$(MERLIN_INFO)/samples.csv --number=$(N_SAMPLES) + file: $(MERLIN_INFO)/samples.csv + column_labels: [WORLD, NUM] diff --git a/tests/unit/study/status_test_files/status_test_study_20230713-000000/dummy.txt b/tests/unit/study/status_test_files/status_test_study_20230713-000000/dummy.txt new file mode 100644 index 000000000..8f773e5c9 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230713-000000/dummy.txt @@ -0,0 +1 @@ +this is a dummy test folder to see if the status command can realize multiple study output directories \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/MERLIN_STATUS.json new file mode 100644 index 000000000..8bed4fb33 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/MERLIN_STATUS.json @@ -0,0 +1,13 @@ +{ + "cancel_step": { + "task_queue": "cancel_queue", + "worker_name": "other_worker", + "cancel_step": { + "status": "CANCELLED", + "return_code": "MERLIN_STOP_WORKERS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/cancel_step/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/MERLIN_STATUS.json new file mode 100644 index 000000000..f1adbf765 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/MERLIN_STATUS.json @@ -0,0 +1,13 @@ +{ + "fail_step": { + "task_queue": "fail_queue", + "worker_name": "other_worker", + "fail_step": { + "status": "FAILED", + "return_code": "MERLIN_SOFT_FAIL", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/fail_step/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json new file mode 100644 index 000000000..320f10731 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json @@ -0,0 +1,13 @@ +{ + "just_parameters_GREET.hello.LEAVE.goodbye": { + "task_queue": "just_parameters_queue", + "worker_name": "other_worker", + "just_parameters/GREET.hello.LEAVE.goodbye": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json new file mode 100644 index 000000000..8249207cb --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json @@ -0,0 +1,13 @@ +{ + "just_parameters_GREET.hola.LEAVE.adios": { + "task_queue": "just_parameters_queue", + "worker_name": "other_worker", + "just_parameters/GREET.hola.LEAVE.adios": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/00/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/00/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/01/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/01/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/02/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/02/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/03/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/03/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/04/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/04/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json new file mode 100644 index 000000000..e73dd2977 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json @@ -0,0 +1,41 @@ +{ + "just_samples": { + "task_queue": "just_samples_queue", + "worker_name": "sample_worker", + "just_samples/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "just_samples/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "just_samples/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "just_samples/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "just_samples/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/samples.csv b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/samples.csv new file mode 100644 index 000000000..566031673 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/samples.csv @@ -0,0 +1,5 @@ +KELLY,0 +KIMBERY,1 +TAMESHA,2 +TASHA,3 +VIRGIE,4 \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.expanded.yaml b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.expanded.yaml new file mode 100644 index 000000000..7f2c7a3e0 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.expanded.yaml @@ -0,0 +1,112 @@ +batch: + dry_run: false + shell: /bin/bash + type: local +description: + description: a spec file to test the status command + name: status_test_study +env: + dependencies: null + labels: null + sources: null + variables: + N_SAMPLES: 5 + OUTPUT_PATH: . +global.parameters: + GREET: + label: GREET.%% + values: + - hello + - hola + LEAVE: + label: LEAVE.%% + values: + - goodbye + - adios +merlin: + resources: + overlap: false + task_server: celery + workers: + other_worker: + args: -l INFO -Ofair --concurrency 1 --prefetch-multiplier 1 + batch: null + nodes: null + steps: + - just_parameters + - fail_step + - cancel_step + - unstarted_step + sample_worker: + args: -l INFO -Ofair --concurrency 36 --prefetch-multiplier 1 + batch: null + nodes: null + steps: + - just_samples + - params_and_samples + samples: + column_labels: + - WORLD + - NUM + file: the test setup will modify this value + generate: + cmd: python3 make_samples.py --filepath= --number=5 + level_max_dirs: 25 +study: +- description: no parameters, just samples + name: just_samples + run: + cmd: echo "$(NUM). $(WORLD)!" + max_retries: 30 + shell: /bin/bash + task_queue: just_samples_queue +- description: no samples, just parameters + name: just_parameters + run: + cmd: echo "$(GREET)!" + max_retries: 30 + restart: echo "$(LEAVE)." + shell: /bin/bash + task_queue: just_parameters_queue +- description: both parameters and samples + name: params_and_samples + run: + cmd: echo "$(GREET), $(WORLD)!" + depends: + - just_samples_* + - just_parameters_* + max_retries: 30 + shell: /bin/bash + task_queue: both_queue +- description: purposefully fail a step + name: fail_step + run: + cmd: exit 101 + depends: + - params_and_samples_* + max_retries: 30 + shell: /bin/bash + task_queue: fail_queue +- description: purposefully cancel the study + name: cancel_step + run: + cmd: 'exit $(MERLIN_STOP_WORKERS) + + sleep 70 + + ' + depends: + - fail_step + max_retries: 30 + shell: /bin/bash + task_queue: cancel_queue +- description: purposefully don't run this step + name: unstarted_step + run: + cmd: echo "nobody wants to start me :(" + depends: + - cancel_step + max_retries: 30 + shell: /bin/bash + task_queue: unstarted_queue +user: null diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.orig.yaml b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.orig.yaml new file mode 100644 index 000000000..d3f0a3b56 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.orig.yaml @@ -0,0 +1,80 @@ +description: + name: status_test_study + description: a spec file to test the status command + +env: + variables: + N_SAMPLES: 5 + OUTPUT_PATH: . + +global.parameters: + GREET: + values : ["hello", "hola"] + label : GREET.%% + LEAVE: + values: ["goodbye", "adios"] + label: LEAVE.%% + +study: + - name: just_samples + description: no parameters, just samples + run: + cmd: | + echo "$(NUM). $(WORLD)!" + task_queue: just_samples_queue + + - name: just_parameters + description: no samples, just parameters + run: + cmd: | + echo "$(GREET)!" + restart: echo "$(LEAVE)." + task_queue: just_parameters_queue + + - name: params_and_samples + description: both parameters and samples + run: + cmd: | + echo "$(GREET), $(WORLD)!" + task_queue: both_queue + depends: [just_samples_*, just_parameters_*] + + - name: fail_step + description: purposefully fail a step + run: + cmd: | + exit $(MERLIN_SOFT_FAIL) + depends: [params_and_samples_*] + task_queue: fail_queue + + - name: cancel_step + description: purposefully cancel the study + run: + cmd: | + exit $(MERLIN_STOP_WORKERS) + sleep 70 + depends: [fail_step] + task_queue: cancel_queue + + - name: unstarted_step + description: purposefully don't run this step + run: + cmd: | + echo "nobody wants to start me :(" + depends: [cancel_step] + task_queue: unstarted_queue + +merlin: + resources: + workers: + sample_worker: + args: -l INFO -Ofair --concurrency 36 --prefetch-multiplier 1 + steps: [just_samples, params_and_samples] + other_worker: + args: -l INFO -Ofair --concurrency 1 --prefetch-multiplier 1 + steps: [just_parameters, fail_step, cancel_step, unstarted_step] + samples: + generate: + cmd: python3 make_samples.py --filepath=$(MERLIN_INFO)/samples.csv --number=$(N_SAMPLES) + file: $(MERLIN_INFO)/samples.csv + column_labels: [WORLD, NUM] diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.partial.yaml b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.partial.yaml new file mode 100644 index 000000000..61f8aee16 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/merlin_info/status_test_spec.partial.yaml @@ -0,0 +1,118 @@ +description: + name: status_test_study + description: a spec file to test the status command + +batch: + type: local + dry_run: false + shell: /bin/bash + +env: + variables: + N_SAMPLES: 5 + OUTPUT_PATH: . + + sources: + + labels: + + dependencies: + +study: + - name: just_samples + description: no parameters, just samples + run: + cmd: echo "$(NUM). $(WORLD)!" + + task_queue: just_samples_queue + shell: /bin/bash + max_retries: 30 + + - name: just_parameters + description: no samples, just parameters + run: + cmd: echo "$(GREET)!" + + restart: echo "$(LEAVE)." + task_queue: just_parameters_queue + shell: /bin/bash + max_retries: 30 + + - name: params_and_samples + description: both parameters and samples + run: + cmd: echo "$(GREET), $(WORLD)!" + + task_queue: both_queue + depends: [just_samples_*, just_parameters_*] + shell: /bin/bash + max_retries: 30 + + - name: fail_step + description: purposefully fail a step + run: + cmd: exit $(MERLIN_SOFT_FAIL) + + depends: [params_and_samples_*] + task_queue: fail_queue + shell: /bin/bash + max_retries: 30 + + - name: cancel_step + description: purposefully cancel the study + run: + cmd: | + exit $(MERLIN_STOP_WORKERS) + sleep 70 + depends: [fail_step] + task_queue: cancel_queue + shell: /bin/bash + max_retries: 30 + + - name: unstarted_step + description: purposefully don't run this step + run: + cmd: echo "nobody wants to start me :(" + + depends: [cancel_step] + task_queue: unstarted_queue + shell: /bin/bash + max_retries: 30 + +global.parameters: + GREET: + values: [hello, hola] + label: GREET.%% + + LEAVE: + values: [goodbye, adios] + label: LEAVE.%% + +merlin: + resources: + workers: + sample_worker: + args: -l INFO -Ofair --concurrency 36 --prefetch-multiplier 1 + steps: [just_samples, params_and_samples] + nodes: + batch: + + other_worker: + args: -l INFO -Ofair --concurrency 1 --prefetch-multiplier 1 + steps: [just_parameters, fail_step, cancel_step, unstarted_step] + nodes: + batch: + + task_server: celery + overlap: false + + samples: + generate: + cmd: python3 make_samples.py --filepath=$(MERLIN_INFO)/samples.csv --number=$(N_SAMPLES) + + file: $(MERLIN_INFO)/samples.csv + column_labels: [WORLD, NUM] + level_max_dirs: 25 + +user: + diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/00/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/00/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/01/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/01/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/02/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/02/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/03/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/03/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/04/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/04/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json new file mode 100644 index 000000000..3614e500f --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json @@ -0,0 +1,41 @@ +{ + "params_and_samples_GREET.hello": { + "task_queue": "both_queue", + "worker_name": "sample_worker", + "params_and_samples/GREET.hello/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hello/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hello/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hello/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hello/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/00/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/00/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/01/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/01/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/02/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/02/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/03/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/03/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/04/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/04/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json new file mode 100644 index 000000000..8d9479c66 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json @@ -0,0 +1,41 @@ +{ + "params_and_samples_GREET.hola": { + "task_queue": "both_queue", + "worker_name": "sample_worker", + "params_and_samples/GREET.hola/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hola/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hola/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hola/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + }, + "params_and_samples/GREET.hola/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0 + } + } +} \ No newline at end of file diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/status.lock new file mode 100755 index 000000000..e69de29bb diff --git a/tests/unit/study/status_test_files/status_test_variables.py b/tests/unit/study/status_test_files/status_test_variables.py new file mode 100644 index 000000000..84a131e33 --- /dev/null +++ b/tests/unit/study/status_test_files/status_test_variables.py @@ -0,0 +1,471 @@ +############################################################################### +# Copyright (c) 2023, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.2. +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### +"""This module holds variables that will be used to test against output from calls to status methods""" +import os + + +# Global path variables for files we'll need during these status tests +PATH_TO_TEST_FILES = f"{os.path.dirname(__file__)}" +SPEC_PATH = f"{PATH_TO_TEST_FILES}/status_test_spec.yaml" +VALID_WORKSPACE = "status_test_study_20230717-162921" +DUMMY_WORKSPACE = "status_test_study_20230713-000000" +VALID_WORKSPACE_PATH = f"{PATH_TO_TEST_FILES}/{VALID_WORKSPACE}" +DUMMY_WORKSPACE_PATH = f"{PATH_TO_TEST_FILES}/{DUMMY_WORKSPACE}" +MERLIN_INFO_PATH = f"{VALID_WORKSPACE_PATH}/merlin_info" +EXPANDED_SPEC_PATH = f"{MERLIN_INFO_PATH}/status_test_spec.expanded.yaml" +SAMPLES_PATH = f"{MERLIN_INFO_PATH}/samples.csv" + +# These globals are variables that will be tested against to ensure correct output +FULL_STEP_TRACKER = { + "started_steps": ["just_samples", "just_parameters", "params_and_samples", "fail_step", "cancel_step"], + "unstarted_steps": ["unstarted_step"], +} +TASKS_PER_STEP = { + "just_samples": 5, + "just_parameters": 2, + "params_and_samples": 10, + "fail_step": 1, + "cancel_step": 1, + "unstarted_step": 1, +} +REAL_STEP_NAME_MAP = { + "just_samples": ["just_samples"], + "just_parameters": ["just_parameters_GREET.hello.LEAVE.goodbye", "just_parameters_GREET.hola.LEAVE.adios"], + "params_and_samples": ["params_and_samples_GREET.hello", "params_and_samples_GREET.hola"], + "fail_step": ["fail_step"], + "cancel_step": ["cancel_step"], +} +NUM_ALL_REQUESTED_STATUSES = sum(TASKS_PER_STEP.values()) - TASKS_PER_STEP["unstarted_step"] + +# This variable holds the state_info dict of every step from VALID_WORKSPACE +# i.e. the format returned by the display() method when run in test_mode +DISPLAY_INFO = { + "just_samples": { + "FINISHED": {"count": 5, "color": "\033[38;2;0;158;115m", "fill": "█"}, + "CANCELLED": {"count": 0, "color": "\033[38;2;240;228;66m", "fill": "/"}, + "FAILED": {"count": 0, "color": "\033[38;2;213;94;0m", "fill": "⣿"}, + "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, + "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, + "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, + "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL_TASKS": {"total": 5}, + "WORKER_NAME": {"name": "sample_worker"}, + "TASK_QUEUE": {"name": "just_samples_queue"}, + }, + "just_parameters": { + "FINISHED": {"count": 2, "color": "\033[38;2;0;158;115m", "fill": "█"}, + "CANCELLED": {"count": 0, "color": "\033[38;2;240;228;66m", "fill": "/"}, + "FAILED": {"count": 0, "color": "\033[38;2;213;94;0m", "fill": "⣿"}, + "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, + "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, + "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, + "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL_TASKS": {"total": 2}, + "WORKER_NAME": {"name": "other_worker"}, + "TASK_QUEUE": {"name": "just_parameters_queue"}, + }, + "params_and_samples": { + "FINISHED": {"count": 10, "color": "\033[38;2;0;158;115m", "fill": "█"}, + "CANCELLED": {"count": 0, "color": "\033[38;2;240;228;66m", "fill": "/"}, + "FAILED": {"count": 0, "color": "\033[38;2;213;94;0m", "fill": "⣿"}, + "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, + "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, + "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, + "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL_TASKS": {"total": 10}, + "WORKER_NAME": {"name": "sample_worker"}, + "TASK_QUEUE": {"name": "both_queue"}, + }, + "fail_step": { + "FINISHED": {"count": 0, "color": "\033[38;2;0;158;115m", "fill": "█"}, + "CANCELLED": {"count": 0, "color": "\033[38;2;240;228;66m", "fill": "/"}, + "FAILED": {"count": 1, "color": "\033[38;2;213;94;0m", "fill": "⣿"}, + "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, + "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, + "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, + "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL_TASKS": {"total": 1}, + "WORKER_NAME": {"name": "other_worker"}, + "TASK_QUEUE": {"name": "fail_queue"}, + }, + "cancel_step": { + "FINISHED": {"count": 0, "color": "\033[38;2;0;158;115m", "fill": "█"}, + "CANCELLED": {"count": 1, "color": "\033[38;2;240;228;66m", "fill": "/"}, + "FAILED": {"count": 0, "color": "\033[38;2;213;94;0m", "fill": "⣿"}, + "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, + "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, + "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, + "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL_TASKS": {"total": 1}, + "WORKER_NAME": {"name": "other_worker"}, + "TASK_QUEUE": {"name": "cancel_queue"}, + }, + "unstarted_step": "UNSTARTED", +} + +# This variable holds every status from the VALID_WORKSPACE in the format used when we first load them in +# i.e. the format loaded in by load_requested_statuses() +ALL_REQUESTED_STATUSES = { + "just_parameters_GREET.hello.LEAVE.goodbye": { + "task_queue": "just_parameters_queue", + "worker_name": "other_worker", + "just_parameters/GREET.hello.LEAVE.goodbye": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, + "just_parameters_GREET.hola.LEAVE.adios": { + "task_queue": "just_parameters_queue", + "worker_name": "other_worker", + "just_parameters/GREET.hola.LEAVE.adios": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, + "just_samples": { + "task_queue": "just_samples_queue", + "worker_name": "sample_worker", + "just_samples/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "just_samples/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "just_samples/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "just_samples/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "just_samples/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, + "params_and_samples_GREET.hello": { + "task_queue": "both_queue", + "worker_name": "sample_worker", + "params_and_samples/GREET.hello/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, + "params_and_samples_GREET.hola": { + "task_queue": "both_queue", + "worker_name": "sample_worker", + "params_and_samples/GREET.hola/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, + "fail_step": { + "task_queue": "fail_queue", + "worker_name": "other_worker", + "fail_step": { + "status": "FAILED", + "return_code": "MERLIN_SOFT_FAIL", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, + "cancel_step": { + "task_queue": "cancel_queue", + "worker_name": "other_worker", + "cancel_step": { + "status": "CANCELLED", + "return_code": "MERLIN_STOP_WORKERS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, + }, +} + +# This variable holds every status from the VALID_WORKSPACE in the format used for displaying/dumping statuses +# i.e. the format returned by format_status_for_display() +ALL_FORMATTED_STATUSES = { + "step_name": [ + "just_parameters_GREET.hello.LEAVE.goodbye", + "just_parameters_GREET.hola.LEAVE.adios", + "just_samples", + "just_samples", + "just_samples", + "just_samples", + "just_samples", + "params_and_samples_GREET.hello", + "params_and_samples_GREET.hello", + "params_and_samples_GREET.hello", + "params_and_samples_GREET.hello", + "params_and_samples_GREET.hello", + "params_and_samples_GREET.hola", + "params_and_samples_GREET.hola", + "params_and_samples_GREET.hola", + "params_and_samples_GREET.hola", + "params_and_samples_GREET.hola", + "fail_step", + "cancel_step", + ], + "step_workspace": [ + "just_parameters/GREET.hello.LEAVE.goodbye", + "just_parameters/GREET.hola.LEAVE.adios", + "just_samples/00", + "just_samples/01", + "just_samples/02", + "just_samples/03", + "just_samples/04", + "params_and_samples/GREET.hello/00", + "params_and_samples/GREET.hello/01", + "params_and_samples/GREET.hello/02", + "params_and_samples/GREET.hello/03", + "params_and_samples/GREET.hello/04", + "params_and_samples/GREET.hola/00", + "params_and_samples/GREET.hola/01", + "params_and_samples/GREET.hola/02", + "params_and_samples/GREET.hola/03", + "params_and_samples/GREET.hola/04", + "fail_step", + "cancel_step", + ], + "status": [ + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FINISHED", + "FAILED", + "CANCELLED", + ], + "return_code": [ + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SUCCESS", + "MERLIN_SOFT_FAIL", + "MERLIN_STOP_WORKERS", + ], + "elapsed_time": [ + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + ], + "run_time": [ + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + "0d:00h:00m:00s", + ], + "restarts": [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0], + "task_queue": [ + "just_parameters_queue", + "just_parameters_queue", + "just_samples_queue", + "just_samples_queue", + "just_samples_queue", + "just_samples_queue", + "just_samples_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "both_queue", + "fail_queue", + "cancel_queue", + ], + "worker_name": [ + "other_worker", + "other_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "sample_worker", + "other_worker", + "other_worker", + ], +} diff --git a/tests/unit/study/test_status.py b/tests/unit/study/test_status.py new file mode 100644 index 000000000..76be5a307 --- /dev/null +++ b/tests/unit/study/test_status.py @@ -0,0 +1,274 @@ +############################################################################### +# Copyright (c) 2023, Lawrence Livermore National Security, LLC. +# Produced at the Lawrence Livermore National Laboratory +# Written by the Merlin dev team, listed in the CONTRIBUTORS file. +# +# +# LLNL-CODE-797170 +# All rights reserved. +# This file is part of Merlin, Version: 1.10.2. +# +# For details, see https://github.com/LLNL/merlin. +# +# Permission is hereby granted, free of charge, to any person obtaining a copy +# of this software and associated documentation files (the "Software"), to deal +# in the Software without restriction, including without limitation the rights +# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +# copies of the Software, and to permit persons to whom the Software is +# furnished to do so, subject to the following conditions: +# The above copyright notice and this permission notice shall be included in +# all copies or substantial portions of the Software. +# +# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +# SOFTWARE. +############################################################################### +""" +Tests for the Status class in the status.py module +""" +import unittest +from argparse import Namespace +from copy import deepcopy +from datetime import datetime + +import yaml +from deepdiff import DeepDiff + +from merlin.main import get_merlin_spec_with_override +from merlin.study.status import Status +from tests.unit.study.status_test_files import shared_tests, status_test_variables + + +class TestMerlinStatus(unittest.TestCase): + """Test the logic for methods in the Status class.""" + + @classmethod + def setUpClass(cls): + """ + We need to modify the path to the samples file in the expanded spec for these tests. + This will only happen once when these tests are initialized. + """ + # Read in the contents of the expanded spec + with open(status_test_variables.EXPANDED_SPEC_PATH, "r") as expanded_file: + cls.initial_expanded_contents = yaml.load(expanded_file, yaml.Loader) + + # Create a copy of the contents so we can reset the file when these tests are done + modified_contents = deepcopy(cls.initial_expanded_contents) + + # Modify the samples file path + modified_contents["merlin"]["samples"]["file"] = status_test_variables.SAMPLES_PATH + + # Write the new contents to the expanded spec + with open(status_test_variables.EXPANDED_SPEC_PATH, "w") as expanded_file: + yaml.dump(modified_contents, expanded_file) + + @classmethod + def tearDownClass(cls): + """ + When these tests are done we'll reset the contents of the expanded spec path + to their initial states. + """ + with open(status_test_variables.EXPANDED_SPEC_PATH, "w") as expanded_file: + yaml.dump(cls.initial_expanded_contents, expanded_file) + + def setUp(self): + """ + We'll create an argparse namespace here that can be modified on a + test-by-test basis. + """ + # We'll set all of the args needed to create the DetailedStatus object here and then + # just modify them on a test-by-test basis + self.args = Namespace( + subparsers="status", + level="INFO", + detailed=False, + variables=None, + task_server="celery", + cb_help=False, + dump=None, + no_prompts=True, # We'll set this to True here since it's easier to test this way + ) + + def test_spec_setup_nonexistent_file(self): + """ + Test the creation of a Status object using a nonexistent spec file. + This should not let us create the object and instead throw an error. + """ + with self.assertRaises(ValueError): + invalid_spec_path = f"{status_test_variables.PATH_TO_TEST_FILES}/nonexistent.yaml" + self.args.specification = invalid_spec_path + self.args.spec_provided, _ = get_merlin_spec_with_override(self.args) + _ = Status(args=self.args, spec_display=True, file_or_ws=invalid_spec_path) + + def test_spec_setup_no_prompts(self): + """ + Test the creation of a Status object using a valid spec file with no + prompts allowed. By default for this test class, no_prompts is True. + This also tests that the attributes created upon initialization are + correct. The methods covered here are _load_from_spec and _obtain_study, + as well as any methods covered in assert_correct_attribute_creation + """ + self.args.specification = status_test_variables.SPEC_PATH + self.args.spec_provided, _ = get_merlin_spec_with_override(self.args) + status_obj = Status(args=self.args, spec_display=True, file_or_ws=status_test_variables.SPEC_PATH) + assert isinstance(status_obj, Status) + + shared_tests.assert_correct_attribute_creation(status_obj) + + def test_prompt_for_study_with_valid_input(self): + """ + This is testing the prompt that's displayed when multiple study output + directories are found. This tests the _obtain_study method using valid inputs. + """ + # We need to load in the MerlinSpec object and save it to the args we'll give to Status + self.args.specification = status_test_variables.SPEC_PATH + self.args.spec_provided, _ = get_merlin_spec_with_override(self.args) + + # We're going to load in a status object without prompts first and then use that to call the method + # that prompts the user for input + status_obj = Status(args=self.args, spec_display=True, file_or_ws=status_test_variables.SPEC_PATH) + shared_tests.run_study_selector_prompt_valid_input(status_obj) + + def test_prompt_for_study_with_invalid_input(self): + """ + This is testing the prompt that's displayed when multiple study output + directories are found. This tests the _obtain_study method using invalid inputs. + """ + # We need to load in the MerlinSpec object and save it to the args we'll give to Status + self.args.specification = status_test_variables.SPEC_PATH + self.args.spec_provided, _ = get_merlin_spec_with_override(self.args) + + # We're going to load in a status object without prompts first and then use that to call the method + # that prompts the user for input + status_obj = Status(args=self.args, spec_display=True, file_or_ws=status_test_variables.SPEC_PATH) + shared_tests.run_study_selector_prompt_invalid_input(status_obj) + + def test_workspace_setup_nonexistent_workspace(self): + """ + Test the creation of a Status object using a nonexistent workspace directory. + This should not let us create the object and instead throw an error. + """ + # Testing non existent workspace (in reality main.py should deal with this for us but we'll check it just in case) + with self.assertRaises(ValueError): + invalid_workspace = f"{status_test_variables.PATH_TO_TEST_FILES}/nonexistent_20230101-000000/" + _ = Status(args=self.args, spec_display=False, file_or_ws=invalid_workspace) + + def test_workspace_setup_not_a_merlin_directory(self): + """ + Test the creation of a Status object using an existing directory that is NOT + an output directory from a merlin study (i.e. the directory does not have a + merlin_info/ subdirectory). This should not let us create the object and instead + throw an error. + """ + with self.assertRaises(ValueError): + _ = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.DUMMY_WORKSPACE_PATH) + + def test_workspace_setup_valid_workspace(self): + """ + Test the creation of a Status object using a valid workspace directory. + This also tests that the attributes created upon initialization are + correct. The _load_from_workspace method is covered here, as well as any + methods covered in assert_correct_attribute_creation. + """ + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + assert isinstance(status_obj, Status) + + shared_tests.assert_correct_attribute_creation(status_obj) + + def test_json_formatter(self): + """ + Test the json formatter for the dump method. This covers the format_json_dump method. + """ + # Create a timestamp and the status object that we'll run tests on + date = datetime.now().strftime("%Y-%m-%d %H:%M:%S") + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + + # Test json formatter + json_format_diff = DeepDiff(status_obj.format_json_dump(date), {date: status_test_variables.ALL_REQUESTED_STATUSES}) + self.assertEqual(json_format_diff, {}) + + def test_csv_formatter(self): + """ + Test the csv formatter for the dump method. This covers the format_csv_dump method. + """ + # Create a timestamp and the status object that we'll run tests on + date = datetime.now().strftime("%Y-%m-%d %H:%M:%S") + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + + # Build the correct format and store each row in a list (so we can ignore the order) + correct_csv_format = {"time_of_status": [date] * len(status_test_variables.ALL_FORMATTED_STATUSES["step_name"])} + correct_csv_format.update(status_test_variables.ALL_FORMATTED_STATUSES) + correct_csv_format = shared_tests.build_row_list(correct_csv_format) + + # Run the csv_formatter and store each row it creates in a list + actual_csv_format = shared_tests.build_row_list(status_obj.format_csv_dump(date)) + + # Compare differences (should be none) + csv_format_diff = DeepDiff(actual_csv_format, correct_csv_format, ignore_order=True) + self.assertEqual(csv_format_diff, {}) + + def test_json_dump(self): + """ + Test the json dump functionality. This tests both the write and append + dump functionalities. The file needs to exist already for an append so it's + better to keep these tests together. This covers the dump method. + """ + # Create the status object that we'll run tests on + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + # Set the dump file + json_dump_file = f"{status_test_variables.PATH_TO_TEST_FILES}/dump_test.json" + status_obj.args.dump = json_dump_file + + # Run the json dump test + shared_tests.run_json_dump_test(status_obj, status_test_variables.ALL_REQUESTED_STATUSES) + + def test_csv_dump(self): + """ + Test the csv dump functionality. This tests both the write and append + dump functionalities. The file needs to exist already for an append so it's + better to keep these tests together. This covers the format_status_for_display + and dump methods. + """ + # Create the status object that we'll run tests on + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + + # Set the dump file + csv_dump_file = f"{status_test_variables.PATH_TO_TEST_FILES}/dump_test.csv" + status_obj.args.dump = csv_dump_file + + # Run the csv dump test + expected_output = shared_tests.build_row_list(status_test_variables.ALL_FORMATTED_STATUSES) + shared_tests.run_csv_dump_test(status_obj, expected_output) + + def test_display(self): + """ + Test the status display functionality without actually displaying anything. + Running the display in test_mode will just provide us with the state_info + dict created for each step that is typically used for display. We'll ensure + this state_info dict is created properly here. This covers the display method. + """ + # Create the status object that we'll run tests on + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + + # Get the status info that display would use if it were printing output + all_status_info = status_obj.display(test_mode=True) + + # Check the information for each step + for step_name, state_info in all_status_info.items(): + # If state_info is a dict then the step should be started; if it's a string then it's unstarted + if isinstance(state_info, dict): + assert step_name in status_test_variables.FULL_STEP_TRACKER["started_steps"] + elif isinstance(state_info, str): + assert step_name in status_test_variables.FULL_STEP_TRACKER["unstarted_steps"] + + # Make sure all the state info dicts for each step match what they should be + state_info_diff = DeepDiff(state_info, status_test_variables.DISPLAY_INFO[step_name]) + self.assertEqual(state_info_diff, {}) + + +if __name__ == "__main__": + unittest.main() From e47196a362b0c06a9d6010110f0ee327da1b5f78 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Fri, 18 Aug 2023 11:42:15 -0700 Subject: [PATCH 04/13] run fix-style and remove import of deprecated function --- merlin/exceptions/__init__.py | 2 +- merlin/study/celeryadapter.py | 1 - merlin/study/step.py | 2 +- merlin/utils.py | 2 +- tests/unit/study/__init__.py | 2 +- 5 files changed, 4 insertions(+), 5 deletions(-) diff --git a/merlin/exceptions/__init__.py b/merlin/exceptions/__init__.py index 8fa9a21f2..dbfb2c1f5 100644 --- a/merlin/exceptions/__init__.py +++ b/merlin/exceptions/__init__.py @@ -42,7 +42,7 @@ "HardFailException", "InvalidChainException", "RestartException", - "DeepMergeException," + "DeepMergeException,", ) diff --git a/merlin/study/celeryadapter.py b/merlin/study/celeryadapter.py index cb2963f40..69855fe74 100644 --- a/merlin/study/celeryadapter.py +++ b/merlin/study/celeryadapter.py @@ -205,7 +205,6 @@ def query_celery_workers(spec_worker_names, queues, workers_regex): :param `workers_regex`: A list of regexs to filter by """ from merlin.celery import app # pylint: disable=C0415 - from merlin.display import tabulate_info # pylint: disable=C0415 # Ping all workers and grab which ones are running workers = get_workers_from_app() diff --git a/merlin/study/step.py b/merlin/study/step.py index 3a0290b89..3b6259513 100644 --- a/merlin/study/step.py +++ b/merlin/study/step.py @@ -460,4 +460,4 @@ def execute(self, adapter_config): if self.restart and self.get_restart_cmd(): return ReturnCode(self.mstep.restart(adapter)) - return ReturnCode(self.mstep.execute(adapter)) \ No newline at end of file + return ReturnCode(self.mstep.execute(adapter)) diff --git a/merlin/utils.py b/merlin/utils.py index 1e4b5e199..6e6abfead 100644 --- a/merlin/utils.py +++ b/merlin/utils.py @@ -662,4 +662,4 @@ def ws_time_to_dt(ws_time: str) -> datetime: hour = int(ws_time[9:11]) minute = int(ws_time[11:13]) second = int(ws_time[13:]) - return datetime(year, month, day, hour=hour, minute=minute, second=second) \ No newline at end of file + return datetime(year, month, day, hour=hour, minute=minute, second=second) diff --git a/tests/unit/study/__init__.py b/tests/unit/study/__init__.py index a48da8d86..05c5d0bbf 100644 --- a/tests/unit/study/__init__.py +++ b/tests/unit/study/__init__.py @@ -26,4 +26,4 @@ # LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, # OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE # SOFTWARE. -############################################################################### \ No newline at end of file +############################################################################### From 07630594db12fa13e9625fd7d0ffa185f9380c92 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Fri, 18 Aug 2023 11:42:33 -0700 Subject: [PATCH 05/13] update CHANGELOG --- CHANGELOG.md | 36 ++++++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b3ac21def..8fa907f1b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,10 +6,46 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## [unreleased] ### Fixed +- Cyclical imports and config imports that could easily cause ci issues ### Added +- New file `merlin/study/status.py` dedicated to work relating to the status command + - Contains the Status and DetailedStatus classes +- New file `merlin/common/dumper.py` containing a Dumper object to help dump output to outfiles +- Study name and parameter info now stored in the DAG and MerlinStep objects +- Added functions to `merlin/display.py` that help display status information: + - `display_status_summary` handles the display for the `merlin status` command + - `display_progress_bar` generates and displays a progress bar +- Added new methods to the MerlinSpec class: + - get_tasks_per_step() +- Added methods to the MerlinStepRecord class to mark status changes for tasks as they run (follows Maestro's StepRecord format mostly) +- Added methods to the Step class: + - name_no_params() +- Added a property paramater_labels to the MerlinStudy class +- Added two new utility functions: + - dict_deep_merge() that deep merges two dicts into one + - ws_time_to_dt() that converts a workspace timestring (YYYYMMDD-HHMMSS) to a datetime object +- A new celery task `condense_status_files` to be called when sets of samples finish +- Added a celery config setting `worker_cancel_long_running_tasks_on_connection_loss` since this functionality is about to change in the next version of celery +- Tests for the Status class + - this required adding a decent amount of test files to help with the tests; these can be found under the tests/unit/study/status_test_files directory ### Changed +- Reformatted the entire "merlin status" command + - Now accepts both spec files and workspace directories as arguments + - e.g. "merlin status hello.yaml" and "merlin status hello_20230228-111920/" both work + - Removed the --steps flag + - Replaced the --csv flag with the --dump flag + - This will make it easier in the future to adopt more file types to dump to + - New functionality: + - Shows step_by_step progress bar for tasks + - Displays a summary of task statuses below the progress bar +- Split the `add_chains_to_chord` function in `merlin/common/tasks.py` into two functions: + - `get_1d_chain` which converts a 2D list of chains into a 1D list + - `launch_chain` which launches the 1D chain +- Pulled the needs_merlin_expansion() method out of the Step class and made it a function instead +- Removed `tabulate_info` function; replaced with tabulate from the tabulate library +- Moved `verify_filepath` and `verify_dirpath` from `merlin/main.py` to `merlin/utils.py` - Bump certifi from 2022.12.7 to 2023.7.22 in /docs - Bump pygments from 2.13.0 to 2.15.0 in /docs - Bump requests from 2.28.1 to 2.31.0 in /docs From 629ec4061287dbde3a7694834678223fc2e3e200 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Mon, 25 Sep 2023 17:56:16 -0700 Subject: [PATCH 06/13] add more logging statements, make better use of glob --- merlin/common/tasks.py | 17 +++++++--- merlin/display.py | 2 ++ merlin/study/status.py | 75 ++++++++++++++++++++++-------------------- merlin/study/step.py | 16 +++++++++ 4 files changed, 70 insertions(+), 40 deletions(-) diff --git a/merlin/common/tasks.py b/merlin/common/tasks.py index d91ffd29c..d38eb89a2 100644 --- a/merlin/common/tasks.py +++ b/merlin/common/tasks.py @@ -443,15 +443,17 @@ def get_1d_chain(all_chains: List[List["Signature"]]) -> List["Signature"]: # n return chain_steps -def gather_statuses(sample_index: "SampleIndex", workspace: str, condensed_workspace: str) -> Dict: # noqa: F821 +def gather_statuses(sample_index: "SampleIndex", workspace: str, condensed_workspace: str, files_to_remove: List[str]) -> Dict: # noqa: F821 """ Traverse the sample index and gather all of the statuses into one. :param `sample_index`: A SampleIndex object to track this specific sample hierarchy :param `workspace`: The full workspace path to the step we're condensing for :param `condensed_workspace`: A shortened version of `workspace` that's saved in the status files + :param `files_to_remove`: An empty list that we'll add filepaths to that need removed :returns: A dict of condensed statuses """ + LOG.info(f"Gathering statuses to condense for '{condensed_workspace}'") condensed_statuses = {} for path, _ in sample_index.traverse(conditional=lambda c: c.is_parent_of_leaf): # Read in the status data @@ -465,13 +467,13 @@ def gather_statuses(sample_index: "SampleIndex", workspace: str, condensed_works status = json.load(status_file) # This for loop is just to get the step name that we don't have; it's really not even looping - for step_name, _ in status.items(): + for step_name in status: try: # Make sure the status for this sample workspace is in a finished state (not initialized or running) if status[step_name][f"{condensed_workspace}/{path}"]["status"] not in ("INITIALIZED", "RUNNING"): # Add the status data to the statuses we'll write to the condensed file and remove this status file dict_deep_merge(condensed_statuses, status) - os.remove(status_filepath) + files_to_remove.append(status_filepath) except KeyError: LOG.warning(f"Key error when reading from {sample_workspace}") except Timeout: @@ -519,7 +521,8 @@ def condense_status_files(self, *args: Any, **kwargs: Any) -> ReturnCode: # pyl return None # Read in all the statuses from this sample index - condensed_statuses = gather_statuses(sample_index, workspace, condensed_workspace) + files_to_remove = [] + condensed_statuses = gather_statuses(sample_index, workspace, condensed_workspace, files_to_remove) # If there are statuses to write to the condensed status file then write them if condensed_statuses: @@ -530,6 +533,7 @@ def condense_status_files(self, *args: Any, **kwargs: Any) -> ReturnCode: # pyl # Lock the file to avoid race conditions with lock.acquire(timeout=20): # If the condensed file already exists, grab the statuses from it + LOG.info(f"Condensing statuses for '{condensed_workspace}' to '{condensed_status_filepath}'") if os.path.exists(condensed_status_filepath): with open(condensed_status_filepath, "r") as condensed_status_file: existing_condensed_statuses = json.load(condensed_status_file) @@ -541,6 +545,11 @@ def condense_status_files(self, *args: Any, **kwargs: Any) -> ReturnCode: # pyl # Write the condensed statuses to the condensed status file with open(condensed_status_filepath, "w") as condensed_status_file: json.dump(condensed_statuses, condensed_status_file) + + # Remove the status files we just condensed + for file_to_remove in files_to_remove: + LOG.debug(f"Removing '{file_to_remove}'.") + os.remove(file_to_remove) except Timeout: # Raising this celery timeout instead will trigger a restart for this task raise TimeoutError # pylint: disable=W0707 diff --git a/merlin/display.py b/merlin/display.py index 915773f18..8075ee1ab 100644 --- a/merlin/display.py +++ b/merlin/display.py @@ -290,6 +290,7 @@ def display_status_summary( # pylint: disable=R0912 terminal_size = shutil.get_terminal_size() progress_bar_width = terminal_size.columns // 4 + LOG.debug(f"step_tracker in display: {status_obj.step_tracker}") for sstep in status_obj.step_tracker["started_steps"]: # This dict will keep track of the number of tasks at each status state_info = { @@ -305,6 +306,7 @@ def display_status_summary( # pylint: disable=R0912 num_completed_tasks = 0 # Loop through each entry for the step (if there's no parameters there will just be one entry) + LOG.debug(f"real_step_name_map[{sstep}]: {status_obj.real_step_name_map[sstep]}") for real_step_name in status_obj.real_step_name_map[sstep]: # Grab the statuses for this step overall_step_info = status_obj.requested_statuses[real_step_name] diff --git a/merlin/study/status.py b/merlin/study/status.py index ab8b8af35..b60cfe694 100644 --- a/merlin/study/status.py +++ b/merlin/study/status.py @@ -183,21 +183,27 @@ def _load_from_spec(self, filepath: str) -> Tuple[str, "MerlinSpec"]: # noqa: F else: output_path = f"{os.path.dirname(filepath)}/{self.args.spec_provided.output_path}" + LOG.debug(f"Verifying output path: {output_path}...") study_output_dir = verify_dirpath(output_path) + LOG.debug(f"Output path verified. Expanded version: {study_output_dir}") # Build a list of potential study output directories study_output_subdirs = next(os.walk(study_output_dir))[1] timestamp_regex = r"\d{8}-\d{6}" potential_studies = [] num_studies = 0 + LOG.debug(f"All subdirs in output path: {study_output_subdirs}") for subdir in study_output_subdirs: match = re.search(rf"{self.args.spec_provided.name}_{timestamp_regex}", subdir) if match: potential_studies.append((num_studies + 1, subdir)) num_studies += 1 + LOG.debug(f"Potential studies: {potential_studies}") # Obtain the correct study to view the status of based on the list of potential studies we just built + LOG.debug("Obtaining a study to view the status of...") study_to_check = self._obtain_study(study_output_dir, num_studies, potential_studies) + LOG.debug(f"Selected '{study_to_check}' for viewing.") # Verify the directory that the user selected is a merlin study output directory if "merlin_info" not in next(os.walk(study_to_check))[1]: @@ -212,7 +218,10 @@ def _load_from_spec(self, filepath: str) -> Tuple[str, "MerlinSpec"]: # noqa: F raise ValueError(f"Multiple expanded spec options found in the {study_to_check}/merlin_info/ directory") if len(expanded_spec_options) < 1: raise ValueError(f"No expanded spec options found in the {study_to_check}/merlin_info/ directory") + + LOG.debug(f"Creating a spec object from '{expanded_spec_options[0]}'...") actual_spec = get_spec_with_expansion(expanded_spec_options[0]) + LOG.debug("Spec object created.") return study_to_check, actual_spec @@ -223,20 +232,16 @@ def _load_from_workspace(self) -> "MerlinSpec": # noqa: F821 :returns: A MerlinSpec object loaded from the workspace provided by the user """ # Grab the spec file from the directory provided - info_dir = verify_dirpath(f"{self.workspace}/merlin_info") - spec_file = "" - for _, _, files in os.walk(info_dir): - for f in files: # pylint: disable=C0103 - if f.endswith(".expanded.yaml"): - spec_file = f"{info_dir}/{f}" - break - break - - # Make sure we got a spec file and load it in - if not spec_file: - LOG.error(f"Spec file not found in {info_dir}. Cannot display status.") - return None - spec = get_spec_with_expansion(spec_file) + expanded_spec_options = glob(f"{self.workspace}/merlin_info/*.expanded.yaml") + if len(expanded_spec_options) > 1: + raise ValueError(f"Multiple expanded spec options found in the {self.workspace}/merlin_info/ directory") + if len(expanded_spec_options) < 1: + raise ValueError(f"No expanded spec options found in the {self.workspace}/merlin_info/ directory") + + # Create a MerlinSpec object from the expanded spec we grabbed + LOG.debug(f"Creating a spec object from '{expanded_spec_options[0]}'...") + spec = get_spec_with_expansion(expanded_spec_options[0]) + LOG.debug("Spec object created.") return spec @@ -252,12 +257,17 @@ def _create_step_tracker(self, steps_to_check: List[str]) -> Dict[str, List[str] started_steps = next(os.walk(self.workspace))[1] started_steps.remove("merlin_info") + LOG.debug(f"All started steps: {started_steps}") + for sstep in started_steps: if sstep in steps_to_check: step_tracker["started_steps"].append(sstep) steps_to_check.remove(sstep) step_tracker["unstarted_steps"] = steps_to_check + LOG.debug(f"Started steps after (potentially) filtering: {step_tracker['started_steps']}") + LOG.debug(f"Unstarted steps: {step_tracker['unstarted_steps']}") + return step_tracker def get_steps_to_display(self) -> Dict[str, List[str]]: @@ -275,6 +285,8 @@ def get_steps_to_display(self) -> Dict[str, List[str]]: # Filter the steps to display status for by started/unstarted step_tracker = self._create_step_tracker(existing_steps) + LOG.debug("Step tracker created.") + return step_tracker @property @@ -299,36 +311,25 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict """ step_statuses = {} num_statuses_read = 0 + self.real_step_name_map[started_step_name] = set() # Traverse the step workspace and look for MERLIN_STATUS files + LOG.info(f"Traversing '{step_workspace}' to find MERLIN_STATUS.json files...") for root, _, files in os.walk(step_workspace): - if "MERLIN_STATUS.json" in files: - status_filepath = f"{root}/MERLIN_STATUS.json" + # Search for a status file + status_filepath = os.path.join(root, "MERLIN_STATUS.json") + matching_files = glob(status_filepath) + if matching_files: + LOG.debug(f"Found status file at '{status_filepath}'") lock = FileLock(f"{root}/status.lock") # pylint: disable=E0110 statuses_read = read_status(status_filepath, lock) - # Count the number of statuses we just read - for step_name, status_info in statuses_read.items(): - if started_step_name not in self.real_step_name_map: - self.real_step_name_map[started_step_name] = [step_name] - else: - if step_name not in self.real_step_name_map[started_step_name]: - self.real_step_name_map[started_step_name].append(step_name) - num_statuses_read += len(status_info.keys() - NON_WORKSPACE_KEYS) + # Update map of real step names + self.real_step_name_map[started_step_name].update(list(statuses_read.keys())) # Merge the statuses we read with the dict tracking all statuses for this step dict_deep_merge(step_statuses, statuses_read) - - # If we've read all the statuses then we're done - if num_statuses_read == self.tasks_per_step[started_step_name]: - break - # This shouldn't get hit - if num_statuses_read > self.tasks_per_step[started_step_name]: - LOG.error( - f"Read {num_statuses_read} statuses when there should " - f"only be {self.tasks_per_step[started_step_name]} tasks in total." - ) - break + LOG.info(f"Done traversing '{step_workspace}'. Read in {num_statuses_read} {'statuses' if num_statuses_read != 1 else 'status'}.") return step_statuses @@ -345,7 +346,7 @@ def load_requested_statuses(self): dict_deep_merge(self.requested_statuses, step_statuses) # Count how many statuses in total that we just read in - LOG.info(f"Read in {self.num_requested_statuses} statuses.") + LOG.info(f"Read in {self.num_requested_statuses} statuses total.") def display(self, test_mode=False) -> Dict: """ @@ -375,7 +376,9 @@ def format_csv_dump(self, date: datetime) -> Dict: :returns: A dict equivalent of formatted statuses with a timestamp entry at the start of the dict. """ # Reformat the statuses to a new dict where the keys are the column labels and rows are the values + LOG.debug("Formatting statuses for csv dump...") statuses_to_write = self.format_status_for_display() + LOG.debug("Statuses formatted.") # Add date entries as the first column then update this dict with the statuses we just reformatted statuses_with_timestamp = {"time_of_status": [date] * len(statuses_to_write["step_name"])} diff --git a/merlin/study/step.py b/merlin/study/step.py index 3b6259513..dbba3b21e 100644 --- a/merlin/study/step.py +++ b/merlin/study/step.py @@ -101,6 +101,7 @@ def condensed_workspace(self) -> str: end_of_path = self.workspace.value.rsplit(step_name, 1)[1] condensed_workspace = f"{step_name}{end_of_path}" + LOG.debug(f"Condense workspace '{condensed_workspace}'") return condensed_workspace def _execute(self, adapter: "ScriptAdapter", script: str) -> Tuple["SubmissionRecord", int]: # noqa: F821 @@ -114,6 +115,8 @@ def _execute(self, adapter: "ScriptAdapter", script: str) -> Tuple["SubmissionRe :returns: A tuple of a return code and the jobid from the execution of `script` """ self.mark_running() + + LOG.info(f"Submitting script for {self.name}") srecord = adapter.submit(self.step, script, self.workspace.value) retcode = srecord.submission_code @@ -122,6 +125,7 @@ def _execute(self, adapter: "ScriptAdapter", script: str) -> Tuple["SubmissionRe def mark_running(self): """Mark the start time of the record and update the status file.""" + LOG.debug(f"Marking running for {self.name}") super().mark_running() self._update_status_file() @@ -133,6 +137,8 @@ def mark_end(self, state: ReturnCode, max_retries: bool = False): :param `state`: A merlin ReturnCode object representing the end state of a task :param `max_retries`: A bool representing whether we hit the max number of retries or not """ + LOG.debug(f"Marking end for {self.name}") + # Dictionary to keep track of associated variables for each return code state_mapper = { ReturnCode.OK: { @@ -186,12 +192,14 @@ def mark_end(self, state: ReturnCode, max_retries: bool = False): def mark_restart(self): """Increment the number of restarts we've had for this step and update the status file""" + LOG.debug(f"Marking restart for {self.name}") if self.restart_limit == 0 or self._num_restarts < self.restart_limit: self._num_restarts += 1 self._update_status_file() def setup_workspace(self): """Initialize the record's workspace and status file.""" + LOG.debug(f"Setting up workspace for {self.name}") super().setup_workspace() self._update_status_file() @@ -220,8 +228,14 @@ def _update_status_file( State.UNKNOWN: "UNKNOWN", } + LOG.debug(f"Marking status for {self.name} as {state_translator[self.status]}.") + if result: + LOG.debug(f"Result for {self.name} is {result}") + status_filepath = f"{self.workspace.value}/MERLIN_STATUS.json" + LOG.debug(f"Status filepath for {self.name}: '{status_filepath}") + # If the status file already exists then we can just add to it if os.path.exists(status_filepath): lock = FileLock(f"{self.workspace.value}/status.lock") # pylint: disable=E0110 @@ -248,8 +262,10 @@ def _update_status_file( "restarts": self.restarts, } + LOG.info(f"Writing status for {self.name} to '{status_filepath}...") with open(status_filepath, "w") as status_file: json.dump(status_info, status_file) + LOG.info(f"Status for {self.name} successfully written.") class Step: From af046692f58adb953a21f3867674ed510d32bf88 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Tue, 26 Sep 2023 10:17:11 -0700 Subject: [PATCH 07/13] run fix-style --- merlin/common/tasks.py | 6 ++++-- merlin/study/status.py | 8 +++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/merlin/common/tasks.py b/merlin/common/tasks.py index d38eb89a2..5fb17a23c 100644 --- a/merlin/common/tasks.py +++ b/merlin/common/tasks.py @@ -443,7 +443,9 @@ def get_1d_chain(all_chains: List[List["Signature"]]) -> List["Signature"]: # n return chain_steps -def gather_statuses(sample_index: "SampleIndex", workspace: str, condensed_workspace: str, files_to_remove: List[str]) -> Dict: # noqa: F821 +def gather_statuses( + sample_index: "SampleIndex", workspace: str, condensed_workspace: str, files_to_remove: List[str] +) -> Dict: # noqa: F821 """ Traverse the sample index and gather all of the statuses into one. @@ -545,7 +547,7 @@ def condense_status_files(self, *args: Any, **kwargs: Any) -> ReturnCode: # pyl # Write the condensed statuses to the condensed status file with open(condensed_status_filepath, "w") as condensed_status_file: json.dump(condensed_statuses, condensed_status_file) - + # Remove the status files we just condensed for file_to_remove in files_to_remove: LOG.debug(f"Removing '{file_to_remove}'.") diff --git a/merlin/study/status.py b/merlin/study/status.py index b60cfe694..1e9180afb 100644 --- a/merlin/study/status.py +++ b/merlin/study/status.py @@ -218,7 +218,7 @@ def _load_from_spec(self, filepath: str) -> Tuple[str, "MerlinSpec"]: # noqa: F raise ValueError(f"Multiple expanded spec options found in the {study_to_check}/merlin_info/ directory") if len(expanded_spec_options) < 1: raise ValueError(f"No expanded spec options found in the {study_to_check}/merlin_info/ directory") - + LOG.debug(f"Creating a spec object from '{expanded_spec_options[0]}'...") actual_spec = get_spec_with_expansion(expanded_spec_options[0]) LOG.debug("Spec object created.") @@ -237,7 +237,7 @@ def _load_from_workspace(self) -> "MerlinSpec": # noqa: F821 raise ValueError(f"Multiple expanded spec options found in the {self.workspace}/merlin_info/ directory") if len(expanded_spec_options) < 1: raise ValueError(f"No expanded spec options found in the {self.workspace}/merlin_info/ directory") - + # Create a MerlinSpec object from the expanded spec we grabbed LOG.debug(f"Creating a spec object from '{expanded_spec_options[0]}'...") spec = get_spec_with_expansion(expanded_spec_options[0]) @@ -329,7 +329,9 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict # Merge the statuses we read with the dict tracking all statuses for this step dict_deep_merge(step_statuses, statuses_read) - LOG.info(f"Done traversing '{step_workspace}'. Read in {num_statuses_read} {'statuses' if num_statuses_read != 1 else 'status'}.") + LOG.info( + f"Done traversing '{step_workspace}'. Read in {num_statuses_read} {'statuses' if num_statuses_read != 1 else 'status'}." + ) return step_statuses From 99891368200b1720b1697c14a60ae0054eb71573 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Tue, 26 Sep 2023 10:17:24 -0700 Subject: [PATCH 08/13] clean up test files a bit --- tests/unit/study/status_test_files/status_test_spec.yaml | 2 +- .../just_samples/00/status.lock | 0 .../just_samples/01/status.lock | 0 .../just_samples/02/status.lock | 0 .../just_samples/03/status.lock | 0 .../just_samples/04/status.lock | 0 .../params_and_samples/GREET.hello/00/status.lock | 0 .../params_and_samples/GREET.hello/01/status.lock | 0 .../params_and_samples/GREET.hello/02/status.lock | 0 .../params_and_samples/GREET.hello/03/status.lock | 0 .../params_and_samples/GREET.hello/04/status.lock | 0 .../params_and_samples/GREET.hola/00/status.lock | 0 .../params_and_samples/GREET.hola/01/status.lock | 0 .../params_and_samples/GREET.hola/02/status.lock | 0 .../params_and_samples/GREET.hola/03/status.lock | 0 .../params_and_samples/GREET.hola/04/status.lock | 0 16 files changed, 1 insertion(+), 1 deletion(-) delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/00/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/01/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/02/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/03/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/04/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/00/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/01/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/02/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/03/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/04/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/00/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/01/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/02/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/03/status.lock delete mode 100755 tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/04/status.lock diff --git a/tests/unit/study/status_test_files/status_test_spec.yaml b/tests/unit/study/status_test_files/status_test_spec.yaml index 91b678476..df502f1e4 100644 --- a/tests/unit/study/status_test_files/status_test_spec.yaml +++ b/tests/unit/study/status_test_files/status_test_spec.yaml @@ -74,6 +74,6 @@ merlin: steps: [just_parameters, fail_step, cancel_step, unstarted_step] samples: generate: - cmd: python3 make_samples.py --filepath=$(MERLIN_INFO)/samples.csv --number=$(N_SAMPLES) + cmd: python3 $(SPECROOT)/make_samples.py --filepath=$(MERLIN_INFO)/samples.csv --number=$(N_SAMPLES) file: $(MERLIN_INFO)/samples.csv column_labels: [WORLD, NUM] diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/00/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/00/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/01/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/01/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/02/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/02/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/03/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/03/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/04/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/04/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/00/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/00/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/01/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/01/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/02/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/02/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/03/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/03/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/04/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/04/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/00/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/00/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/01/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/01/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/02/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/02/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/03/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/03/status.lock deleted file mode 100755 index e69de29bb..000000000 diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/04/status.lock b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/04/status.lock deleted file mode 100755 index e69de29bb..000000000 From 2fe5be7240d7ab028fc1ae5f4514a3864a0568c1 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Tue, 26 Sep 2023 12:36:40 -0700 Subject: [PATCH 09/13] fix test suite after step_name_map mod --- .../study/status_test_files/status_test_variables.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/unit/study/status_test_files/status_test_variables.py b/tests/unit/study/status_test_files/status_test_variables.py index 84a131e33..fd4c36778 100644 --- a/tests/unit/study/status_test_files/status_test_variables.py +++ b/tests/unit/study/status_test_files/status_test_variables.py @@ -56,11 +56,11 @@ "unstarted_step": 1, } REAL_STEP_NAME_MAP = { - "just_samples": ["just_samples"], - "just_parameters": ["just_parameters_GREET.hello.LEAVE.goodbye", "just_parameters_GREET.hola.LEAVE.adios"], - "params_and_samples": ["params_and_samples_GREET.hello", "params_and_samples_GREET.hola"], - "fail_step": ["fail_step"], - "cancel_step": ["cancel_step"], + "just_samples": set(["just_samples"]), + "just_parameters": set(["just_parameters_GREET.hello.LEAVE.goodbye", "just_parameters_GREET.hola.LEAVE.adios"]), + "params_and_samples": set(["params_and_samples_GREET.hello", "params_and_samples_GREET.hola"]), + "fail_step": set(["fail_step"]), + "cancel_step": set(["cancel_step"]), } NUM_ALL_REQUESTED_STATUSES = sum(TASKS_PER_STEP.values()) - TASKS_PER_STEP["unstarted_step"] From dd8c331b6280981dba8ddd6e1af364a3d57594c9 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Wed, 27 Sep 2023 12:25:33 -0700 Subject: [PATCH 10/13] add avg/std dev run time calculations to status --- merlin/display.py | 31 ++++++---- merlin/study/status.py | 131 ++++++++++++++++++++++++++++------------- merlin/utils.py | 34 +++++++++++ 3 files changed, 145 insertions(+), 51 deletions(-) diff --git a/merlin/display.py b/merlin/display.py index 75f0ffd51..52a46cd44 100644 --- a/merlin/display.py +++ b/merlin/display.py @@ -255,12 +255,16 @@ def _display_summary(state_info: Dict[str, str], cb_help: bool): # Grab the value associated with the label value = None - if "count" in val and val["count"] > 0: - value = val["count"] + if "count" in val: + if val["count"] > 0: + value = val["count"] elif "total" in val: value = val["total"] elif "name" in val: value = val["name"] + else: + value = val + # Add the label and value as an entry to the summary if value: summary.append([label, value]) @@ -300,22 +304,27 @@ def display_status_summary( # pylint: disable=R0912 "UNKNOWN": {"count": 0, "color": ANSI_COLORS["GREY"], "fill": "?"}, "INITIALIZED": {"count": 0, "color": ANSI_COLORS["LIGHT_BLUE"]}, "RUNNING": {"count": 0, "color": ANSI_COLORS["BLUE"]}, - "DRY_RUN": {"count": 0, "color": ANSI_COLORS["ORANGE"], "fill": "\\"}, - "TOTAL_TASKS": {"total": status_obj.tasks_per_step[sstep]}, + "DRY RUN": {"count": 0, "color": ANSI_COLORS["ORANGE"], "fill": "\\"}, + "TOTAL TASKS": {"total": status_obj.tasks_per_step[sstep]}, + "AVG RUN TIME": status_obj.requested_statuses[sstep]["avg_run_time"], + "RUN TIME STD DEV": status_obj.requested_statuses[sstep]["run_time_std_dev"] } + # Initialize a var to track # of completed tasks and grab the statuses for this step num_completed_tasks = 0 + step_statuses = status_obj.requested_statuses[sstep] + # Loop through each entry for the step (if there's no parameters there will just be one entry) - LOG.debug(f"real_step_name_map[{sstep}]: {status_obj.real_step_name_map[sstep]}") - for real_step_name in status_obj.real_step_name_map[sstep]: - # Grab the statuses for this step - overall_step_info = status_obj.requested_statuses[real_step_name] + for real_step_name, overall_step_info in step_statuses.items(): + # Non-dict entries are just for run time info at the moment + if not isinstance(overall_step_info, dict): + continue # If this was a non-local run we should have a task queue and worker name to add to state_info if "task_queue" in overall_step_info: - state_info["TASK_QUEUE"] = {"name": overall_step_info["task_queue"]} + state_info["TASK QUEUE"] = {"name": overall_step_info["task_queue"]} if "worker_name" in overall_step_info: - state_info["WORKER_NAME"] = {"name": overall_step_info["worker_name"]} + state_info["WORKER NAME"] = {"name": overall_step_info["worker_name"]} # Loop through all workspaces for this step (if there's no samples for this step it'll just be one path) for sub_step_workspace, task_status_info in overall_step_info.items(): @@ -399,7 +408,7 @@ def display_progress_bar( # pylint: disable=R0913,R0914 print(f"\r{prefix} |", end="") for key, val in state_info.items(): # Only fill bar with completed tasks - if key in ("INITIALIZED", "RUNNING", "TASK_QUEUE", "WORKER_NAME", "TOTAL_TASKS"): + if key in ("INITIALIZED", "RUNNING", "TASK QUEUE", "WORKER NAME", "TOTAL TASKS", "AVG RUN TIME", "RUN TIME STD DEV"): continue # Get the length to fill for this specific state diff --git a/merlin/study/status.py b/merlin/study/status.py index 1e9180afb..285daa23e 100644 --- a/merlin/study/status.py +++ b/merlin/study/status.py @@ -30,6 +30,7 @@ """This module handles all the functionality of getting the statuses of studies""" import json import logging +import numpy as np import os import re from argparse import Namespace @@ -43,7 +44,7 @@ from merlin.common.dumper import dump_handler from merlin.display import ANSI_COLORS, display_status_summary from merlin.spec.expansion import get_spec_with_expansion -from merlin.utils import dict_deep_merge, verify_dirpath, ws_time_to_dt +from merlin.utils import convert_timestring, convert_to_timedelta, dict_deep_merge, pretty_format_HMS, verify_dirpath, ws_time_to_dt LOG = logging.getLogger(__name__) @@ -51,7 +52,9 @@ VALID_RETURN_CODES = ("SUCCESS", "SOFT_FAIL", "HARD_FAIL", "STOP_WORKERS", "RESTART", "RETRY", "DRY_SUCCESS", "UNRECOGNIZED") VALID_EXIT_FILTERS = ("E", "EXIT") ALL_VALID_FILTERS = VALID_STATUS_FILTERS + VALID_RETURN_CODES + VALID_EXIT_FILTERS + ("MAX_TASKS",) -NON_WORKSPACE_KEYS = set(["task_queue", "worker_name"]) +CELERY_KEYS = set(["task_queue", "worker_name"]) +RUN_TIME_STAT_KEYS = set(["avg_run_time", "run_time_std_dev"]) +NON_WORKSPACE_KEYS = CELERY_KEYS.union(RUN_TIME_STAT_KEYS) class Status: @@ -74,17 +77,12 @@ def __init__(self, args: Namespace, spec_display: bool, file_or_ws: str): # Verify the filter args (this will only do something for DetailedStatus) self._verify_filter_args() - # Create a step tracker that will tell us which steps we'll display that have started/not started + # Create a step tracker that will tell us which steps have started/not started self.step_tracker = self.get_steps_to_display() # Create a tasks per step mapping in order to give accurate totals for each step self.tasks_per_step = self.spec.get_tasks_per_step() - # This attribute will store a map between the overall step name and the real step names - # that are created with parameters (e.g. step name is hello and uses a "GREET: hello" parameter - # so the real step name is hello_GREET.hello) - self.real_step_name_map = {} - # Variable to store the statuses that the user wants self.requested_statuses = {} self.load_requested_statuses() @@ -296,8 +294,11 @@ def num_requested_statuses(self): We need to ignore non workspace keys when we count. """ num_statuses = 0 - for status_info in self.requested_statuses.values(): - num_statuses += len(status_info.keys() - NON_WORKSPACE_KEYS) + for step_name in self.step_tracker["started_steps"]: + for status_key, status_info in self.requested_statuses[step_name].items(): + if status_key in RUN_TIME_STAT_KEYS: + continue + num_statuses += len(status_info.keys() - NON_WORKSPACE_KEYS) return num_statuses def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict[str, List[str]]: @@ -309,9 +310,8 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict :param `started_step_name`: The name of the started step that we're getting statuses from :returns: A dict of statuses for the given step """ - step_statuses = {} + step_statuses = {started_step_name: {}} num_statuses_read = 0 - self.real_step_name_map[started_step_name] = set() # Traverse the step workspace and look for MERLIN_STATUS files LOG.info(f"Traversing '{step_workspace}' to find MERLIN_STATUS.json files...") @@ -321,18 +321,22 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict matching_files = glob(status_filepath) if matching_files: LOG.debug(f"Found status file at '{status_filepath}'") + # Read in the statuses and count how many statuses we read lock = FileLock(f"{root}/status.lock") # pylint: disable=E0110 statuses_read = read_status(status_filepath, lock) - - # Update map of real step names - self.real_step_name_map[started_step_name].update(list(statuses_read.keys())) + for status_info in statuses_read.values(): + num_statuses_read += len(status_info.keys() - NON_WORKSPACE_KEYS) # Merge the statuses we read with the dict tracking all statuses for this step - dict_deep_merge(step_statuses, statuses_read) + dict_deep_merge(step_statuses[started_step_name], statuses_read) + LOG.info( f"Done traversing '{step_workspace}'. Read in {num_statuses_read} {'statuses' if num_statuses_read != 1 else 'status'}." ) + # Calculate run time average and standard deviation for this step + step_statuses = self.get_runtime_avg_std_dev(step_statuses, started_step_name) + return step_statuses def load_requested_statuses(self): @@ -350,6 +354,47 @@ def load_requested_statuses(self): # Count how many statuses in total that we just read in LOG.info(f"Read in {self.num_requested_statuses} statuses total.") + def get_runtime_avg_std_dev(self, step_statuses: Dict, step_name: str) -> Dict: + """ + Calculate the mean and standard deviation for the runtime of each step. + Add this to the state information once calculated. + + :param `step_statuses`: A dict of step status information that we'll parse for run times + :param `step_name`: The name of the step + :returns: An updated dict of step status info with run time avg and std dev + """ + # Initialize a list to track all existing runtimes + run_times_in_seconds = [] + + # This outer loop will only loop once + LOG.info(f"Calculating run time avg and std dev for {step_name}...") + for _, overall_step_info in step_statuses[step_name].items(): + for step_info_key, step_status_info in overall_step_info.items(): + # Ignore non-workspace keys and any run times that have been yet to be calculated + if step_info_key in NON_WORKSPACE_KEYS or step_status_info["run_time"] == "--:--:--": + LOG.debug(f"Skipping {step_info_key}.") + continue + + # Parse the runtime value, convert it to seconds, and add it to the lsit of existing run times + run_time = step_status_info["run_time"].replace("d", "").replace("h", "").replace("m", "").replace("s", "") + run_time_tdelta = convert_to_timedelta(run_time) + run_times_in_seconds.append(run_time_tdelta.total_seconds()) + + # Using the list of existing run times, calculate avg and std dev + LOG.debug(f"Using the following run times for our calculations: {run_times_in_seconds}") + np_run_times_in_seconds = np.array(run_times_in_seconds) + run_time_mean = round(np.mean(np_run_times_in_seconds)) + run_time_std_dev = round(np.std(np_run_times_in_seconds)) + LOG.debug(f"Run time avg in seconds: {run_time_mean}") + LOG.debug(f"Run time std dev in seconds: {run_time_std_dev}") + + # Pretty format the avg and std dev and store them as new entries to the status information for the step + step_statuses[step_name]["avg_run_time"] = pretty_format_HMS(convert_timestring(run_time_mean)) + step_statuses[step_name]["run_time_std_dev"] = f"±{pretty_format_HMS(convert_timestring(run_time_std_dev))}" + LOG.info(f"Run time avg and std dev for {step_name} calculated.") + + return step_statuses + def display(self, test_mode=False) -> Dict: """ Displays the high level summary of the status. @@ -425,30 +470,36 @@ def format_status_for_display(self) -> Dict: "worker_name": [], } - # Loop through all statuses - for step_name, overall_step_info in self.requested_statuses.items(): - # Get the number of statuses for this step so we know how many entries there should be - num_statuses = len(overall_step_info.keys() - NON_WORKSPACE_KEYS) - - # Loop through information for each step - for step_info_key, step_info_value in overall_step_info.items(): - # Format non-workspace keys (task_queue and worker_name) - if step_info_key in NON_WORKSPACE_KEYS: - # Set the val_to_add value based on if a value exists for the key - val_to_add = step_info_value if step_info_value else "-------" - # Add the val_to_add entry for each row - key_entries = [val_to_add] * num_statuses - reformatted_statuses[step_info_key].extend(key_entries) - - # Format workspace keys - else: - # Put the step name and workspace in each entry - reformatted_statuses["step_name"].append(step_name) - reformatted_statuses["step_workspace"].append(step_info_key) - - # Add the rest of the information for each task (status, return code, elapsed & run time, num restarts) - for key, val in step_info_value.items(): - reformatted_statuses[key].append(val) + # We only care about started steps since unstarted steps won't have any status to report + for step_name in self.step_tracker["started_steps"]: + # Obtain and loop through all statuses + step_statuses = self.requested_statuses[step_name] + for full_step_name, overall_step_info in step_statuses.items(): + if full_step_name in RUN_TIME_STAT_KEYS: + continue + + # Get the number of statuses for this step so we know how many entries there should be + num_statuses = len(overall_step_info.keys() - NON_WORKSPACE_KEYS) + + # Loop through information for each step + for step_info_key, step_info_value in overall_step_info.items(): + # Format celery specific keys + if step_info_key in CELERY_KEYS: + # Set the val_to_add value based on if a value exists for the key + val_to_add = step_info_value if step_info_value else "-------" + # Add the val_to_add entry for each row + key_entries = [val_to_add] * num_statuses + reformatted_statuses[step_info_key].extend(key_entries) + + # Format workspace keys + else: + # Put the step name and workspace in each entry + reformatted_statuses["step_name"].append(step_name) + reformatted_statuses["step_workspace"].append(step_info_key) + + # Add the rest of the information for each task (status, return code, elapsed & run time, num restarts) + for key, val in step_info_value.items(): + reformatted_statuses[key].append(val) # For local runs, there will be no task queue or worker name so delete these entries for celery_specific_key in ("task_queue", "worker_name"): diff --git a/merlin/utils.py b/merlin/utils.py index dbd8d6d05..dbbd55ed2 100644 --- a/merlin/utils.py +++ b/merlin/utils.py @@ -649,6 +649,40 @@ def convert_timestring(timestring: Union[str, int], format_method: str = "HMS") return repr_timedelta(tdelta, method=format_method) +def pretty_format_HMS(timestring: str) -> str: + """ + Given an HMS timestring, format it so it removes blank entries and adds + labels. + + :param `timestring`: the HMS timestring we'll format + :returns: a formatted timestring + + Examples: + - "00:00:34:00" -> "34m" + - "01:00:00:25" -> "01d:25s" + - "00:19:44:28" -> "19h:44m:28s" + """ + # Create labels and split the timestring + labels = ["d", "h", "m", "s"] + parsed_ts = timestring.split(":") + if len(parsed_ts) > 4: + raise ValueError("The timestring to label must be in the format DD:HH:MM:SS") + + # Label each integer with its corresponding unit + labeled_time_list = [] + for i in range(1, len(parsed_ts) + 1): + if parsed_ts[-i] != "00": + labeled_time_list.append(parsed_ts[-i] + labels[-i]) + + # Join the labeled time list into a string. + if len(labeled_time_list) == 0: + labeled_time_list.append("00s") + labeled_time_list.reverse() + labeled_time_string = ":".join(labeled_time_list) + + return labeled_time_string + + def ws_time_to_dt(ws_time: str) -> datetime: """ Converts a workspace timestring to a datetime object. From f2b23d827ff4fa79dccebe3c9d5c780196b5a863 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Wed, 27 Sep 2023 12:26:02 -0700 Subject: [PATCH 11/13] modify status tests to accommodate new avg/std dev calculations --- .../study/status_test_files/shared_tests.py | 6 - .../MERLIN_STATUS.json | 4 +- .../GREET.hola.LEAVE.adios/MERLIN_STATUS.json | 4 +- .../just_samples/MERLIN_STATUS.json | 20 +- .../GREET.hello/MERLIN_STATUS.json | 20 +- .../GREET.hola/MERLIN_STATUS.json | 20 +- .../status_test_variables.py | 457 +++++++++--------- tests/unit/study/test_status.py | 2 +- 8 files changed, 275 insertions(+), 258 deletions(-) diff --git a/tests/unit/study/status_test_files/shared_tests.py b/tests/unit/study/status_test_files/shared_tests.py index 601fcd5d2..ecf553b56 100644 --- a/tests/unit/study/status_test_files/shared_tests.py +++ b/tests/unit/study/status_test_files/shared_tests.py @@ -63,12 +63,6 @@ def assert_correct_attribute_creation(status_obj: Union[Status, DetailedStatus]) tasks_per_step_diff = DeepDiff(status_obj.tasks_per_step, status_test_variables.TASKS_PER_STEP, ignore_order=True) assert tasks_per_step_diff == {} - # Ensuring real_step_name_map was created properly - real_step_name_map_diff = DeepDiff( - status_obj.real_step_name_map, status_test_variables.REAL_STEP_NAME_MAP, ignore_order=True - ) - assert real_step_name_map_diff == {} - # Ensuring requested_statuses was created properly requested_statuses_diff = DeepDiff( status_obj.requested_statuses, status_test_variables.ALL_REQUESTED_STATUSES, ignore_order=True diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json index 320f10731..b70bfd369 100644 --- a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hello.LEAVE.goodbye/MERLIN_STATUS.json @@ -5,8 +5,8 @@ "just_parameters/GREET.hello.LEAVE.goodbye": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:30s", "restarts": 0 } } diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json index 8249207cb..2640668f7 100644 --- a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_parameters/GREET.hola.LEAVE.adios/MERLIN_STATUS.json @@ -5,8 +5,8 @@ "just_parameters/GREET.hola.LEAVE.adios": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:01m:00s", + "run_time": "0d:00h:01m:00s", "restarts": 0 } } diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json index e73dd2977..3f49c3df2 100644 --- a/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/just_samples/MERLIN_STATUS.json @@ -5,36 +5,36 @@ "just_samples/00": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:00s", "restarts": 0 }, "just_samples/01": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:15s", "restarts": 0 }, "just_samples/02": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:30s", "restarts": 0 }, "just_samples/03": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:45s", "restarts": 0 }, "just_samples/04": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:02m:00s", "restarts": 0 } } diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json index 3614e500f..42d38c58d 100644 --- a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hello/MERLIN_STATUS.json @@ -5,36 +5,36 @@ "params_and_samples/GREET.hello/00": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:10s", "restarts": 0 }, "params_and_samples/GREET.hello/01": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:11s", "restarts": 0 }, "params_and_samples/GREET.hello/02": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:12s", "restarts": 0 }, "params_and_samples/GREET.hello/03": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:13s", "restarts": 0 }, "params_and_samples/GREET.hello/04": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:14s", "restarts": 0 } } diff --git a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json index 8d9479c66..6d3596b5b 100644 --- a/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json +++ b/tests/unit/study/status_test_files/status_test_study_20230717-162921/params_and_samples/GREET.hola/MERLIN_STATUS.json @@ -5,36 +5,36 @@ "params_and_samples/GREET.hola/00": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:10s", "restarts": 0 }, "params_and_samples/GREET.hola/01": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:18s", "restarts": 0 }, "params_and_samples/GREET.hola/02": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:23s", "restarts": 0 }, "params_and_samples/GREET.hola/03": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:29s", "restarts": 0 }, "params_and_samples/GREET.hola/04": { "status": "FINISHED", "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:16s", "restarts": 0 } } diff --git a/tests/unit/study/status_test_files/status_test_variables.py b/tests/unit/study/status_test_files/status_test_variables.py index fd4c36778..0832204e5 100644 --- a/tests/unit/study/status_test_files/status_test_variables.py +++ b/tests/unit/study/status_test_files/status_test_variables.py @@ -55,13 +55,6 @@ "cancel_step": 1, "unstarted_step": 1, } -REAL_STEP_NAME_MAP = { - "just_samples": set(["just_samples"]), - "just_parameters": set(["just_parameters_GREET.hello.LEAVE.goodbye", "just_parameters_GREET.hola.LEAVE.adios"]), - "params_and_samples": set(["params_and_samples_GREET.hello", "params_and_samples_GREET.hola"]), - "fail_step": set(["fail_step"]), - "cancel_step": set(["cancel_step"]), -} NUM_ALL_REQUESTED_STATUSES = sum(TASKS_PER_STEP.values()) - TASKS_PER_STEP["unstarted_step"] # This variable holds the state_info dict of every step from VALID_WORKSPACE @@ -74,10 +67,12 @@ "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, - "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, - "TOTAL_TASKS": {"total": 5}, - "WORKER_NAME": {"name": "sample_worker"}, - "TASK_QUEUE": {"name": "just_samples_queue"}, + "DRY RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL TASKS": {"total": 5}, + "WORKER NAME": {"name": "sample_worker"}, + "TASK QUEUE": {"name": "just_samples_queue"}, + "AVG RUN TIME": "01m:30s", + "RUN TIME STD DEV": "±21s", }, "just_parameters": { "FINISHED": {"count": 2, "color": "\033[38;2;0;158;115m", "fill": "█"}, @@ -86,10 +81,12 @@ "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, - "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, - "TOTAL_TASKS": {"total": 2}, - "WORKER_NAME": {"name": "other_worker"}, - "TASK_QUEUE": {"name": "just_parameters_queue"}, + "DRY RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL TASKS": {"total": 2}, + "WORKER NAME": {"name": "other_worker"}, + "TASK QUEUE": {"name": "just_parameters_queue"}, + "AVG RUN TIME": "01m:15s", + "RUN TIME STD DEV": "±15s", }, "params_and_samples": { "FINISHED": {"count": 10, "color": "\033[38;2;0;158;115m", "fill": "█"}, @@ -98,10 +95,12 @@ "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, - "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, - "TOTAL_TASKS": {"total": 10}, - "WORKER_NAME": {"name": "sample_worker"}, - "TASK_QUEUE": {"name": "both_queue"}, + "DRY RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL TASKS": {"total": 10}, + "WORKER NAME": {"name": "sample_worker"}, + "TASK QUEUE": {"name": "both_queue"}, + "AVG RUN TIME": "16s", + "RUN TIME STD DEV": "±06s", }, "fail_step": { "FINISHED": {"count": 0, "color": "\033[38;2;0;158;115m", "fill": "█"}, @@ -110,10 +109,12 @@ "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, - "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, - "TOTAL_TASKS": {"total": 1}, - "WORKER_NAME": {"name": "other_worker"}, - "TASK_QUEUE": {"name": "fail_queue"}, + "DRY RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL TASKS": {"total": 1}, + "WORKER NAME": {"name": "other_worker"}, + "TASK QUEUE": {"name": "fail_queue"}, + "AVG RUN TIME": "00s", + "RUN TIME STD DEV": "±00s", }, "cancel_step": { "FINISHED": {"count": 0, "color": "\033[38;2;0;158;115m", "fill": "█"}, @@ -122,10 +123,12 @@ "UNKNOWN": {"count": 0, "color": "\033[38;2;102;102;102m", "fill": "?"}, "INITIALIZED": {"count": 0, "color": "\033[38;2;86;180;233m"}, "RUNNING": {"count": 0, "color": "\033[38;2;0;114;178m"}, - "DRY_RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, - "TOTAL_TASKS": {"total": 1}, - "WORKER_NAME": {"name": "other_worker"}, - "TASK_QUEUE": {"name": "cancel_queue"}, + "DRY RUN": {"count": 0, "color": "\033[38;2;230;159;0m", "fill": "\\"}, + "TOTAL TASKS": {"total": 1}, + "WORKER NAME": {"name": "other_worker"}, + "TASK QUEUE": {"name": "cancel_queue"}, + "AVG RUN TIME": "00s", + "RUN TIME STD DEV": "±00s", }, "unstarted_step": "UNSTARTED", } @@ -133,165 +136,185 @@ # This variable holds every status from the VALID_WORKSPACE in the format used when we first load them in # i.e. the format loaded in by load_requested_statuses() ALL_REQUESTED_STATUSES = { - "just_parameters_GREET.hello.LEAVE.goodbye": { - "task_queue": "just_parameters_queue", - "worker_name": "other_worker", - "just_parameters/GREET.hello.LEAVE.goodbye": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "just_parameters": { + "avg_run_time": "01m:15s", + "run_time_std_dev": "±15s", + "just_parameters_GREET.hello.LEAVE.goodbye": { + "task_queue": "just_parameters_queue", + "worker_name": "other_worker", + "just_parameters/GREET.hello.LEAVE.goodbye": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:30s", + "restarts": 0, + }, }, - }, - "just_parameters_GREET.hola.LEAVE.adios": { - "task_queue": "just_parameters_queue", - "worker_name": "other_worker", - "just_parameters/GREET.hola.LEAVE.adios": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "just_parameters_GREET.hola.LEAVE.adios": { + "task_queue": "just_parameters_queue", + "worker_name": "other_worker", + "just_parameters/GREET.hola.LEAVE.adios": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:01m:00s", + "run_time": "0d:00h:01m:00s", + "restarts": 0, + }, }, }, "just_samples": { - "task_queue": "just_samples_queue", - "worker_name": "sample_worker", - "just_samples/00": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "just_samples/01": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "just_samples/02": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "just_samples/03": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "just_samples/04": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - }, - "params_and_samples_GREET.hello": { - "task_queue": "both_queue", - "worker_name": "sample_worker", - "params_and_samples/GREET.hello/00": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hello/01": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hello/02": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hello/03": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hello/04": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "avg_run_time": "01m:30s", + "run_time_std_dev": "±21s", + "just_samples": { + "task_queue": "just_samples_queue", + "worker_name": "sample_worker", + "just_samples/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:00s", + "restarts": 0, + }, + "just_samples/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:15s", + "restarts": 0, + }, + "just_samples/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:30s", + "restarts": 0, + }, + "just_samples/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:01m:45s", + "restarts": 0, + }, + "just_samples/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:02m:00s", + "run_time": "0d:00h:02m:00s", + "restarts": 0, + }, }, }, - "params_and_samples_GREET.hola": { - "task_queue": "both_queue", - "worker_name": "sample_worker", - "params_and_samples/GREET.hola/00": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hola/01": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hola/02": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, - }, - "params_and_samples/GREET.hola/03": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "params_and_samples": { + "avg_run_time": "16s", + "run_time_std_dev": "±06s", + "params_and_samples_GREET.hello": { + "task_queue": "both_queue", + "worker_name": "sample_worker", + "params_and_samples/GREET.hello/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:10s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:11s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:12s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:13s", + "restarts": 0, + }, + "params_and_samples/GREET.hello/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:15s", + "run_time": "0d:00h:00m:14s", + "restarts": 0, + }, }, - "params_and_samples/GREET.hola/04": { - "status": "FINISHED", - "return_code": "MERLIN_SUCCESS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "params_and_samples_GREET.hola": { + "task_queue": "both_queue", + "worker_name": "sample_worker", + "params_and_samples/GREET.hola/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:10s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:18s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/02": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:23s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/03": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:29s", + "restarts": 0, + }, + "params_and_samples/GREET.hola/04": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:00h:00m:30s", + "run_time": "0d:00h:00m:16s", + "restarts": 0, + }, }, }, "fail_step": { - "task_queue": "fail_queue", - "worker_name": "other_worker", + "avg_run_time": "00s", + "run_time_std_dev": "±00s", "fail_step": { - "status": "FAILED", - "return_code": "MERLIN_SOFT_FAIL", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "task_queue": "fail_queue", + "worker_name": "other_worker", + "fail_step": { + "status": "FAILED", + "return_code": "MERLIN_SOFT_FAIL", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, }, }, "cancel_step": { - "task_queue": "cancel_queue", - "worker_name": "other_worker", + "avg_run_time": "00s", + "run_time_std_dev": "±00s", "cancel_step": { - "status": "CANCELLED", - "return_code": "MERLIN_STOP_WORKERS", - "elapsed_time": "0d:00h:00m:00s", - "run_time": "0d:00h:00m:00s", - "restarts": 0, + "task_queue": "cancel_queue", + "worker_name": "other_worker", + "cancel_step": { + "status": "CANCELLED", + "return_code": "MERLIN_STOP_WORKERS", + "elapsed_time": "0d:00h:00m:00s", + "run_time": "0d:00h:00m:00s", + "restarts": 0, + }, }, }, } @@ -300,23 +323,23 @@ # i.e. the format returned by format_status_for_display() ALL_FORMATTED_STATUSES = { "step_name": [ - "just_parameters_GREET.hello.LEAVE.goodbye", - "just_parameters_GREET.hola.LEAVE.adios", + "just_parameters", + "just_parameters", "just_samples", "just_samples", "just_samples", "just_samples", "just_samples", - "params_and_samples_GREET.hello", - "params_and_samples_GREET.hello", - "params_and_samples_GREET.hello", - "params_and_samples_GREET.hello", - "params_and_samples_GREET.hello", - "params_and_samples_GREET.hola", - "params_and_samples_GREET.hola", - "params_and_samples_GREET.hola", - "params_and_samples_GREET.hola", - "params_and_samples_GREET.hola", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", + "params_and_samples", "fail_step", "cancel_step", ], @@ -384,44 +407,44 @@ "MERLIN_STOP_WORKERS", ], "elapsed_time": [ - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", + "0d:00h:02m:00s", + "0d:00h:01m:00s", + "0d:00h:02m:00s", + "0d:00h:02m:00s", + "0d:00h:02m:00s", + "0d:00h:02m:00s", + "0d:00h:02m:00s", + "0d:00h:00m:15s", + "0d:00h:00m:15s", + "0d:00h:00m:15s", + "0d:00h:00m:15s", + "0d:00h:00m:15s", + "0d:00h:00m:30s", + "0d:00h:00m:30s", + "0d:00h:00m:30s", + "0d:00h:00m:30s", + "0d:00h:00m:30s", "0d:00h:00m:00s", "0d:00h:00m:00s", ], "run_time": [ - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", - "0d:00h:00m:00s", + "0d:00h:01m:30s", + "0d:00h:01m:00s", + "0d:00h:01m:00s", + "0d:00h:01m:15s", + "0d:00h:01m:30s", + "0d:00h:01m:45s", + "0d:00h:02m:00s", + "0d:00h:00m:10s", + "0d:00h:00m:11s", + "0d:00h:00m:12s", + "0d:00h:00m:13s", + "0d:00h:00m:14s", + "0d:00h:00m:10s", + "0d:00h:00m:18s", + "0d:00h:00m:23s", + "0d:00h:00m:29s", + "0d:00h:00m:16s", "0d:00h:00m:00s", "0d:00h:00m:00s", ], diff --git a/tests/unit/study/test_status.py b/tests/unit/study/test_status.py index 76be5a307..d4a13c6f9 100644 --- a/tests/unit/study/test_status.py +++ b/tests/unit/study/test_status.py @@ -266,7 +266,7 @@ def test_display(self): assert step_name in status_test_variables.FULL_STEP_TRACKER["unstarted_steps"] # Make sure all the state info dicts for each step match what they should be - state_info_diff = DeepDiff(state_info, status_test_variables.DISPLAY_INFO[step_name]) + state_info_diff = DeepDiff(state_info, status_test_variables.DISPLAY_INFO[step_name], ignore_order=True) self.assertEqual(state_info_diff, {}) From d63a54c85ad346ce9c427f006553aea5af61da5c Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Wed, 27 Sep 2023 12:30:58 -0700 Subject: [PATCH 12/13] fix linter issues --- merlin/common/tasks.py | 4 ++-- merlin/display.py | 12 ++++++++++-- merlin/study/status.py | 20 ++++++++++++++------ 3 files changed, 26 insertions(+), 10 deletions(-) diff --git a/merlin/common/tasks.py b/merlin/common/tasks.py index 3894cf85b..50477c3c6 100644 --- a/merlin/common/tasks.py +++ b/merlin/common/tasks.py @@ -444,8 +444,8 @@ def get_1d_chain(all_chains: List[List["Signature"]]) -> List["Signature"]: # n def gather_statuses( - sample_index: "SampleIndex", workspace: str, condensed_workspace: str, files_to_remove: List[str] -) -> Dict: # noqa: F821 + sample_index: "SampleIndex", workspace: str, condensed_workspace: str, files_to_remove: List[str] # noqa: F821 +) -> Dict: """ Traverse the sample index and gather all of the statuses into one. diff --git a/merlin/display.py b/merlin/display.py index 52a46cd44..2cf2adf4a 100644 --- a/merlin/display.py +++ b/merlin/display.py @@ -307,7 +307,7 @@ def display_status_summary( # pylint: disable=R0912 "DRY RUN": {"count": 0, "color": ANSI_COLORS["ORANGE"], "fill": "\\"}, "TOTAL TASKS": {"total": status_obj.tasks_per_step[sstep]}, "AVG RUN TIME": status_obj.requested_statuses[sstep]["avg_run_time"], - "RUN TIME STD DEV": status_obj.requested_statuses[sstep]["run_time_std_dev"] + "RUN TIME STD DEV": status_obj.requested_statuses[sstep]["run_time_std_dev"], } # Initialize a var to track # of completed tasks and grab the statuses for this step @@ -408,7 +408,15 @@ def display_progress_bar( # pylint: disable=R0913,R0914 print(f"\r{prefix} |", end="") for key, val in state_info.items(): # Only fill bar with completed tasks - if key in ("INITIALIZED", "RUNNING", "TASK QUEUE", "WORKER NAME", "TOTAL TASKS", "AVG RUN TIME", "RUN TIME STD DEV"): + if key in ( + "INITIALIZED", + "RUNNING", + "TASK QUEUE", + "WORKER NAME", + "TOTAL TASKS", + "AVG RUN TIME", + "RUN TIME STD DEV", + ): continue # Get the length to fill for this specific state diff --git a/merlin/study/status.py b/merlin/study/status.py index 285daa23e..05eccc019 100644 --- a/merlin/study/status.py +++ b/merlin/study/status.py @@ -30,7 +30,6 @@ """This module handles all the functionality of getting the statuses of studies""" import json import logging -import numpy as np import os import re from argparse import Namespace @@ -38,13 +37,21 @@ from glob import glob from typing import Dict, List, Optional, Tuple +import numpy as np from filelock import FileLock, Timeout from tabulate import tabulate from merlin.common.dumper import dump_handler from merlin.display import ANSI_COLORS, display_status_summary from merlin.spec.expansion import get_spec_with_expansion -from merlin.utils import convert_timestring, convert_to_timedelta, dict_deep_merge, pretty_format_HMS, verify_dirpath, ws_time_to_dt +from merlin.utils import ( + convert_timestring, + convert_to_timedelta, + dict_deep_merge, + pretty_format_HMS, + verify_dirpath, + ws_time_to_dt, +) LOG = logging.getLogger(__name__) @@ -315,7 +322,7 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict # Traverse the step workspace and look for MERLIN_STATUS files LOG.info(f"Traversing '{step_workspace}' to find MERLIN_STATUS.json files...") - for root, _, files in os.walk(step_workspace): + for root, _, _ in os.walk(step_workspace): # Search for a status file status_filepath = os.path.join(root, "MERLIN_STATUS.json") matching_files = glob(status_filepath) @@ -331,7 +338,8 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict dict_deep_merge(step_statuses[started_step_name], statuses_read) LOG.info( - f"Done traversing '{step_workspace}'. Read in {num_statuses_read} {'statuses' if num_statuses_read != 1 else 'status'}." + f"Done traversing '{step_workspace}'. Read in {num_statuses_read} " \ + f"{'statuses' if num_statuses_read != 1 else 'status'}." ) # Calculate run time average and standard deviation for this step @@ -365,7 +373,7 @@ def get_runtime_avg_std_dev(self, step_statuses: Dict, step_name: str) -> Dict: """ # Initialize a list to track all existing runtimes run_times_in_seconds = [] - + # This outer loop will only loop once LOG.info(f"Calculating run time avg and std dev for {step_name}...") for _, overall_step_info in step_statuses[step_name].items(): @@ -392,7 +400,7 @@ def get_runtime_avg_std_dev(self, step_statuses: Dict, step_name: str) -> Dict: step_statuses[step_name]["avg_run_time"] = pretty_format_HMS(convert_timestring(run_time_mean)) step_statuses[step_name]["run_time_std_dev"] = f"±{pretty_format_HMS(convert_timestring(run_time_std_dev))}" LOG.info(f"Run time avg and std dev for {step_name} calculated.") - + return step_statuses def display(self, test_mode=False) -> Dict: From 672b1a70885b341976d78fff2d94448357ec11d3 Mon Sep 17 00:00:00 2001 From: Brian Gunnarson Date: Wed, 27 Sep 2023 13:12:08 -0700 Subject: [PATCH 13/13] fix lint issue and add test for avg/std dev calc --- merlin/study/status.py | 2 +- tests/unit/study/test_status.py | 57 +++++++++++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 1 deletion(-) diff --git a/merlin/study/status.py b/merlin/study/status.py index 05eccc019..7ad15a0ce 100644 --- a/merlin/study/status.py +++ b/merlin/study/status.py @@ -338,7 +338,7 @@ def get_step_statuses(self, step_workspace: str, started_step_name: str) -> Dict dict_deep_merge(step_statuses[started_step_name], statuses_read) LOG.info( - f"Done traversing '{step_workspace}'. Read in {num_statuses_read} " \ + f"Done traversing '{step_workspace}'. Read in {num_statuses_read} " f"{'statuses' if num_statuses_read != 1 else 'status'}." ) diff --git a/tests/unit/study/test_status.py b/tests/unit/study/test_status.py index d4a13c6f9..b97f7eeb2 100644 --- a/tests/unit/study/test_status.py +++ b/tests/unit/study/test_status.py @@ -269,6 +269,63 @@ def test_display(self): state_info_diff = DeepDiff(state_info, status_test_variables.DISPLAY_INFO[step_name], ignore_order=True) self.assertEqual(state_info_diff, {}) + def test_get_runtime_avg_std_dev(self): + """ + Test the functionality that calculates the run time average and standard + deviation for each step. This test covers the get_runtime_avg_std_dev method. + """ + dummy_step_status = { + "dummy_step": { + "dummy_step_PARAM.1": { + "task_queue": "dummy_queue", + "worker_name": "dummy_worker", + "dummy_step/PARAM.1/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:02h:00m:00s", + "run_time": "0d:01h:38m:27s", # 3600 + 2280 + 27 = 5907 seconds + "restarts": 0, + }, + "dummy_step/PARAM.1/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:02h:00m:00s", + "run_time": "0d:01h:45m:08s", # 3600 + 2700 + 8 = 6308 seconds + "restarts": 0, + }, + }, + "dummy_step_PARAM.2": { + "task_queue": "dummy_queue", + "worker_name": "dummy_worker", + "dummy_step/PARAM.2/00": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:02h:00m:00s", + "run_time": "0d:01h:52m:33s", # 3600 + 3120 + 33 = 6753 seconds + "restarts": 0, + }, + "dummy_step/PARAM.2/01": { + "status": "FINISHED", + "return_code": "MERLIN_SUCCESS", + "elapsed_time": "0d:02h:00m:00s", + "run_time": "0d:01h:08m:40s", # 3600 + 480 + 40 = 4120 seconds + "restarts": 0, + }, + }, + } + } + + status_obj = Status(args=self.args, spec_display=False, file_or_ws=status_test_variables.VALID_WORKSPACE_PATH) + status_obj.get_runtime_avg_std_dev(dummy_step_status, "dummy_step") + + # Set expected values + expected_avg = "01h:36m:12s" # Mean is 5772 seconds = 01h:36m:12s + expected_std_dev = "±16m:40s" # Std dev is 1000 seconds = 16m:40s + + # Make sure the values were calculated as expected + self.assertEqual(dummy_step_status["dummy_step"]["avg_run_time"], expected_avg) + self.assertEqual(dummy_step_status["dummy_step"]["run_time_std_dev"], expected_std_dev) + if __name__ == "__main__": unittest.main()