From 642ab8d22652f0aa3d5576d29736355a98b337d0 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Mon, 19 Jun 2023 12:22:29 +0100 Subject: [PATCH 01/24] reload: wait for pending tasks to submit and pause the workflow * Closes https://github.com/cylc/cylc-flow/issues/5107 * Reload now waits for pending tasks to submit before attempting to reload the config itself. * Reload now also puts the workflow into the paused state during the reload process. This doesn't actually achieve anything as the reload command is blocking in the main loop, but it does help to communicate that the workflow will not de-queue or submit and new tasks during this process. * The workflow status message is now updated to reflect the reload progress. --- CHANGES.md | 4 + cylc/flow/scheduler.py | 227 +++++++++++++++++++++------ cylc/flow/workflow_status.py | 3 + tests/conftest.py | 32 ++++ tests/functional/logging/01-basic.t | 8 +- tests/functional/reload/09-garbage.t | 4 +- tests/integration/test_reload.py | 148 +++++++++++++++++ tests/integration/test_resolvers.py | 2 +- tests/integration/test_scheduler.py | 74 --------- tests/integration/test_task_pool.py | 6 +- tests/unit/test_workflow_status.py | 10 +- 11 files changed, 385 insertions(+), 133 deletions(-) create mode 100644 tests/integration/test_reload.py diff --git a/CHANGES.md b/CHANGES.md index 7370fff01c7..41128f3a19e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -14,6 +14,10 @@ ones in. --> ### Enhancements +[#5992](https://github.com/cylc/cylc-flow/pull/5992) - +The scheduler will now wait for preparing tasks to submit before attempting +to perform a reload and will also pause/unpause the workflow. + [#5405](https://github.com/cylc/cylc-flow/pull/5405) - Improve scan command help, and add scheduler PID to the output. diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 7b6cfe6a44d..16f0a55c1b0 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -20,6 +20,7 @@ from collections import deque from optparse import Values import os +import inspect from pathlib import Path from queue import Empty, Queue from shlex import quote @@ -32,12 +33,12 @@ from typing import ( TYPE_CHECKING, Callable, + Dict, Iterable, + List, NoReturn, Optional, - List, Set, - Dict, Tuple, Union, ) @@ -150,6 +151,10 @@ if TYPE_CHECKING: from cylc.flow.task_proxy import TaskProxy + # BACK COMPAT: typing_extensions.Literal + # FROM: Python 3.7 + # TO: Python 3.8 + from typing_extensions import Literal class SchedulerStop(CylcError): @@ -244,6 +249,7 @@ class Scheduler: stop_mode: Optional[StopMode] = None stop_task: Optional[str] = None stop_clock_time: Optional[int] = None + reload_pending: 'Union[Literal[False], str]' = False # task event loop is_paused = False @@ -427,7 +433,8 @@ async def configure(self): self.profiler.log_memory("scheduler.py: before load_flow_file") try: - self.load_flow_file() + cfg = self.load_flow_file() + self.apply_new_config(cfg, is_reload=False) except ParsecError as exc: # Mark this exc as expected (see docstring for .schd_expected): exc.schd_expected = True @@ -521,8 +528,7 @@ async def configure(self): self.command_set_hold_point(holdcp) if self.options.paused_start: - LOG.info("Paused on start up") - self.pause_workflow() + self.pause_workflow('Paused on start up') self.profiler.log_memory("scheduler.py: begin run while loop") self.is_updated = True @@ -881,7 +887,7 @@ def queue_command(self, command: str, kwargs: dict) -> None: tuple(kwargs.values()), {} )) - def process_command_queue(self) -> None: + async def process_command_queue(self) -> None: """Process queued commands.""" qsize = self.command_queue.qsize() if qsize <= 0: @@ -900,8 +906,12 @@ def process_command_queue(self) -> None: sep = ', ' if kwargs_string and args_string else '' cmdstr = f"{name}({args_string}{sep}{kwargs_string})" try: - n_warnings: Optional[int] = self.get_command_method(name)( - *args, **kwargs) + fcn = self.get_command_method(name) + n_warnings: Optional[int] + if inspect.iscoroutinefunction(fcn): + n_warnings = await fcn(*args, **kwargs) + else: + n_warnings = fcn(*args, **kwargs) except Exception as exc: # Don't let a bad command bring the workflow down. if ( @@ -1046,37 +1056,95 @@ def command_remove_tasks(self, items) -> int: """Remove tasks.""" return self.pool.remove_tasks(items) - def command_reload_workflow(self) -> None: + async def command_reload_workflow(self) -> None: """Reload workflow configuration.""" - LOG.info("Reloading the workflow definition.") - old_tasks = set(self.config.get_task_name_list()) - # Things that can't change on workflow reload: - self.workflow_db_mgr.pri_dao.select_workflow_params( - self._load_workflow_params - ) + # pause the workflow if not already + was_paused_before_reload = self.is_paused + if not was_paused_before_reload: + self.pause_workflow('Reloading workflow') + self.process_workflow_db_queue() # see #5593 + + # flush out preparing tasks before attempting reload + self.reload_pending = 'waiting for pending tasks to submit' + while self.release_queued_tasks(): + # Run the subset of main-loop functionality required to push + # preparing through the submission pipeline and keep the workflow + # responsive (e.g. to the `cylc stop` command). + + # NOTE: this reload method was called by process_command_queue + # which is called synchronously in the main loop so this call is + # blocking to other main loop functions + + # subproc pool - for issueing/tracking remote-init commands + self.proc_pool.process() + # task messages - for tracking task status changes + self.process_queued_task_messages() + # command queue - keeps the scheduler responsive + await self.process_command_queue() + # allows the scheduler to shutdown --now + await self.workflow_shutdown() + # keep the data store up to date with what's going on + await self.update_data_structure() + self.update_data_store() + # give commands time to complete + sleep(1) # give any remove-init's time to complete + # reload the workflow definition + self.reload_pending = 'loading the workflow definition' + self.update_data_store() # update workflow status msg + self._update_workflow_state() + LOG.info("Reloading the workflow definition.") try: - self.load_flow_file(is_reload=True) + cfg = self.load_flow_file(is_reload=True) except (ParsecError, CylcConfigError) as exc: - raise CommandFailedError(exc) - self.broadcast_mgr.linearized_ancestors = ( - self.config.get_linearized_ancestors()) - self.pool.set_do_reload(self.config) - self.task_events_mgr.mail_interval = self.cylc_config['mail'][ - 'task event batch interval'] - self.task_events_mgr.mail_smtp = self._get_events_conf("smtp") - self.task_events_mgr.mail_footer = self._get_events_conf("footer") - - # Log tasks that have been added by the reload, removed tasks are - # logged by the TaskPool. - add = set(self.config.get_task_name_list()) - old_tasks - for task in add: - LOG.warning(f"Added task: '{task}'") - self.workflow_db_mgr.put_workflow_template_vars(self.template_vars) - self.workflow_db_mgr.put_runtime_inheritance(self.config) - self.workflow_db_mgr.put_workflow_params(self) - self.is_updated = True - self.is_reloaded = True + if cylc.flow.flags.verbosity > 1: + # log full traceback in debug mode + LOG.exception(exc) + LOG.critical( + f'Reload failed - {exc.__class__.__name__}: {exc}' + '\nThis is probably due to an issue with the new' + ' configuration.' + '\nTo continue with the pre-reload config, un-pause the' + ' workflow.' + '\nOtherwise, fix the configuration and attempt to reload' + ' again.' + ) + else: + self.reload_pending = 'applying the new config' + old_tasks = set(self.config.get_task_name_list()) + # Things that can't change on workflow reload: + self.workflow_db_mgr.pri_dao.select_workflow_params( + self._load_workflow_params + ) + self.apply_new_config(cfg, is_reload=True) + self.broadcast_mgr.linearized_ancestors = ( + self.config.get_linearized_ancestors()) + self.pool.set_do_reload(self.config) + self.task_events_mgr.mail_interval = self.cylc_config['mail'][ + 'task event batch interval'] + self.task_events_mgr.mail_smtp = self._get_events_conf("smtp") + self.task_events_mgr.mail_footer = self._get_events_conf("footer") + + # Log tasks that have been added by the reload, removed tasks are + # logged by the TaskPool. + add = set(self.config.get_task_name_list()) - old_tasks + for task in add: + LOG.warning(f"Added task: '{task}'") + self.workflow_db_mgr.put_workflow_template_vars(self.template_vars) + self.workflow_db_mgr.put_runtime_inheritance(self.config) + self.workflow_db_mgr.put_workflow_params(self) + self.process_workflow_db_queue() # see #5593 + self.is_updated = True + self.is_reloaded = True + self._update_workflow_state() + + # resume the workflow if previously paused + self.reload_pending = False + self.update_data_store() # update workflow status msg + self._update_workflow_state() + if not was_paused_before_reload: + self.resume_workflow() + self.process_workflow_db_queue() # see #5593 def get_restart_num(self) -> int: """Return the number of the restart, else 0 if not a restart. @@ -1147,7 +1215,7 @@ def _configure_contact(self) -> None: def load_flow_file(self, is_reload=False): """Load, and log the workflow definition.""" # Local workflow environment set therein. - self.config = WorkflowConfig( + return WorkflowConfig( self.workflow, self.flow_file, self.options, @@ -1163,11 +1231,13 @@ def load_flow_file(self, is_reload=False): work_dir=self.workflow_work_dir, share_dir=self.workflow_share_dir, ) + + def apply_new_config(self, config, is_reload=False): + self.config = config self.cylc_config = DictTree( self.config.cfg['scheduler'], glbl_cfg().get(['scheduler']) ) - self.flow_file_update_time = time() # Dump the loaded flow.cylc file for future reference. config_dir = get_workflow_run_config_log_dir( @@ -1294,7 +1364,7 @@ def run_event_handlers(self, event, reason=""): return self.workflow_event_handler.handle(self, event, str(reason)) - def release_queued_tasks(self) -> None: + def release_queued_tasks(self) -> bool: """Release queued tasks, and submit jobs. The task queue manages references to task proxies in the task pool. @@ -1312,19 +1382,32 @@ def release_queued_tasks(self) -> None: * https://github.com/cylc/cylc-flow/pull/4620 * https://github.com/cylc/cylc-flow/issues/4974 + Returns: + True if tasks were passed through the submit-pipeline + (i.e. new waiting tasks have entered the preparing state OR + preparing tasks have been passed back through for + submission). + """ if ( not self.is_paused and self.stop_mode is None and self.auto_restart_time is None + and self.reload_pending is False ): pre_prep_tasks = self.pool.release_queued_tasks() elif ( - self.should_auto_restart_now() - and self.auto_restart_mode == AutoRestartMode.RESTART_NORMAL + ( + # Need to get preparing tasks to submit before auto restart + self.should_auto_restart_now() + and self.auto_restart_mode == AutoRestartMode.RESTART_NORMAL + ) or ( + # Need to get preparing tasks to submit before reload + self.reload_pending + ) ): - # Need to get preparing tasks to submit before auto restart + # don't release queued tasks, finish processing preparing tasks pre_prep_tasks = [ itask for itask in self.pool.get_tasks() if itask.state(TASK_STATUS_PREPARING) @@ -1332,9 +1415,9 @@ def release_queued_tasks(self) -> None: # Return, if no tasks to submit. else: - return + return False if not pre_prep_tasks: - return + return False # Start the job submission process. self.is_updated = True @@ -1362,6 +1445,9 @@ def release_queued_tasks(self) -> None: f"{itask.state.get_resolved_dependencies()} in flow {flow}" ) + # one or more tasks were passed through the submission pipeline + return True + def process_workflow_db_queue(self): """Update workflow DB.""" self.workflow_db_mgr.process_queued_ops() @@ -1431,7 +1517,7 @@ async def workflow_shutdown(self): LOG.info(stop_process_pool_empty_msg) stop_process_pool_empty_msg = None self.proc_pool.process() - self.process_command_queue() + await self.process_command_queue() if self.options.profile_mode: self.profiler.log_memory( "scheduler.py: end main loop (total loops %d): %s" % @@ -1574,7 +1660,7 @@ async def main_loop(self) -> None: self.is_reloaded = True self.is_updated = True - self.process_command_queue() + await self.process_command_queue() self.proc_pool.process() # Tasks in the main pool that are waiting but not queued must be @@ -1632,7 +1718,7 @@ async def main_loop(self) -> None: self.late_tasks_check() self.process_queued_task_messages() - self.process_command_queue() + await self.process_command_queue() self.task_events_mgr.process_events(self) # Update state summary, database, and uifeed @@ -1690,6 +1776,27 @@ async def main_loop(self) -> None: self.main_loop_intervals.append(time() - tinit) # END MAIN LOOP + def _update_workflow_state(self): + """Update workflow state in the data store and push out any deltas. + + A cut-down version of update_data_structure which only considers + workflow state changes e.g. status, status message, state totals, etc. + """ + # update the workflow state in the data store + self.data_store_mgr.update_workflow() + + # push out update deltas + self.data_store_mgr.batch_deltas() + self.data_store_mgr.apply_delta_batch() + self.data_store_mgr.apply_delta_checksum() + self.data_store_mgr.publish_deltas = self.data_store_mgr.get_publish_deltas() + self.server.publish_queue.put( + self.data_store_mgr.publish_deltas) + + # Non-async sleep - yield to other threads rather + # than event loop + sleep(0) + async def update_data_structure(self) -> Union[bool, List['TaskProxy']]: """Update DB, UIS, Summary data elements""" updated_tasks = [ @@ -1818,7 +1925,7 @@ async def _shutdown(self, reason: BaseException) -> None: LOG.exception(ex) # disconnect from workflow-db, stop db queue try: - self.workflow_db_mgr.process_queued_ops() + self.process_workflow_db_queue() self.workflow_db_mgr.on_workflow_shutdown() except Exception as exc: LOG.exception(exc) @@ -1931,12 +2038,22 @@ def check_auto_shutdown(self): return True - def pause_workflow(self) -> None: - """Pause the workflow.""" + def pause_workflow(self, msg: Optional[str] = None) -> None: + """Pause the workflow. + + Args: + msg: + A user-facing string explaining why the workflow was paused if + helpful. + + """ if self.is_paused: LOG.info("Workflow is already paused") return - LOG.info("PAUSING the workflow now") + _msg = "PAUSING the workflow now" + if msg: + _msg += f': {msg}' + LOG.info(_msg) self.is_paused = True self.workflow_db_mgr.put_workflow_paused() self.update_data_store() @@ -1945,11 +2062,17 @@ def resume_workflow(self, quiet: bool = False) -> None: """Resume the workflow. Args: - quiet: whether to log anything. + quiet: + Whether to log anything in the event the workflow is not + paused. + """ + if self.reload_pending: + LOG.warning('Cannot resume - workflow is reloading') + return if not self.is_paused: if not quiet: - LOG.warning("Cannot resume - workflow is not paused") + LOG.warning("No need to resume - workflow is not paused") return if not quiet: LOG.info("RESUMING the workflow now") diff --git a/cylc/flow/workflow_status.py b/cylc/flow/workflow_status.py index b0cdf4a976c..95f25e39358 100644 --- a/cylc/flow/workflow_status.py +++ b/cylc/flow/workflow_status.py @@ -165,6 +165,9 @@ def get_workflow_status(schd: 'Scheduler') -> Tuple[str, str]: if schd.stop_mode is not None: status = WorkflowStatus.STOPPING status_msg = f'stopping: {schd.stop_mode.explain()}' + elif schd.reload_pending: + status = WorkflowStatus.PAUSED + status_msg = f'reloading: {schd.reload_pending}' elif schd.is_stalled: status_msg = 'stalled' elif schd.is_paused: diff --git a/tests/conftest.py b/tests/conftest.py index fb7a7408f31..8f07a4f3441 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -109,6 +109,38 @@ def _log_filter( return _log_filter +@pytest.fixture +def log_scan(): + """Ensure log messages appear in the correct order. + + TRY TO AVOID DOING THIS! + + If you are trying to test a sequence of events you are likely better off + doing this a different way (e.g. mock the functions you are interested in + and test the call arguments/returns later). + + However, there are some occasions where this might be necessary, e.g. + testing a monolithic synchronous function. + + Args: + log: The caplog fixture. + items: Iterable of string messages to compare. All are tested + by "contains" i.e. "item in string". + + """ + def _log_scan(log, items): + records = iter(log.records) + record = next(records) + for item in items: + while item not in record.message: + try: + record = next(records) + except StopIteration: + raise Exception(f'Reached end of log looking for: {item}') + + return _log_scan + + @pytest.fixture(scope='session') def port_range(): return glbl_cfg().get(['scheduler', 'run hosts', 'ports']) diff --git a/tests/functional/logging/01-basic.t b/tests/functional/logging/01-basic.t index d16e6e8d3e9..05a1a6b6bcf 100644 --- a/tests/functional/logging/01-basic.t +++ b/tests/functional/logging/01-basic.t @@ -31,7 +31,11 @@ init_workflow "${TEST_NAME_BASE}" <<'__FLOW_CONFIG__' R1 = reloader1 => stopper => reloader2 [runtime] [[reloader1, reloader2]] - script = cylc reload "${CYLC_WORKFLOW_ID}" + script = """ + cylc reload "${CYLC_WORKFLOW_ID}" + # wait for the command to complete + cylc__job__poll_grep_workflow_log 'Reload completed' + """ [[stopper]] script = cylc stop --now --now "${CYLC_WORKFLOW_ID}" __FLOW_CONFIG__ @@ -55,6 +59,8 @@ cmp_ok conf_1.out << __EOF__ flow-processed.cylc __EOF__ +mv "$WORKFLOW_RUN_DIR/cylc.flow.main_loop.log_db.sql" "$WORKFLOW_RUN_DIR/01.cylc.flow.main_loop.log_db.sql" + workflow_run_ok "${TEST_NAME_BASE}-run" cylc play --no-detach "${WORKFLOW_NAME}" ls "${WORKFLOW_RUN_DIR}/log/scheduler/" > schd_2.out diff --git a/tests/functional/reload/09-garbage.t b/tests/functional/reload/09-garbage.t index 5c9e8c6b889..0853a4e4c1e 100644 --- a/tests/functional/reload/09-garbage.t +++ b/tests/functional/reload/09-garbage.t @@ -29,6 +29,8 @@ TEST_NAME="${TEST_NAME_BASE}-run" workflow_run_ok "${TEST_NAME}" \ cylc play --reference-test --debug --no-detach "${WORKFLOW_NAME}" #------------------------------------------------------------------------------- -grep_ok 'Command failed: reload_workflow' "${WORKFLOW_RUN_DIR}/log/scheduler/log" +grep_ok \ + 'Reload failed - IllegalItemError: \[scheduling\]garbage' \ + "${WORKFLOW_RUN_DIR}/log/scheduler/log" #------------------------------------------------------------------------------- purge diff --git a/tests/integration/test_reload.py b/tests/integration/test_reload.py new file mode 100644 index 00000000000..d9de0826dfb --- /dev/null +++ b/tests/integration/test_reload.py @@ -0,0 +1,148 @@ +# THIS FILE IS PART OF THE CYLC WORKFLOW ENGINE. +# Copyright (C) NIWA & British Crown (Met Office) & Contributors. +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. +# +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU General Public License for more details. +# +# You should have received a copy of the GNU General Public License +# along with this program. If not, see . + +"""Tests for reload behaviour in the scheduler.""" + +from contextlib import suppress + +from cylc.flow.task_state import ( + TASK_STATUS_WAITING, + TASK_STATUS_PREPARING, + TASK_STATUS_SUBMITTED, +) + + +async def test_reload_waits_for_pending_tasks( + flow, + scheduler, + start, + monkeypatch, + capture_submission, + log_scan, +): + """Reload should flush out preparing tasks and pause the workflow. + + Reloading a workflow with preparing tasks may be unsafe and is at least + confusing. For safety we should pause the workflow and flush out any + preparing tasks before attempting reload. + + See https://github.com/cylc/cylc-flow/issues/5107 + """ + # a simple workflow with a single task + id_ = flow({ + 'scheduling': { + 'graph': { + 'R1': 'foo', + }, + }, + 'runtime': { + 'foo': {}, + }, + }) + schd = scheduler(id_, paused_start=False) + + # we will artificially push the task through these states + state_seq = [ + # repeat the preparing state a few times to simulate a task + # taking multiple main-loop cycles to submit + TASK_STATUS_PREPARING, + TASK_STATUS_PREPARING, + TASK_STATUS_PREPARING, + TASK_STATUS_SUBMITTED, + ] + + # start the scheduler + async with start(schd) as log: + # disable submission events to prevent anything from actually running + capture_submission(schd) + + # set the task to go through some state changes + def change_state(_=0): + with suppress(IndexError): + foo.state_reset(state_seq.pop(0)) + monkeypatch.setattr( + 'cylc.flow.scheduler.sleep', + change_state + ) + + # the task should start as waiting + tasks = schd.pool.get_tasks() + assert len(tasks) == 1 + foo = tasks[0] + assert tasks[0].state(TASK_STATUS_WAITING) + + # put the task into the preparing state + change_state() + + # reload the workflow + await schd.command_reload_workflow() + + # the task should end in the submitted state + assert foo.state(TASK_STATUS_SUBMITTED) + + # ensure the order of events was correct + log_scan( + log, + [ + # the task should have entered the preparing state before the + # reload was requested + '[1/foo waiting(queued) job:00 flows:1] => preparing(queued)', + # the reload should have put the workflow into the paused state + 'PAUSING the workflow now: Reloading workflow', + # reload should have waited for the task to submit + '[1/foo preparing(queued) job:00 flows:1]' + ' => submitted(queued)', + # before then reloading the workflow config + 'Reloading the workflow definition.', + # post-reload the workflow should have been resumed + 'RESUMING the workflow now', + ], + ) + + +async def test_reload_failure( + flow, + one_conf, + scheduler, + start, + log_filter, +): + """Reload should not crash the workflow on config errors. + + A warning should be logged along with the error. + """ + id_ = flow(one_conf) + schd = scheduler(id_) + async with start(schd) as log: + # corrupt the config by removing the scheduling section + two_conf = {**one_conf, 'scheduling': {}} + flow(two_conf, id_=id_) + + # reload the workflow + await schd.command_reload_workflow() + + # the reload should have failed but the workflow should still be + # running + assert log_filter( + log, + contains=( + 'Reload failed - WorkflowConfigError:' + ' missing [scheduling][[graph]] section' + ) + ) + + # the config should be unchanged + assert schd.config.cfg['scheduling']['graph']['R1'] == 'one' diff --git a/tests/integration/test_resolvers.py b/tests/integration/test_resolvers.py index 2f452cda760..b6e32708505 100644 --- a/tests/integration/test_resolvers.py +++ b/tests/integration/test_resolvers.py @@ -232,7 +232,7 @@ async def test_stop( schd=one ) resolvers.stop(StopMode.REQUEST_CLEAN) - one.process_command_queue() + await one.process_command_queue() assert log_filter( log, level=logging.INFO, contains="Command actioned: stop" ) diff --git a/tests/integration/test_scheduler.py b/tests/integration/test_scheduler.py index c183036d004..5330e516758 100644 --- a/tests/integration/test_scheduler.py +++ b/tests/integration/test_scheduler.py @@ -241,80 +241,6 @@ async def test_no_poll_waiting_tasks( assert "Orphaned tasks:\n* 1/one (running)" in log.messages -@pytest.mark.parametrize('reload', [False, True]) -@pytest.mark.parametrize( - 'test_conf, expected_msg', - [ - pytest.param( - {'Alan Wake': "It's not a lake, it's an ocean"}, - "IllegalItemError: Alan Wake", - id="illegal item" - ), - pytest.param( - { - 'scheduling': { - 'initial cycle point': "2k22", - 'graph': {'R1': "a => b"} - } - }, - ("IllegalValueError: (type=cycle point) " - "[scheduling]initial cycle point = 2k22 - (Invalid cycle point)"), - id="illegal cycle point" - ) - ] -) -async def test_illegal_config_load( - test_conf: dict, - expected_msg: str, - reload: bool, - flow: Callable, - one_conf: dict, - start: Callable, - run: Callable, - scheduler: Callable, - log_filter: Callable -): - """Test that ParsecErrors (illegal config) - that occur during config load - when running a workflow - are displayed without traceback. - - Params: - test_conf: Dict to update one_conf with. - expected_msg: Expected log message at error level. - reload: If False, test a workflow start with invalid config. - If True, test a workflow start with valid config followed by - reload with invalid config. - """ - if not reload: - one_conf.update(test_conf) - id_: str = flow(one_conf) - schd: Scheduler = scheduler(id_) - log: pytest.LogCaptureFixture - - if reload: - one_conf.update(test_conf) - async with run(schd) as log: - # Shouldn't be any errors at this stage: - assert not log_filter(log, level=logging.ERROR) - # Modify flow.cylc: - flow(one_conf, id_=id_) - schd.queue_command('reload_workflow', {}) - assert log_filter( - log, level=logging.ERROR, - exact_match=f"Command failed: reload_workflow()\n{expected_msg}" - ) - else: - with pytest.raises(ParsecError): - async with start(schd) as log: - pass - assert log_filter( - log, - level=logging.ERROR, - exact_match=f"Workflow shutting down - {expected_msg}" - ) - - assert TRACEBACK_MSG not in log.text - - async def test_unexpected_ParsecError( one: Scheduler, start: Callable, diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index 530e0063b70..7ca64d6bfc6 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -558,7 +558,7 @@ async def test_reload_stopcp( schd: Scheduler = scheduler(flow(cfg)) async with start(schd): assert str(schd.pool.stop_point) == '2020' - schd.command_reload_workflow() + await schd.command_reload_workflow() assert str(schd.pool.stop_point) == '2020' @@ -827,7 +827,7 @@ async def test_reload_prereqs( flow(conf, id_=id_) # Reload the workflow config - schd.command_reload_workflow() + await schd.command_reload_workflow() schd.pool.reload_taskdefs() assert list_tasks(schd) == expected_3 @@ -958,7 +958,7 @@ async def test_graph_change_prereq_satisfaction( flow(conf, id_=id_) # Reload the workflow config - schd.command_reload_workflow() + await schd.command_reload_workflow() schd.pool.reload_taskdefs() await test.asend(schd) diff --git a/tests/unit/test_workflow_status.py b/tests/unit/test_workflow_status.py index 046783d4ff1..af88de3daab 100644 --- a/tests/unit/test_workflow_status.py +++ b/tests/unit/test_workflow_status.py @@ -36,12 +36,14 @@ def schd( stop_mode=None, stop_point=None, stop_task_id=None, + reload_pending=False, ): return SimpleNamespace( is_paused=is_paused, is_stalled=is_stalled, stop_clock_time=stop_clock_time, stop_mode=stop_mode, + reload_pending=reload_pending, pool=SimpleNamespace( hold_point=hold_point, stop_point=stop_point, @@ -58,7 +60,13 @@ def schd( ( {'is_paused': True}, WorkflowStatus.PAUSED, - 'paused'), + 'paused' + ), + ( + {'reload_pending': 'message'}, + WorkflowStatus.PAUSED, + 'reloading: message' + ), ( {'stop_mode': StopMode.AUTO}, WorkflowStatus.STOPPING, From 820d337ea431fe9b913c563041aa52ffe9187dcf Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Mon, 19 Jun 2023 17:28:28 +0100 Subject: [PATCH 02/24] reload: simplify execution pathway * The reload code was spread around four places: * Scheduler.command_reload_workflow * Scheduler.main_loop * Pool.set_do_reload * Pool.reload_taskdefs * This commit co-locates the Scheduler/Pool parts and turns them into a single synchronous operation (no main-loop moving parts) to simplify the pathway. * This removes the need for the `do_reload` pool flag. --- cylc/flow/scheduler.py | 39 ++++++++++++----------- cylc/flow/task_pool.py | 49 +++++++++++++---------------- tests/integration/test_task_pool.py | 2 -- 3 files changed, 42 insertions(+), 48 deletions(-) diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 16f0a55c1b0..6008beaefc6 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -1095,7 +1095,7 @@ async def command_reload_workflow(self) -> None: self._update_workflow_state() LOG.info("Reloading the workflow definition.") try: - cfg = self.load_flow_file(is_reload=True) + config = self.load_flow_file(is_reload=True) except (ParsecError, CylcConfigError) as exc: if cylc.flow.flags.verbosity > 1: # log full traceback in debug mode @@ -1116,10 +1116,10 @@ async def command_reload_workflow(self) -> None: self.workflow_db_mgr.pri_dao.select_workflow_params( self._load_workflow_params ) - self.apply_new_config(cfg, is_reload=True) + self.apply_new_config(config, is_reload=True) self.broadcast_mgr.linearized_ancestors = ( self.config.get_linearized_ancestors()) - self.pool.set_do_reload(self.config) + self.task_events_mgr.mail_interval = self.cylc_config['mail'][ 'task event batch interval'] self.task_events_mgr.mail_smtp = self._get_events_conf("smtp") @@ -1138,6 +1138,24 @@ async def command_reload_workflow(self) -> None: self.is_reloaded = True self._update_workflow_state() + # Re-initialise data model on reload + self.data_store_mgr.initiate_data_model(reloaded=True) + + # Reset the remote init map to trigger fresh file installation + self.task_job_mgr.task_remote_mgr.remote_init_map.clear() + self.task_job_mgr.task_remote_mgr.is_reload = True + self.pool.reload_taskdefs(config) + # Load jobs from DB + self.workflow_db_mgr.pri_dao.select_jobs_for_restart( + self.data_store_mgr.insert_db_job + ) + if self.pool.compute_runahead(force=True): + self.pool.release_runahead_tasks() + self.is_reloaded = True + self.is_updated = True + + LOG.info("Reload completed.") + # resume the workflow if previously paused self.reload_pending = False self.update_data_store() # update workflow status msg @@ -1644,21 +1662,6 @@ async def main_loop(self) -> None: # self.pool.log_task_pool(logging.CRITICAL) if self.incomplete_ri_map: self.manage_remote_init() - if self.pool.do_reload: - # Re-initialise data model on reload - self.data_store_mgr.initiate_data_model(reloaded=True) - # Reset the remote init map to trigger fresh file installation - self.task_job_mgr.task_remote_mgr.remote_init_map.clear() - self.task_job_mgr.task_remote_mgr.is_reload = True - self.pool.reload_taskdefs() - # Load jobs from DB - self.workflow_db_mgr.pri_dao.select_jobs_for_restart( - self.data_store_mgr.insert_db_job) - LOG.info("Reload completed.") - if self.pool.compute_runahead(force=True): - self.pool.release_runahead_tasks() - self.is_reloaded = True - self.is_updated = True await self.process_command_queue() self.proc_pool.process() diff --git a/cylc/flow/task_pool.py b/cylc/flow/task_pool.py index e50df1591a0..a1835c9bb2d 100644 --- a/cylc/flow/task_pool.py +++ b/cylc/flow/task_pool.py @@ -111,7 +111,6 @@ def __init__( self.data_store_mgr: 'DataStoreMgr' = data_store_mgr self.flow_mgr: 'FlowMgr' = flow_mgr - self.do_reload = False self.max_future_offset: Optional['IntervalBase'] = None self._prev_runahead_base_point: Optional['PointBase'] = None self._prev_runahead_sequence_points: Optional[Set['PointBase']] = None @@ -132,7 +131,6 @@ def __init__( self.abort_task_failed = False self.expected_failed_tasks = self.config.get_expected_failed_tasks() - self.orphans: List[str] = [] self.task_name_list = self.config.get_task_name_list() self.task_queue_mgr = IndepQueueManager( self.config.cfg['scheduling']['queues'], @@ -913,25 +911,7 @@ def set_max_future_offset(self): if max_offset != orig and self.compute_runahead(force=True): self.release_runahead_tasks() - def set_do_reload(self, config: 'WorkflowConfig') -> None: - """Set the task pool to reload mode.""" - self.config = config - self.stop_point = config.stop_point or config.final_point - self.do_reload = True - - # find any old tasks that have been removed from the workflow - old_task_name_list = self.task_name_list - self.task_name_list = self.config.get_task_name_list() - for name in old_task_name_list: - if name not in self.task_name_list: - self.orphans.append(name) - for name in self.task_name_list: - if name in self.orphans: - self.orphans.remove(name) - # adjust the new workflow config to handle the orphans - self.config.adopt_orphans(self.orphans) - - def reload_taskdefs(self) -> None: + def reload_taskdefs(self, config: 'WorkflowConfig') -> None: """Reload the definitions of task proxies in the pool. Orphaned tasks (whose definitions were removed from the workflow): @@ -941,18 +921,33 @@ def reload_taskdefs(self) -> None: Otherwise: replace task definitions but copy over existing outputs etc. """ + self.config = config + self.stop_point = config.stop_point or config.final_point + + # find any old tasks that have been removed from the workflow + old_task_name_list = self.task_name_list + self.task_name_list = self.config.get_task_name_list() + orphans = [ + task + for task in old_task_name_list + if task not in self.task_name_list + ] + + # adjust the new workflow config to handle the orphans + self.config.adopt_orphans(orphans) + LOG.info("Reloading task definitions.") tasks = self.get_all_tasks() # Log tasks orphaned by a reload but not currently in the task pool. - for name in self.orphans: + for name in orphans: if name not in (itask.tdef.name for itask in tasks): LOG.warning("Removed task: '%s'", name) for itask in tasks: - if itask.tdef.name in self.orphans: + if itask.tdef.name in orphans: if ( - itask.state(TASK_STATUS_WAITING) - or itask.state.is_held - or itask.state.is_queued + itask.state(TASK_STATUS_WAITING) + or itask.state.is_held + or itask.state.is_queued ): # Remove orphaned task if it hasn't started running yet. self.remove(itask, 'task definition removed') @@ -1007,8 +1002,6 @@ def reload_taskdefs(self) -> None: if all(ready_check_items) and not itask.state.is_runahead: self.queue_task(itask) - self.do_reload = False - def set_stop_point(self, stop_point: 'PointBase') -> bool: """Set the workflow stop cycle point. diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index 7ca64d6bfc6..a1e38c26bd5 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -828,7 +828,6 @@ async def test_reload_prereqs( # Reload the workflow config await schd.command_reload_workflow() - schd.pool.reload_taskdefs() assert list_tasks(schd) == expected_3 # Check resulting dependencies of task z @@ -959,6 +958,5 @@ async def test_graph_change_prereq_satisfaction( # Reload the workflow config await schd.command_reload_workflow() - schd.pool.reload_taskdefs() await test.asend(schd) From 9eb0d4de04551028dd99c58c19336dd421cdca32 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Wed, 28 Jun 2023 14:44:10 +0100 Subject: [PATCH 03/24] Update cylc/flow/scheduler.py --- cylc/flow/scheduler.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 6008beaefc6..8c11f123ded 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -1792,7 +1792,9 @@ def _update_workflow_state(self): self.data_store_mgr.batch_deltas() self.data_store_mgr.apply_delta_batch() self.data_store_mgr.apply_delta_checksum() - self.data_store_mgr.publish_deltas = self.data_store_mgr.get_publish_deltas() + self.data_store_mgr.publish_deltas = ( + self.data_store_mgr.get_publish_deltas() + ) self.server.publish_queue.put( self.data_store_mgr.publish_deltas) From 9262b1b9f149fd8be584a1a78f82d2ff1efe4987 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Wed, 5 Jul 2023 15:51:06 +0100 Subject: [PATCH 04/24] Update CHANGES.md Co-authored-by: Tim Pillinger <26465611+wxtim@users.noreply.github.com> --- CHANGES.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index fc99d59ffa0..2455843c8b4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -22,8 +22,9 @@ issue which could cause jobs to fail if this variable became too long. ### Enhancements [#5992](https://github.com/cylc/cylc-flow/pull/5992) - -The scheduler will now wait for preparing tasks to submit before attempting -to perform a reload and will also pause/unpause the workflow. +Before trying to reload the workflow definition, the scheduler will +now wait for preparing tasks to submit, and pause the workflow. +After successful reload the scheduler will unpause the workflow. -[#5605](https://github.com/cylc/cylc-flow/pull/5605) - A shorthand for defining -a list of strings - Before: `cylc command -s "X=['a', 'bc', 'd']"` - After: From c25afe117a049ef15301a1924bfbc73a65700037 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Wed, 5 Jul 2023 16:11:49 +0100 Subject: [PATCH 05/24] pool: ensure the DB is updated when tasks are removed (#5619) * Before this change, the `task_pool` table was only being updated when tasks were added, or changed state, but not when they were removed. * Closes #5598 --- CHANGES.md | 3 ++ cylc/flow/scheduler.py | 7 +++- cylc/flow/task_pool.py | 2 + tests/integration/test_task_pool.py | 64 +++++++++++++++++++++++++++++ 4 files changed, 75 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 6fad5e29ee5..2374ff5fcdd 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,9 @@ Various enhancements to `cylc lint`: ### Fixes +[#5619](https://github.com/cylc/cylc-flow/pull/5619) - +Fix an issue where the `task_pool` table in the database wasn't being updated +in a timely fashion when tasks completed. [#5606](https://github.com/cylc/cylc-flow/pull/5606) - Task outputs and messages are now validated to avoid conflicts with built-in diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index e192185c388..7c782c58bae 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -1691,7 +1691,11 @@ async def update_data_structure(self) -> Union[bool, List['TaskProxy']]: """Update DB, UIS, Summary data elements""" updated_tasks = [ t for t in self.pool.get_tasks() if t.state.is_updated] - has_updated = self.is_updated or updated_tasks + has_updated = ( + self.is_updated + or updated_tasks + or self.pool.tasks_removed + ) reloaded = self.is_reloaded # Add tasks that have moved moved from runahead to live pool. if has_updated or self.data_store_mgr.updates_pending: @@ -1716,6 +1720,7 @@ async def update_data_structure(self) -> Union[bool, List['TaskProxy']]: for itask in updated_tasks: itask.state.is_updated = False self.update_data_store() + self.pool.tasks_removed = False return has_updated def check_workflow_timers(self): diff --git a/cylc/flow/task_pool.py b/cylc/flow/task_pool.py index ae193c044a4..d03db8a9bef 100644 --- a/cylc/flow/task_pool.py +++ b/cylc/flow/task_pool.py @@ -123,6 +123,7 @@ def __init__( self._hidden_pool_list: List[TaskProxy] = [] self.main_pool_changed = False self.hidden_pool_changed = False + self.tasks_removed = False self.hold_point: Optional['PointBase'] = None self.abs_outputs_done: Set[Tuple[str, str, str]] = set() @@ -736,6 +737,7 @@ def spawn_to_rh_limit(self, tdef, point, flow_nums) -> None: def remove(self, itask, reason=""): """Remove a task from the pool (e.g. after a reload).""" + self.tasks_removed = True msg = "task proxy removed" if reason: msg += f" ({reason})" diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index 4cf4ab78208..f25d2410ef8 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -989,3 +989,67 @@ async def test_runahead_limit_for_sequence_before_start_cycle( schd = scheduler(id_, startcp='2005') async with start(schd): assert str(schd.pool.runahead_limit_point) == '20070101T0000Z' + + +def list_pool_from_db(schd): + """Returns the task pool table as a sorted list.""" + db_task_pool = [] + schd.workflow_db_mgr.pri_dao.select_task_pool( + lambda _, row: db_task_pool.append(row) + ) + return sorted(db_task_pool) + + +async def test_db_update_on_removal( + flow, + scheduler, + start, +): + """It should updated the task_pool table when tasks complete. + + There was a bug where the task_pool table was only being updated when tasks + in the pool were updated. This meant that if a task was removed the DB + would not reflect this change and would hold a record of the task in the + wrong state. + + This test ensures that the DB is updated when a task is removed from the + pool. + + See: https://github.com/cylc/cylc-flow/issues/5598 + """ + id_ = flow({ + 'scheduler': { + 'allow implicit tasks': 'true', + }, + 'scheduling': { + 'graph': { + 'R1': 'a', + }, + }, + }) + schd = scheduler(id_) + async with start(schd): + task_a = schd.pool.get_tasks()[0] + + # set the task to running + task_a.state_reset('running') + + # update the db + await schd.update_data_structure() + schd.workflow_db_mgr.process_queued_ops() + + # the task should appear in the DB + assert list_pool_from_db(schd) == [ + ['1', 'a', 'running', 0], + ] + + # mark the task as succeeded and allow it to be removed from the pool + task_a.state_reset('succeeded') + schd.pool.remove_if_complete(task_a) + + # update the DB, note no new tasks have been added to the pool + await schd.update_data_structure() + schd.workflow_db_mgr.process_queued_ops() + + # the task should be gone from the DB + assert list_pool_from_db(schd) == [] From b6906ca44d248d4aa4cc9d6de475aebfb81668dd Mon Sep 17 00:00:00 2001 From: Ronnie Dutta <61982285+MetRonnie@users.noreply.github.com> Date: Tue, 4 Jul 2023 13:49:41 +0100 Subject: [PATCH 06/24] DB mgr: ditch callback in `select_workfow_params` --- cylc/flow/network/scan.py | 10 +--- cylc/flow/rundb.py | 14 ++--- cylc/flow/scheduler.py | 118 +++++++++++++++++++------------------- 3 files changed, 66 insertions(+), 76 deletions(-) diff --git a/cylc/flow/network/scan.py b/cylc/flow/network/scan.py index 4dfe5510848..54222a510c7 100644 --- a/cylc/flow/network/scan.py +++ b/cylc/flow/network/scan.py @@ -536,13 +536,6 @@ async def workflow_params(flow): Requires: * is_active(True) """ - params = {} - - def _callback(_, entry): - nonlocal params - key, value = entry - params[key] = value - # NOTE: use the public DB for reading # (only the scheduler process/thread should access the private database) db_file = Path(get_workflow_run_dir( @@ -550,7 +543,6 @@ def _callback(_, entry): )) if db_file.exists(): with CylcWorkflowDAO(db_file, is_public=True) as dao: - dao.select_workflow_params(_callback) - flow['workflow_params'] = params + flow['workflow_params'] = dict(dao.select_workflow_params()) return flow diff --git a/cylc/flow/rundb.py b/cylc/flow/rundb.py index d5ce41bdf83..0bbb934ab53 100644 --- a/cylc/flow/rundb.py +++ b/cylc/flow/rundb.py @@ -21,7 +21,7 @@ from pprint import pformat import sqlite3 import traceback -from typing import TYPE_CHECKING, List, Optional, Tuple, Union +from typing import TYPE_CHECKING, Iterable, List, Optional, Tuple, Union from cylc.flow import LOG from cylc.flow.exceptions import PlatformLookupError @@ -545,13 +545,10 @@ def select_broadcast_states(self, callback, sort=None): for row_idx, row in enumerate(self.connect().execute(stmt)): callback(row_idx, list(row)) - def select_workflow_params(self, callback): - """Select from workflow_params. + def select_workflow_params(self) -> Iterable[Tuple[str, str]]: + """Select all from workflow_params. - Invoke callback(row_idx, row) on each row, where each row contains: - [key, value] - - E.g. a row might be ['UTC mode', '1'] + E.g. a row might be ('UTC mode', '1') """ stmt = rf''' SELECT @@ -559,8 +556,7 @@ def select_workflow_params(self, callback): FROM {self.TABLE_WORKFLOW_PARAMS} ''' # nosec (table name is code constant) - for row_idx, row in enumerate(self.connect().execute(stmt)): - callback(row_idx, list(row)) + return self.connect().execute(stmt) def select_workflow_flows(self, flow_nums): """Return flow data for selected flows.""" diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 7c782c58bae..23ab5a9bb53 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -285,7 +285,7 @@ def __init__(self, reg: str, options: Values) -> None: self._profile_update_times = {} self.bad_hosts: Set[str] = set() - self.restored_stop_task_id = None + self.restored_stop_task_id: Optional[str] = None self.timers: Dict[str, Timer] = {} @@ -536,7 +536,7 @@ def load_workflow_params_and_tmpl_vars(self) -> None: with self.workflow_db_mgr.get_pri_dao() as pri_dao: # This logic handles lack of initial cycle point in flow.cylc and # things that can't change on workflow restart/reload. - pri_dao.select_workflow_params(self._load_workflow_params) + self._load_workflow_params(pri_dao.select_workflow_params()) pri_dao.select_workflow_template_vars(self._load_template_vars) pri_dao.execute_queued_items() @@ -1048,8 +1048,8 @@ def command_reload_workflow(self) -> None: LOG.info("Reloading the workflow definition.") old_tasks = set(self.config.get_task_name_list()) # Things that can't change on workflow reload: - self.workflow_db_mgr.pri_dao.select_workflow_params( - self._load_workflow_params + self._load_workflow_params( + self.workflow_db_mgr.pri_dao.select_workflow_params() ) try: @@ -1203,7 +1203,7 @@ def load_flow_file(self, is_reload=False): 'CYLC_WORKFLOW_FINAL_CYCLE_POINT': str(self.config.final_point), }) - def _load_workflow_params(self, row_idx, row): + def _load_workflow_params(self, params: Iterable[Tuple[str, str]]) -> None: """Load a row in the "workflow_params" table in a restart/reload. This currently includes: @@ -1214,62 +1214,64 @@ def _load_workflow_params(self, row_idx, row): * A flag to indicate if the workflow should be paused or not. * Original workflow run time zone. """ - if row_idx == 0: - LOG.info('LOADING workflow parameters') - key, value = row - if key in self.workflow_db_mgr.KEY_INITIAL_CYCLE_POINT_COMPATS: - self.options.icp = value - LOG.info(f"+ initial point = {value}") - elif key in self.workflow_db_mgr.KEY_START_CYCLE_POINT_COMPATS: - self.options.startcp = value - LOG.info(f"+ start point = {value}") - elif key in self.workflow_db_mgr.KEY_FINAL_CYCLE_POINT_COMPATS: - if self.is_restart and self.options.fcp == 'reload': - LOG.debug(f"- final point = {value} (ignored)") - elif self.options.fcp is None: - self.options.fcp = value - LOG.info(f"+ final point = {value}") - elif key == self.workflow_db_mgr.KEY_STOP_CYCLE_POINT: - if self.is_restart and self.options.stopcp == 'reload': - LOG.debug(f"- stop point = {value} (ignored)") - elif self.options.stopcp is None: - self.options.stopcp = value - LOG.info(f"+ stop point = {value}") - elif key == self.workflow_db_mgr.KEY_RUN_MODE: - if self.options.run_mode is None: + LOG.info('LOADING workflow parameters') + for key, value in params: + if key in self.workflow_db_mgr.KEY_INITIAL_CYCLE_POINT_COMPATS: + self.options.icp = value + LOG.info(f"+ initial point = {value}") + elif key in self.workflow_db_mgr.KEY_START_CYCLE_POINT_COMPATS: + self.options.startcp = value + LOG.info(f"+ start point = {value}") + elif key in self.workflow_db_mgr.KEY_FINAL_CYCLE_POINT_COMPATS: + if self.is_restart and self.options.fcp == 'reload': + LOG.debug(f"- final point = {value} (ignored)") + elif self.options.fcp is None: + self.options.fcp = value + LOG.info(f"+ final point = {value}") + elif key == self.workflow_db_mgr.KEY_STOP_CYCLE_POINT: + if self.is_restart and self.options.stopcp == 'reload': + LOG.debug(f"- stop point = {value} (ignored)") + elif self.options.stopcp is None: + self.options.stopcp = value + LOG.info(f"+ stop point = {value}") + elif ( + key == self.workflow_db_mgr.KEY_RUN_MODE + and self.options.run_mode is None + ): self.options.run_mode = value LOG.info(f"+ run mode = {value}") - elif key == self.workflow_db_mgr.KEY_UUID_STR: - self.uuid_str = value - LOG.info('+ workflow UUID = %s', value) - elif key == self.workflow_db_mgr.KEY_PAUSED: - if self.options.paused_start is None: - self.options.paused_start = bool(value) - LOG.info(f'+ paused = {bool(value)}') - elif key == self.workflow_db_mgr.KEY_HOLD_CYCLE_POINT: - if self.options.holdcp is None: + elif key == self.workflow_db_mgr.KEY_UUID_STR: + self.uuid_str = value + LOG.info(f"+ workflow UUID = {value}") + elif key == self.workflow_db_mgr.KEY_PAUSED: + bool_val = bool(int(value)) + if bool_val and not self.options.paused_start: + self.options.paused_start = bool_val + LOG.info(f"+ paused = {bool_val}") + elif ( + key == self.workflow_db_mgr.KEY_HOLD_CYCLE_POINT + and self.options.holdcp is None + ): self.options.holdcp = value - LOG.info('+ hold point = %s', value) - elif key == self.workflow_db_mgr.KEY_STOP_CLOCK_TIME: - value = int(value) - if time() <= value: - self.stop_clock_time = value - LOG.info('+ stop clock time = %d (%s)', value, time2str(value)) - else: - LOG.debug( - '- stop clock time = %d (%s) (ignored)', - value, - time2str(value)) - elif key == self.workflow_db_mgr.KEY_STOP_TASK: - self.restored_stop_task_id = value - LOG.info('+ stop task = %s', value) - elif key == self.workflow_db_mgr.KEY_UTC_MODE: - value = bool(int(value)) - self.options.utc_mode = value - LOG.info(f"+ UTC mode = {value}") - elif key == self.workflow_db_mgr.KEY_CYCLE_POINT_TIME_ZONE: - self.options.cycle_point_tz = value - LOG.info(f"+ cycle point time zone = {value}") + LOG.info(f"+ hold point = {value}") + elif key == self.workflow_db_mgr.KEY_STOP_CLOCK_TIME: + int_val = int(value) + msg = f"stop clock time = {int_val} ({time2str(int_val)})" + if time() <= int_val: + self.stop_clock_time = int_val + LOG.info(f"+ {msg}") + else: + LOG.debug(f"- {msg} (ignored)") + elif key == self.workflow_db_mgr.KEY_STOP_TASK: + self.restored_stop_task_id = value + LOG.info(f"+ stop task = {value}") + elif key == self.workflow_db_mgr.KEY_UTC_MODE: + bool_val = bool(int(value)) + self.options.utc_mode = bool_val + LOG.info(f"+ UTC mode = {bool_val}") + elif key == self.workflow_db_mgr.KEY_CYCLE_POINT_TIME_ZONE: + self.options.cycle_point_tz = value + LOG.info(f"+ cycle point time zone = {value}") def _load_template_vars(self, _, row): """Load workflow start up template variables.""" From 383bf8d82190bdadb8829ca4b45b3a69e55bd21c Mon Sep 17 00:00:00 2001 From: Ronnie Dutta <61982285+MetRonnie@users.noreply.github.com> Date: Wed, 5 Jul 2023 13:47:39 +0100 Subject: [PATCH 07/24] DB `workflow_params`: store None instead of deleting/not storing params Prevents problems caused by rapidly toggled params before processing DB queue --- cylc/flow/scheduler.py | 10 +-- cylc/flow/scheduler_cli.py | 1 + cylc/flow/task_pool.py | 7 +- cylc/flow/workflow_db_mgr.py | 71 +++++-------------- tests/flakyfunctional/database/00-simple.t | 12 +++- .../restart/46-stop-clock-time.t | 2 +- .../restart/08-stop-after-cycle-point.t | 10 +-- tests/functional/restart/45-stop-task.t | 8 +-- .../restart/51-final-point-reload.t | 3 +- tests/integration/test_task_pool.py | 4 +- tests/integration/test_workflow_db_mgr.py | 22 +++++- 11 files changed, 73 insertions(+), 77 deletions(-) diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 23ab5a9bb53..362e95064b9 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -1216,6 +1216,8 @@ def _load_workflow_params(self, params: Iterable[Tuple[str, str]]) -> None: """ LOG.info('LOADING workflow parameters') for key, value in params: + if not value: + continue if key in self.workflow_db_mgr.KEY_INITIAL_CYCLE_POINT_COMPATS: self.options.icp = value LOG.info(f"+ initial point = {value}") @@ -1901,7 +1903,7 @@ def stop_clock_done(self): LOG.info("Wall clock stop time reached: %s", time2str( self.stop_clock_time)) self.stop_clock_time = None - self.workflow_db_mgr.delete_workflow_stop_clock_time() + self.workflow_db_mgr.put_workflow_stop_clock_time(None) self.update_data_store() return True LOG.debug("stop time=%d; current time=%d", self.stop_clock_time, now) @@ -1935,7 +1937,7 @@ def check_auto_shutdown(self): # Can shut down. if self.pool.stop_point: # Forget early stop point in case of a restart. - self.workflow_db_mgr.delete_workflow_stop_cycle_point() + self.workflow_db_mgr.put_workflow_stop_cycle_point(None) return True @@ -1946,7 +1948,7 @@ def pause_workflow(self) -> None: return LOG.info("PAUSING the workflow now") self.is_paused = True - self.workflow_db_mgr.put_workflow_paused() + self.workflow_db_mgr.put_workflow_paused(True) self.update_data_store() def resume_workflow(self, quiet: bool = False) -> None: @@ -1962,7 +1964,7 @@ def resume_workflow(self, quiet: bool = False) -> None: if not quiet: LOG.info("RESUMING the workflow now") self.is_paused = False - self.workflow_db_mgr.delete_workflow_paused() + self.workflow_db_mgr.put_workflow_paused(False) self.update_data_store() def command_force_trigger_tasks(self, items, flow, flow_wait, flow_descr): diff --git a/cylc/flow/scheduler_cli.py b/cylc/flow/scheduler_cli.py index bd425c8190f..b38b797b878 100644 --- a/cylc/flow/scheduler_cli.py +++ b/cylc/flow/scheduler_cli.py @@ -203,6 +203,7 @@ ["--pause"], help="Pause the workflow immediately on start up.", action='store_true', + default=False, dest="paused_start", sources={'play'}, ), diff --git a/cylc/flow/task_pool.py b/cylc/flow/task_pool.py index d03db8a9bef..f4b298dac30 100644 --- a/cylc/flow/task_pool.py +++ b/cylc/flow/task_pool.py @@ -161,10 +161,9 @@ def stop_task_done(self): LOG.info("Stop task %s finished" % self.stop_task_id) self.stop_task_id = None self.stop_task_finished = False - self.workflow_db_mgr.delete_workflow_stop_task() + self.workflow_db_mgr.put_workflow_stop_task(None) return True - else: - return False + return False def _swap_out(self, itask): """Swap old task for new, during reload.""" @@ -1241,7 +1240,7 @@ def release_hold_point(self) -> None: self.release_held_active_task(itask) self.tasks_to_hold.clear() self.workflow_db_mgr.put_tasks_to_hold(self.tasks_to_hold) - self.workflow_db_mgr.delete_workflow_hold_cycle_point() + self.workflow_db_mgr.put_workflow_hold_cycle_point(None) def check_abort_on_task_fails(self): """Check whether workflow should abort on task failure. diff --git a/cylc/flow/workflow_db_mgr.py b/cylc/flow/workflow_db_mgr.py index ad46214e561..8fd090f5612 100644 --- a/cylc/flow/workflow_db_mgr.py +++ b/cylc/flow/workflow_db_mgr.py @@ -30,7 +30,7 @@ from sqlite3 import OperationalError from tempfile import mkstemp from typing import ( - Any, AnyStr, Dict, List, Set, TYPE_CHECKING, Tuple, Union + Any, AnyStr, Dict, List, Optional, Set, TYPE_CHECKING, Tuple, Union ) from cylc.flow import LOG @@ -169,33 +169,6 @@ def copy_pri_to_pub(self) -> None: os.remove(temp_pub_db_file_name) raise - def delete_workflow_params(self, *keys): - """Schedule deletion of rows from workflow_params table by keys.""" - for key in keys: - self.db_deletes_map[self.TABLE_WORKFLOW_PARAMS].append( - {'key': key} - ) - - def delete_workflow_paused(self): - """Delete paused status.""" - self.delete_workflow_params(self.KEY_PAUSED) - - def delete_workflow_hold_cycle_point(self): - """Delete workflow hold cycle point.""" - self.delete_workflow_params(self.KEY_HOLD_CYCLE_POINT) - - def delete_workflow_stop_clock_time(self): - """Delete workflow stop clock time from workflow_params table.""" - self.delete_workflow_params(self.KEY_STOP_CLOCK_TIME) - - def delete_workflow_stop_cycle_point(self): - """Delete workflow stop cycle point from workflow_params table.""" - self.delete_workflow_params(self.KEY_STOP_CYCLE_POINT) - - def delete_workflow_stop_task(self): - """Delete workflow stop task from workflow_params table.""" - self.delete_workflow_params(self.KEY_STOP_TASK) - def get_pri_dao(self) -> CylcWorkflowDAO: """Return the primary DAO. @@ -345,14 +318,12 @@ def put_workflow_params(self, schd: 'Scheduler') -> None: {"key": self.KEY_CYLC_VERSION, "value": CYLC_VERSION}, {"key": self.KEY_UTC_MODE, "value": get_utc_mode()}, {"key": self.KEY_RESTART_COUNT, "value": self.n_restart}, + {"key": self.KEY_CYCLE_POINT_FORMAT, + "value": schd.config.cycle_point_dump_format}, + {"key": self.KEY_PAUSED, "value": int(schd.is_paused)}, + {"key": self.KEY_STOP_CLOCK_TIME, "value": schd.stop_clock_time}, + {"key": self.KEY_STOP_TASK, "value": schd.stop_task}, ]) - if schd.config.cycle_point_dump_format is not None: - self.put_workflow_params_1( - self.KEY_CYCLE_POINT_FORMAT, - schd.config.cycle_point_dump_format - ) - if schd.is_paused: - self.put_workflow_params_1(self.KEY_PAUSED, 1) for key in ( self.KEY_INITIAL_CYCLE_POINT, self.KEY_FINAL_CYCLE_POINT, @@ -360,38 +331,34 @@ def put_workflow_params(self, schd: 'Scheduler') -> None: self.KEY_STOP_CYCLE_POINT ): value = getattr(schd.options, key, None) - if value is not None and value != 'reload': - self.put_workflow_params_1(key, value) + value = None if value == 'reload' else value + self.put_workflow_params_1(key, value) for key in ( self.KEY_RUN_MODE, self.KEY_CYCLE_POINT_TIME_ZONE ): - value = getattr(schd.options, key, None) - if value is not None: - self.put_workflow_params_1(key, value) - for key in ( - self.KEY_STOP_CLOCK_TIME, - self.KEY_STOP_TASK - ): - value = getattr(schd, key, None) - if value is not None: - self.put_workflow_params_1(key, value) + self.put_workflow_params_1(key, getattr(schd.options, key, None)) def put_workflow_params_1( - self, key: str, value: Union[AnyStr, float] + self, key: str, value: Union[AnyStr, float, None] ) -> None: """Queue insertion of 1 key=value pair to the workflow_params table.""" self.db_inserts_map[self.TABLE_WORKFLOW_PARAMS].append( {"key": key, "value": value} ) - def put_workflow_paused(self): + def put_workflow_paused(self, value: bool) -> None: """Put workflow paused flag to workflow_params table.""" - self.put_workflow_params_1(self.KEY_PAUSED, 1) + self.put_workflow_params_1(self.KEY_PAUSED, int(value)) - def put_workflow_hold_cycle_point(self, value): + def put_workflow_hold_cycle_point( + self, value: Optional['PointBase'] + ) -> None: """Put workflow hold cycle point to workflow_params table.""" - self.put_workflow_params_1(self.KEY_HOLD_CYCLE_POINT, str(value)) + self.put_workflow_params_1( + self.KEY_HOLD_CYCLE_POINT, + str(value) if value is not None else None + ) def put_workflow_stop_clock_time(self, value): """Put workflow stop clock time to workflow_params table.""" diff --git a/tests/flakyfunctional/database/00-simple.t b/tests/flakyfunctional/database/00-simple.t index ac48d4d778c..edf86107e51 100644 --- a/tests/flakyfunctional/database/00-simple.t +++ b/tests/flakyfunctional/database/00-simple.t @@ -41,11 +41,19 @@ sqlite3 "${DB_FILE}" \ 'SELECT key, value FROM workflow_params WHERE key != "uuid_str" AND key != "cycle_point_tz" ORDER BY key' \ >"${NAME}" -sed -i "s/$(cylc --version)//g" "${NAME}" cmp_ok "${NAME}" << __EOF__ UTC_mode|0 -cylc_version| +cycle_point_format| +cylc_version|$(cylc --version) +fcp| +icp| +is_paused|0 n_restart|0 +run_mode| +startcp| +stop_clock_time| +stop_task| +stopcp| __EOF__ NAME='select-task-events.out' diff --git a/tests/flakyfunctional/restart/46-stop-clock-time.t b/tests/flakyfunctional/restart/46-stop-clock-time.t index 263eb519ad4..f220ecec0f9 100644 --- a/tests/flakyfunctional/restart/46-stop-clock-time.t +++ b/tests/flakyfunctional/restart/46-stop-clock-time.t @@ -68,7 +68,7 @@ cmp_ok 'stopclocktime.out' <<<"stop_clock_time|${CLOCKTIME}" workflow_run_ok "${TEST_NAME_BASE}-restart-1" \ cylc play "${WORKFLOW_NAME}" --no-detach dumpdbtables -cmp_ok 'stopclocktime.out' <'/dev/null' +cmp_ok 'stopclocktime.out' <<<"stop_clock_time|" cut -d ' ' -f 4- "${WORKFLOW_RUN_DIR}/log/scheduler/log" >'log.edited' if [[ "$(date +%:z)" == '+00:00' ]]; then CLOCKTIMESTR="$(date --date="@${CLOCKTIME}" +%FT%TZ)" diff --git a/tests/functional/restart/08-stop-after-cycle-point.t b/tests/functional/restart/08-stop-after-cycle-point.t index b7287920eed..dd788a4b8f1 100644 --- a/tests/functional/restart/08-stop-after-cycle-point.t +++ b/tests/functional/restart/08-stop-after-cycle-point.t @@ -47,7 +47,7 @@ cmp_ok db_taskpool.out.out << '__OUT__' 1971|hello|waiting __OUT__ # Check that the config stop point does not get stored in DB -cmp_ok db_stopcp.out < /dev/null +cmp_ok db_stopcp.out <<< '' CFG_STOPCP="1972" @@ -65,7 +65,7 @@ cmp_ok db_taskpool.out.out << '__OUT__' 1972|hello|waiting 1973|hello|waiting __OUT__ -cmp_ok db_stopcp.out < /dev/null +cmp_ok db_stopcp.out <<< '' # Check that the command line stop point works @@ -92,7 +92,7 @@ cmp_ok db_taskpool.out.out <<'__OUT__' 1975|hello|waiting __OUT__ # Stop point should be removed from DB once reached -cmp_ok db_stopcp.out < /dev/null +cmp_ok db_stopcp.out <<< '' # Restart again with new CLI stop point @@ -122,7 +122,7 @@ cmp_ok db_taskpool.out.out << '__OUT__' 1978|hello|waiting __OUT__ # Stop point should be removed from DB if --stopcp=reload used -cmp_ok db_stopcp.out < /dev/null +cmp_ok db_stopcp.out <<< '' CFG_STOPCP="1971" @@ -136,6 +136,6 @@ cmp_ok db_taskpool.out.out << '__OUT__' 1977|hello|waiting 1978|hello|waiting __OUT__ -cmp_ok db_stopcp.out < /dev/null +cmp_ok db_stopcp.out <<< '' purge diff --git a/tests/functional/restart/45-stop-task.t b/tests/functional/restart/45-stop-task.t index a02be2f5501..f910fe98b38 100644 --- a/tests/functional/restart/45-stop-task.t +++ b/tests/functional/restart/45-stop-task.t @@ -21,7 +21,7 @@ dumpdbtables() { sqlite3 "${WORKFLOW_RUN_DIR}/log/db" \ - 'SELECT * FROM workflow_params WHERE key=="stop_task";' >'stoptask.out' + 'SELECT value FROM workflow_params WHERE key=="stop_task";' >'stoptask.out' sqlite3 "${WORKFLOW_RUN_DIR}/log/db" \ 'SELECT cycle, name, status FROM task_pool ORDER BY cycle, name;' >'taskpool.out' } @@ -61,7 +61,7 @@ run_ok "${TEST_NAME_BASE}-validate" cylc validate "${WORKFLOW_NAME}" workflow_run_ok "${TEST_NAME_BASE}-run" cylc play "${WORKFLOW_NAME}" --no-detach dumpdbtables -cmp_ok 'stoptask.out' <<<'stop_task|1/t_i5' +cmp_ok 'stoptask.out' <<<'1/t_i5' cmp_ok 'taskpool.out' <<'__OUT__' 1|t_i3|waiting __OUT__ @@ -69,7 +69,7 @@ __OUT__ workflow_run_ok "${TEST_NAME_BASE}-restart-1" \ cylc play "${WORKFLOW_NAME}" --no-detach dumpdbtables -cmp_ok 'stoptask.out' <'/dev/null' +cmp_ok 'stoptask.out' <<<'' cmp_ok 'taskpool.out' <<'__OUT__' 1|t_i6|waiting __OUT__ @@ -77,7 +77,7 @@ __OUT__ workflow_run_ok "${TEST_NAME_BASE}-restart-2" \ cylc play "${WORKFLOW_NAME}" --no-detach dumpdbtables -cmp_ok 'stoptask.out' <'/dev/null' +cmp_ok 'stoptask.out' <<<'' cmp_ok 'taskpool.out' <'/dev/null' purge diff --git a/tests/functional/restart/51-final-point-reload.t b/tests/functional/restart/51-final-point-reload.t index c11718b155e..89c40c90c52 100644 --- a/tests/functional/restart/51-final-point-reload.t +++ b/tests/functional/restart/51-final-point-reload.t @@ -87,8 +87,7 @@ cmp_ok 'taskpool.out' <'/dev/null' workflow_run_ok "${TEST_NAME_BASE}-restart-3" \ cylc play "${WORKFLOW_NAME}" --no-detach --fcp=reload dumpdbtables -cmp_ok 'fcp.out' <'/dev/null' +cmp_ok 'fcp.out' <<<'fcp|' cmp_ok 'taskpool.out' <'/dev/null' purge -exit diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index f25d2410ef8..14f01e3dc5f 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -470,7 +470,9 @@ async def test_hold_point( # Test release task_pool.release_hold_point() - assert db_select(example_flow, True, 'workflow_params', key='holdcp') == [] + assert db_select(example_flow, True, 'workflow_params', key='holdcp') == [ + ('holdcp', None) + ] for itask in task_pool.get_all_tasks(): assert itask.state.is_held is False diff --git a/tests/integration/test_workflow_db_mgr.py b/tests/integration/test_workflow_db_mgr.py index 8fb292301e1..8c6c3c20e8a 100644 --- a/tests/integration/test_workflow_db_mgr.py +++ b/tests/integration/test_workflow_db_mgr.py @@ -17,8 +17,6 @@ import pytest import sqlite3 -from pkg_resources import parse_version - from cylc.flow.scheduler import Scheduler @@ -109,3 +107,23 @@ async def test_db_upgrade_pre_803( # Restart should now succeed. async with start(schd): assert ('n_restart', '2') in db_select(schd, False, 'workflow_params') + + +async def test_workflow_param_rapid_toggle( + one_conf, flow, scheduler, run +): + """Check that queuing a workflow param toggle operation twice before + processing does not cause any problems. + + https://github.com/cylc/cylc-flow/issues/5593 + """ + schd: Scheduler = scheduler(flow(one_conf), paused_start=False) + async with run(schd): + assert schd.is_paused is False + schd.pause_workflow() + schd.resume_workflow() + schd.process_workflow_db_queue() + assert schd.is_paused is False + + w_params = dict(schd.workflow_db_mgr.pri_dao.select_workflow_params()) + assert w_params['is_paused'] == '0' From dd4a09c26e21606581b723cfc32bf48f384fc403 Mon Sep 17 00:00:00 2001 From: Ronnie Dutta <61982285+MetRonnie@users.noreply.github.com> Date: Wed, 5 Jul 2023 15:09:17 +0100 Subject: [PATCH 08/24] Type annotations --- cylc/flow/rundb.py | 2 +- cylc/flow/scheduler.py | 19 ++++++++++--------- cylc/flow/workflow_db_mgr.py | 17 ++++++++++------- 3 files changed, 21 insertions(+), 17 deletions(-) diff --git a/cylc/flow/rundb.py b/cylc/flow/rundb.py index 0bbb934ab53..f7632b8997e 100644 --- a/cylc/flow/rundb.py +++ b/cylc/flow/rundb.py @@ -545,7 +545,7 @@ def select_broadcast_states(self, callback, sort=None): for row_idx, row in enumerate(self.connect().execute(stmt)): callback(row_idx, list(row)) - def select_workflow_params(self) -> Iterable[Tuple[str, str]]: + def select_workflow_params(self) -> Iterable[Tuple[str, Optional[str]]]: """Select all from workflow_params. E.g. a row might be ('UTC mode', '1') diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 362e95064b9..22bbcc5262a 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -31,6 +31,7 @@ import traceback from typing import ( TYPE_CHECKING, + Any, Callable, Iterable, NoReturn, @@ -194,6 +195,7 @@ class Scheduler: id: str # noqa: A003 (instance attr not local) uuid_str: str is_restart: bool + bad_hosts: Set[str] # directories workflow_log_dir: str @@ -223,6 +225,7 @@ class Scheduler: config: WorkflowConfig # flow config options: Values cylc_config: DictTree # [scheduler] config + template_vars: Dict[str, Any] # tcp / zmq server: WorkflowRuntimeServer @@ -231,15 +234,11 @@ class Scheduler: # flow information contact_data: Optional[dict] = None - bad_hosts: Optional[Set[str]] = None # configuration flow_file: Optional[str] = None flow_file_update_time: Optional[float] = None - # run options - template_vars: Optional[dict] = None - # workflow params stop_mode: Optional[StopMode] = None stop_task: Optional[str] = None @@ -1104,14 +1103,14 @@ def get_contact_data(self) -> Dict[str, str]: fields.OWNER: self.owner, fields.PORT: - str(self.server.port), # type: ignore + str(self.server.port), fields.PID: str(proc.pid), fields.COMMAND: cli_format(proc.cmdline()), fields.PUBLISH_PORT: - str(self.server.pub_port), # type: ignore - fields.WORKFLOW_RUN_DIR_ON_WORKFLOW_HOST: # type: ignore + str(self.server.pub_port), + fields.WORKFLOW_RUN_DIR_ON_WORKFLOW_HOST: self.workflow_run_dir, fields.UUID: self.uuid_str, @@ -1203,7 +1202,9 @@ def load_flow_file(self, is_reload=False): 'CYLC_WORKFLOW_FINAL_CYCLE_POINT': str(self.config.final_point), }) - def _load_workflow_params(self, params: Iterable[Tuple[str, str]]) -> None: + def _load_workflow_params( + self, params: Iterable[Tuple[str, Optional[str]]] + ) -> None: """Load a row in the "workflow_params" table in a restart/reload. This currently includes: @@ -1216,7 +1217,7 @@ def _load_workflow_params(self, params: Iterable[Tuple[str, str]]) -> None: """ LOG.info('LOADING workflow parameters') for key, value in params: - if not value: + if value is None: continue if key in self.workflow_db_mgr.KEY_INITIAL_CYCLE_POINT_COMPATS: self.options.icp = value diff --git a/cylc/flow/workflow_db_mgr.py b/cylc/flow/workflow_db_mgr.py index 8fd090f5612..091d1712429 100644 --- a/cylc/flow/workflow_db_mgr.py +++ b/cylc/flow/workflow_db_mgr.py @@ -360,26 +360,29 @@ def put_workflow_hold_cycle_point( str(value) if value is not None else None ) - def put_workflow_stop_clock_time(self, value): + def put_workflow_stop_clock_time(self, value: Optional[str]) -> None: """Put workflow stop clock time to workflow_params table.""" self.put_workflow_params_1(self.KEY_STOP_CLOCK_TIME, value) - def put_workflow_stop_cycle_point(self, value): + def put_workflow_stop_cycle_point(self, value: Optional[str]) -> None: """Put workflow stop cycle point to workflow_params table.""" self.put_workflow_params_1(self.KEY_STOP_CYCLE_POINT, value) - def put_workflow_stop_task(self, value): + def put_workflow_stop_task(self, value: Optional[str]) -> None: """Put workflow stop task to workflow_params table.""" self.put_workflow_params_1(self.KEY_STOP_TASK, value) - def put_workflow_template_vars(self, template_vars): + def put_workflow_template_vars( + self, template_vars: Dict[str, Any] + ) -> None: """Put template_vars in runtime database. This method queues the relevant insert statements. """ - for key, value in template_vars.items(): - self.db_inserts_map[self.TABLE_WORKFLOW_TEMPLATE_VARS].append( - {"key": key, "value": repr(value)}) + self.db_inserts_map[self.TABLE_WORKFLOW_TEMPLATE_VARS].extend( + {"key": key, "value": repr(value)} + for key, value in template_vars.items() + ) def put_task_event_timers(self, task_events_mgr): """Put statements to update the task_action_timers table.""" From 8ab9f7f7bc72eb6401aeda8d866ed7e4f22665b2 Mon Sep 17 00:00:00 2001 From: Ronnie Dutta <61982285+MetRonnie@users.noreply.github.com> Date: Wed, 5 Jul 2023 15:11:59 +0100 Subject: [PATCH 09/24] Update changelog --- CHANGES.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 2374ff5fcdd..c4d2bfca311 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -94,6 +94,10 @@ mode before running pre-configure plugins. Permit commas in xtrigger arguments and fix minor issues with the parsing of xtrigger function signatures. +[#5618](https://github.com/cylc/cylc-flow/pull/5618) - +Fix a bug when rapidly issuing the same/opposite commands e.g. pausing & +resuming a workflow. + ## __cylc-8.1.4 (Released 2023-05-04)__ ### Fixes From 6add02999e7eb68a695dfe7f18e4000396aa73bb Mon Sep 17 00:00:00 2001 From: Tim Pillinger <26465611+wxtim@users.noreply.github.com> Date: Fri, 7 Jul 2023 10:41:54 +0100 Subject: [PATCH 10/24] Fix doc build error in lint --- cylc/flow/scripts/lint.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cylc/flow/scripts/lint.py b/cylc/flow/scripts/lint.py index 1461634b9bf..9c11e630ae4 100755 --- a/cylc/flow/scripts/lint.py +++ b/cylc/flow/scripts/lint.py @@ -878,7 +878,7 @@ def get_reference_rst(checks): output += ( '\n.. note::\n' '\n U998 and U999 represent automatically generated ' - 'sets of deprecations and upgrades.' + 'sets of deprecations and upgrades.\n\n' ) if current_checkset == 'A': From fa2321e4dde0a47953759a237294af09d81a7c96 Mon Sep 17 00:00:00 2001 From: Hilary James Oliver Date: Mon, 19 Jun 2023 18:33:23 +1200 Subject: [PATCH 11/24] Move to workflow dir during parsing. --- cylc/flow/parsec/fileparse.py | 8 ++++++++ tests/unit/parsec/test_fileparse.py | 26 ++++++++++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/cylc/flow/parsec/fileparse.py b/cylc/flow/parsec/fileparse.py index 1c1d34738ba..9dd757c7cdb 100644 --- a/cylc/flow/parsec/fileparse.py +++ b/cylc/flow/parsec/fileparse.py @@ -401,6 +401,12 @@ def read_and_proc( fpath = _get_fpath_for_source(fpath, opts) fdir = os.path.dirname(fpath) + odir = os.getcwd() + + # Move to the file location to give the template processor easy access to + # other files in the workflow directory (whether source or installed). + os.chdir(fdir) + # Allow Python modules in lib/python/ (e.g. for use by Jinja2 filters). workflow_lib_python = os.path.join(fdir, "lib", "python") if ( @@ -494,6 +500,8 @@ def read_and_proc( if do_contin: flines = _concatenate(flines) + os.chdir(odir) + # return rstripped lines return [fl.rstrip() for fl in flines] diff --git a/tests/unit/parsec/test_fileparse.py b/tests/unit/parsec/test_fileparse.py index fde67099c0a..728a261b440 100644 --- a/tests/unit/parsec/test_fileparse.py +++ b/tests/unit/parsec/test_fileparse.py @@ -16,6 +16,7 @@ import tempfile +from pathlib import Path import os import pytest from pytest import param @@ -357,6 +358,31 @@ def test_read_and_proc_jinja2(): viewcfg=viewcfg) assert r == ['a=Cylc'] +def test_read_and_proc_cwd(): + """The template processor should be able to read workflow files. + + This relies on moving to the config dir during file parsing. + """ + with tempfile.TemporaryDirectory() as td: + sdir = Path(td + "/sub") + sdir.mkdir() + for sub in ["a", "b", "c"]: + (sdir / Path(sub)).touch() + viewcfg = { + 'empy': False, 'jinja2': True, + 'contin': False, 'inline': False + } + with open(td / Path("a.conf"), 'w') as tf: + tf.write( +"""#!Jinja2 +{% from "os" import listdir %} +{% for f in listdir("sub") %} +{{f}} +{% endfor %} +""") + tf.flush() + r = read_and_proc(fpath=tf.name, viewcfg=viewcfg) + assert r == ['a', 'b', 'c'] def test_read_and_proc_jinja2_error(): with tempfile.NamedTemporaryFile() as tf: From ca2cad1f0fe031ee0c5b157f32a437be55db472e Mon Sep 17 00:00:00 2001 From: Hilary James Oliver Date: Mon, 19 Jun 2023 17:03:36 +1200 Subject: [PATCH 12/24] Update change log. --- CHANGES.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 982774d091d..9ef9abe91e4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -33,6 +33,10 @@ After successful reload the scheduler will unpause the workflow. [#5537](https://github.com/cylc/cylc-flow/pull/5537) - Allow parameters in family names to be split, e.g. `FAM`. +[#5589](https://github.com/cylc/cylc-flow/pull/5589) - Move to workflow +directory during file parsing, to give the template processor access to +workflow files. + [#5405](https://github.com/cylc/cylc-flow/pull/5405) - Improve scan command help, and add scheduler PID to the output. From 27bbe1cced9ce8e669db88e6b8ccae1f89ae353d Mon Sep 17 00:00:00 2001 From: Hilary James Oliver Date: Mon, 19 Jun 2023 18:47:56 +1200 Subject: [PATCH 13/24] Tweak new test. --- tests/unit/parsec/test_fileparse.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/unit/parsec/test_fileparse.py b/tests/unit/parsec/test_fileparse.py index 728a261b440..dd3a154b286 100644 --- a/tests/unit/parsec/test_fileparse.py +++ b/tests/unit/parsec/test_fileparse.py @@ -382,7 +382,7 @@ def test_read_and_proc_cwd(): """) tf.flush() r = read_and_proc(fpath=tf.name, viewcfg=viewcfg) - assert r == ['a', 'b', 'c'] + assert sorted(r) == ['a', 'b', 'c'] def test_read_and_proc_jinja2_error(): with tempfile.NamedTemporaryFile() as tf: From 9d3b8bfc3779b2dc1cf93a6282f8c4eb4e40916b Mon Sep 17 00:00:00 2001 From: Hilary James Oliver Date: Mon, 10 Jul 2023 20:47:45 +1200 Subject: [PATCH 14/24] Tidy new unit test. --- tests/unit/parsec/test_fileparse.py | 56 +++++++++++++++++------------ 1 file changed, 33 insertions(+), 23 deletions(-) diff --git a/tests/unit/parsec/test_fileparse.py b/tests/unit/parsec/test_fileparse.py index dd3a154b286..09f13ed1ede 100644 --- a/tests/unit/parsec/test_fileparse.py +++ b/tests/unit/parsec/test_fileparse.py @@ -16,7 +16,6 @@ import tempfile -from pathlib import Path import os import pytest from pytest import param @@ -358,31 +357,42 @@ def test_read_and_proc_jinja2(): viewcfg=viewcfg) assert r == ['a=Cylc'] -def test_read_and_proc_cwd(): + +def test_read_and_proc_cwd(tmp_path): """The template processor should be able to read workflow files. - This relies on moving to the config dir during file parsing. + This relies on moving to the config dir during file parsing. """ - with tempfile.TemporaryDirectory() as td: - sdir = Path(td + "/sub") - sdir.mkdir() - for sub in ["a", "b", "c"]: - (sdir / Path(sub)).touch() - viewcfg = { - 'empy': False, 'jinja2': True, - 'contin': False, 'inline': False - } - with open(td / Path("a.conf"), 'w') as tf: - tf.write( -"""#!Jinja2 -{% from "os" import listdir %} -{% for f in listdir("sub") %} -{{f}} -{% endfor %} -""") - tf.flush() - r = read_and_proc(fpath=tf.name, viewcfg=viewcfg) - assert sorted(r) == ['a', 'b', 'c'] + + sdir = tmp_path / "sub" + sdir.mkdir() + + for sub in ["a", "b", "c"]: + (sdir / sub).touch() + + viewcfg = { + 'empy': False, + 'jinja2': True, + 'contin': False, + 'inline': False + } + + tmpf = tmp_path / "a.conf" + + with open(tmpf, 'w') as tf: + tf.write( + '#!Jinja2' + '\n{% from "os" import listdir %}' + '\n{% for f in listdir("sub") %}' + '\n{{f}}' + '\n{% endfor %}' + ) + + with open(tmpf, 'r') as tf: + r = read_and_proc(fpath=tf.name, viewcfg=viewcfg) + + assert sorted(r) == ['a', 'b', 'c'] + def test_read_and_proc_jinja2_error(): with tempfile.NamedTemporaryFile() as tf: From 54ee8faa8ed25426f7165329de90f9d202f52d99 Mon Sep 17 00:00:00 2001 From: Hilary James Oliver Date: Mon, 10 Jul 2023 20:53:42 +1200 Subject: [PATCH 15/24] Flake8 tidy. --- tests/unit/parsec/test_fileparse.py | 235 ++++++++++++++-------------- 1 file changed, 114 insertions(+), 121 deletions(-) diff --git a/tests/unit/parsec/test_fileparse.py b/tests/unit/parsec/test_fileparse.py index 09f13ed1ede..04b7245f492 100644 --- a/tests/unit/parsec/test_fileparse.py +++ b/tests/unit/parsec/test_fileparse.py @@ -14,7 +14,8 @@ # You should have received a copy of the GNU General Public License # along with this program. If not, see . -import tempfile +from tempfile import NamedTemporaryFile +from contextlib import suppress import os import pytest @@ -278,7 +279,7 @@ def test_multiline(): def test_read_and_proc_no_template_engine(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = None viewcfg = { @@ -305,7 +306,7 @@ def test_read_and_proc_no_template_engine(): def test_inline(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = None viewcfg = { @@ -313,7 +314,7 @@ def test_inline(): 'contin': False, 'inline': True, 'mark': None, 'single': None, 'label': None } - with tempfile.NamedTemporaryFile() as include_file: + with NamedTemporaryFile() as include_file: include_file.write("c=d".encode()) include_file.flush() tf.write(("a=b\n%include \"{0}\"" @@ -325,7 +326,7 @@ def test_inline(): def test_inline_error(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = None viewcfg = { @@ -342,7 +343,7 @@ def test_inline_error(): def test_read_and_proc_jinja2(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = { 'name': 'Cylc' @@ -395,7 +396,7 @@ def test_read_and_proc_cwd(tmp_path): def test_read_and_proc_jinja2_error(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = { 'name': 'Cylc' @@ -416,7 +417,7 @@ def test_read_and_proc_jinja2_error(): def test_read_and_proc_jinja2_error_missing_shebang(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = { 'name': 'Cylc' @@ -436,112 +437,107 @@ def test_read_and_proc_jinja2_error_missing_shebang(): # --- originally we had a test for empy here, moved to test_empysupport def test_parse_keys_only_singleline(): - with tempfile.NamedTemporaryFile() as of: - with tempfile.NamedTemporaryFile() as tf: - fpath = tf.name - template_vars = { - 'name': 'Cylc' - } - tf.write("#!jinja2\na={{ name }}\n".encode()) - tf.flush() - r = parse(fpath=fpath, output_fname=of.name, - template_vars=template_vars) - expected = OrderedDictWithDefaults() - expected['a'] = 'Cylc' - assert r == expected - of.flush() - output_file_contents = of.read().decode() - assert output_file_contents == 'a=Cylc\n' + with NamedTemporaryFile() as of, NamedTemporaryFile() as tf: + fpath = tf.name + template_vars = { + 'name': 'Cylc' + } + tf.write("#!jinja2\na={{ name }}\n".encode()) + tf.flush() + r = parse(fpath=fpath, output_fname=of.name, + template_vars=template_vars) + expected = OrderedDictWithDefaults() + expected['a'] = 'Cylc' + assert r == expected + of.flush() + output_file_contents = of.read().decode() + assert output_file_contents == 'a=Cylc\n' def test_parse_keys_only_multiline(): - with tempfile.NamedTemporaryFile() as of: - with tempfile.NamedTemporaryFile() as tf: - fpath = tf.name - template_vars = { - 'name': 'Cylc' - } - tf.write( - "#!jinja2\na='''value is \\\n{{ name }}'''\n".encode()) - tf.flush() - r = parse(fpath=fpath, output_fname=of.name, - template_vars=template_vars) - expected = OrderedDictWithDefaults() - expected['a'] = "'''value is Cylc'''" - assert r == expected + with NamedTemporaryFile() as of, NamedTemporaryFile() as tf: + fpath = tf.name + template_vars = { + 'name': 'Cylc' + } + tf.write( + "#!jinja2\na='''value is \\\n{{ name }}'''\n".encode()) + tf.flush() + r = parse(fpath=fpath, output_fname=of.name, + template_vars=template_vars) + expected = OrderedDictWithDefaults() + expected['a'] = "'''value is Cylc'''" + assert r == expected def test_parse_invalid_line(): - with tempfile.NamedTemporaryFile() as of: - with tempfile.NamedTemporaryFile() as tf: - fpath = tf.name - template_vars = { - 'name': 'Cylc' - } - tf.write("#!jinja2\n{{ name }}\n".encode()) - tf.flush() - with pytest.raises(FileParseError) as cm: - parse(fpath=fpath, output_fname=of.name, - template_vars=template_vars) - exc = cm.value - assert exc.reason == 'Invalid line' - assert exc.line_num == 1 - assert exc.line == 'Cylc' + with NamedTemporaryFile() as of, NamedTemporaryFile() as tf: + fpath = tf.name + template_vars = { + 'name': 'Cylc' + } + tf.write("#!jinja2\n{{ name }}\n".encode()) + tf.flush() + with pytest.raises(FileParseError) as cm: + parse(fpath=fpath, output_fname=of.name, + template_vars=template_vars) + exc = cm.value + assert exc.reason == 'Invalid line' + assert exc.line_num == 1 + assert exc.line == 'Cylc' def test_parse_comments(): - with tempfile.NamedTemporaryFile() as of: - with tempfile.NamedTemporaryFile() as tf: - fpath = tf.name - template_vars = { - 'name': 'Cylc' - } - tf.write("#!jinja2\na={{ name }}\n# comment!".encode()) - tf.flush() - r = parse(fpath=fpath, output_fname=of.name, - template_vars=template_vars) - expected = OrderedDictWithDefaults() - expected['a'] = 'Cylc' - assert r == expected - of.flush() - output_file_contents = of.read().decode() - assert output_file_contents == 'a=Cylc\n# comment!\n' + with NamedTemporaryFile() as of, NamedTemporaryFile() as tf: + fpath = tf.name + template_vars = { + 'name': 'Cylc' + } + tf.write("#!jinja2\na={{ name }}\n# comment!".encode()) + tf.flush() + r = parse(fpath=fpath, output_fname=of.name, + template_vars=template_vars) + expected = OrderedDictWithDefaults() + expected['a'] = 'Cylc' + assert r == expected + of.flush() + output_file_contents = of.read().decode() + assert output_file_contents == 'a=Cylc\n# comment!\n' def test_parse_with_sections(): - with tempfile.NamedTemporaryFile() as of: - with tempfile.NamedTemporaryFile() as tf: - fpath = tf.name - template_vars = { - 'name': 'Cylc' - } - tf.write(("#!jinja2\n[section1]\n" - "a={{ name }}\n# comment!\n" - "[[subsection1]]\n" - "[[subsection2]]\n" - "[section2]").encode()) - tf.flush() - r = parse(fpath=fpath, output_fname=of.name, - template_vars=template_vars) - expected = OrderedDictWithDefaults() - expected['section1'] = OrderedDictWithDefaults() - expected['section1']['a'] = 'Cylc' - expected['section1']['subsection1'] = OrderedDictWithDefaults() - expected['section1']['subsection2'] = OrderedDictWithDefaults() - expected['section2'] = OrderedDictWithDefaults() - assert r == expected - of.flush() - output_file_contents = of.read().decode() - assert output_file_contents == ( - '[section1]\na=Cylc\n# comment!\n' - '[[subsection1]]\n' - '[[subsection2]]\n' - '[section2]\n' - ) + with NamedTemporaryFile() as of, NamedTemporaryFile() as tf: + fpath = tf.name + template_vars = { + 'name': 'Cylc' + } + tf.write(("#!jinja2\n[section1]\n" + "a={{ name }}\n# comment!\n" + "[[subsection1]]\n" + "[[subsection2]]\n" + "[section2]").encode()) + tf.flush() + r = parse(fpath=fpath, output_fname=of.name, + template_vars=template_vars) + expected = OrderedDictWithDefaults() + expected['section1'] = OrderedDictWithDefaults() + expected['section1']['a'] = 'Cylc' + expected['section1']['subsection1'] = OrderedDictWithDefaults() + expected['section1']['subsection2'] = OrderedDictWithDefaults() + expected['section2'] = OrderedDictWithDefaults() + assert r == expected + of.flush() + output_file_contents = of.read().decode() + assert output_file_contents == ( + '[section1]\na=Cylc\n# comment!\n' + '[[subsection1]]\n' + '[[subsection2]]\n' + '[section2]\n' + ) def test_parse_with_sections_missing_bracket(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = { 'name': 'Cylc' @@ -558,27 +554,26 @@ def test_parse_with_sections_missing_bracket(): def test_parse_with_sections_error_wrong_level(): - with tempfile.NamedTemporaryFile() as of: - with tempfile.NamedTemporaryFile() as tf: - fpath = tf.name - template_vars = { - 'name': 'Cylc' - } - tf.write(("#!jinja2\n[section1]\n" - "a={{ name }}\n# comment!\n" - "[[[subsection1]]]\n") # expected [[]] instead! - .encode()) - tf.flush() - with pytest.raises(FileParseError) as cm: - parse(fpath=fpath, output_fname=of.name, - template_vars=template_vars) - exc = cm.value - assert exc.line_num == 4 - assert exc.line == '[[[subsection1]]]' + with NamedTemporaryFile() as of, NamedTemporaryFile() as tf: + fpath = tf.name + template_vars = { + 'name': 'Cylc' + } + tf.write(("#!jinja2\n[section1]\n" + "a={{ name }}\n# comment!\n" + "[[[subsection1]]]\n") # expected [[]] instead! + .encode()) + tf.flush() + with pytest.raises(FileParseError) as cm: + parse(fpath=fpath, output_fname=of.name, + template_vars=template_vars) + exc = cm.value + assert exc.line_num == 4 + assert exc.line == '[[[subsection1]]]' def test_unclosed_multiline(): - with tempfile.NamedTemporaryFile() as tf: + with NamedTemporaryFile() as tf: fpath = tf.name template_vars = { 'name': 'Cylc' @@ -681,11 +676,9 @@ def _inner(create_srclink=True): src.mkdir(exist_ok=True) link = tmp_path.parent / '_cylc-install/source' link.parent.mkdir(exist_ok=True) - try: - os.symlink(src, link) - except FileExistsError: + with suppress(FileExistsError): # We don't mind the link persisting. - pass + os.symlink(src, link) return tmp_path / 'flow.cylc' yield _inner From d5c9fd626825ea580e387f9867b0c013ad502e07 Mon Sep 17 00:00:00 2001 From: Tim Pillinger <26465611+wxtim@users.noreply.github.com> Date: Tue, 11 Jul 2023 10:57:51 +0100 Subject: [PATCH 16/24] ensure output from xtrigger scripts are logged (#5617) xtriggers: ensure stdout/err gets logged --- cylc/flow/parsec/validate.py | 10 ++--- cylc/flow/subprocctx.py | 9 ----- tests/integration/test_subprocctx.py | 60 ++++++++++++++++++++++++++++ 3 files changed, 65 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_subprocctx.py diff --git a/cylc/flow/parsec/validate.py b/cylc/flow/parsec/validate.py index 5055d01ab79..4ef66e19862 100644 --- a/cylc/flow/parsec/validate.py +++ b/cylc/flow/parsec/validate.py @@ -1065,11 +1065,11 @@ def coerce_xtrigger(cls, value, keys): Examples: >>> xtrig = CylcConfigValidator.coerce_xtrigger - # parse xtrig function signatures - >>> xtrig('a(b, c):PT1M', [None]) - a(b, c):60.0 - >>> xtrig('a(x, "1,2,3", y):PT1S', [None]) - a(x, 1,2,3, y):1.0 + >>> ctx = xtrig('a(b, c):PT1M', [None]) + >>> ctx.get_signature() + 'a(b, c)' + >>> ctx.intvl + 60.0 # cast types >>> x = xtrig('a(1, 1.1, True, abc, x=True, y=1.1)', [None]) diff --git a/cylc/flow/subprocctx.py b/cylc/flow/subprocctx.py index 85bbc284503..84f1c68e70b 100644 --- a/cylc/flow/subprocctx.py +++ b/cylc/flow/subprocctx.py @@ -158,12 +158,3 @@ def get_signature(self): args = self.func_args + [ "%s=%s" % (i, self.func_kwargs[i]) for i in skeys] return "%s(%s)" % (self.func_name, ", ".join([str(a) for a in args])) - - def __str__(self): - return ( - f'{self.func_name}(' - f'{", ".join(map(str, self.func_args + list(self.func_kwargs)))}' - f'):{self.intvl}' - ) - - __repr__ = __str__ diff --git a/tests/integration/test_subprocctx.py b/tests/integration/test_subprocctx.py new file mode 100644 index 00000000000..a7f9dd50bba --- /dev/null +++ b/tests/integration/test_subprocctx.py @@ -0,0 +1,60 @@ +# THIS FILE IS PART OF THE CYLC WORKFLOW ENGINE. +# Copyright (C) NIWA & British Crown (Met Office) & Contributors. +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. +# +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU General Public License for more details. +# +# You should have received a copy of the GNU General Public License +# along with this program. If not, see . +"""Tests involving the Cylc Subprocess Context Object +""" + +from logging import DEBUG + + +async def test_log_xtrigger_stdout( + flow, scheduler, run_dir, start, log_filter +): + """Output from xtriggers should appear in the scheduler log: + + (As per the toy example in the Cylc Docs) + """ + # Setup a workflow: + id_ = flow({ + 'scheduler': {'allow implicit tasks': True}, + 'scheduling': { + 'graph': {'R1': '@myxtrigger => foo'}, + 'xtriggers': {'myxtrigger': 'myxtrigger()'} + } + }) + # Create an xtrigger: + xt_lib = run_dir / id_ / 'lib/python/myxtrigger.py' + xt_lib.parent.mkdir(parents=True, exist_ok=True) + xt_lib.write_text( + "from sys import stderr\n\n\n" + "def myxtrigger():\n" + " print('Hello World')\n" + " print('Hello Hades', file=stderr)\n" + " return True, {}" + ) + schd = scheduler(id_) + async with start(schd, level=DEBUG) as log: + # Set off check for x-trigger: + task = schd.pool.get_tasks()[0] + schd.xtrigger_mgr.call_xtriggers_async(task) + + # while not schd.xtrigger_mgr._get_xtrigs(task): + while schd.proc_pool.is_not_done(): + schd.proc_pool.process() + + # Assert that both stderr and out from the print statement + # in our xtrigger appear in the log. + for expected in ['Hello World', 'Hello Hades']: + assert log_filter(log, contains=expected, level=DEBUG) From 5a62db2e422e8d31a55d2968e373b26aedbeea7f Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Tue, 11 Jul 2023 11:24:19 +0100 Subject: [PATCH 17/24] tests/integration: set run_mode=simulation as the default * Integration tests shouldn't go running workflows for real. * Set the simulation run mode as the default, this avoids shell interaction and reduces the number of moving parts. --- tests/integration/events/test_workflow_events.py | 8 ++++++-- tests/integration/test_scheduler.py | 5 +++-- tests/integration/utils/flow_tools.py | 8 ++++++-- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/integration/events/test_workflow_events.py b/tests/integration/events/test_workflow_events.py index f5632ddd1bc..6b742264636 100644 --- a/tests/integration/events/test_workflow_events.py +++ b/tests/integration/events/test_workflow_events.py @@ -57,7 +57,9 @@ async def test_mail_footer_template( # start the workflow and get it to send an email async with start(mod_one) as one_log: - mod_one.run_event_handlers( + one_log.clear() # clear previous log messages + mod_one.workflow_event_handler.handle( + mod_one, WorkflowEventHandler.EVENT_STARTUP, 'event message' ) @@ -100,7 +102,9 @@ async def test_custom_event_handler_template( # start the workflow and get it to send an email async with start(mod_one) as one_log: - mod_one.run_event_handlers( + one_log.clear() # clear previous log messages + mod_one.workflow_event_handler.handle( + mod_one, WorkflowEventHandler.EVENT_STARTUP, 'event message' ) diff --git a/tests/integration/test_scheduler.py b/tests/integration/test_scheduler.py index 5330e516758..05d26a1296f 100644 --- a/tests/integration/test_scheduler.py +++ b/tests/integration/test_scheduler.py @@ -202,11 +202,12 @@ async def test_no_poll_waiting_tasks( See https://github.com/cylc/cylc-flow/issues/4658 """ reg: str = flow(one_conf) - one: Scheduler = scheduler(reg, paused_start=True) + # start the scheduler in live mode in order to activate regular polling + # logic + one: Scheduler = scheduler(reg, run_mode='live') log: pytest.LogCaptureFixture async with start(one) as log: - # Test assumes start up with a waiting task. task = (one.pool.get_all_tasks())[0] assert task.state.status == TASK_STATUS_WAITING diff --git a/tests/integration/utils/flow_tools.py b/tests/integration/utils/flow_tools.py index 927884c233d..e4d7685e997 100644 --- a/tests/integration/utils/flow_tools.py +++ b/tests/integration/utils/flow_tools.py @@ -79,8 +79,12 @@ def _make_scheduler(): schd: Scheduler = None # type: ignore[assignment] def __make_scheduler(reg: str, **opts: Any) -> Scheduler: - # This allows paused_start to be overridden: - opts = {'paused_start': True, **opts} + opts = { + # safe n sane defaults for integration tests + 'paused_start': True, + 'run_mode': 'simulation', + **opts, + } options = RunOptions(**opts) # create workflow nonlocal schd From 1d8a6231c1b22d83109c04ff7a64f269af463845 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Tue, 11 Jul 2023 11:37:21 +0100 Subject: [PATCH 18/24] terminology: reg->id * Reg was a Cylc <= 6 concept relating to the registration DB. * Beyond Cylc 6 we didn't have a better name for this. * With the universal ID, workflow reg became workflow ID. --- cylc/flow/clean.py | 34 +++--- cylc/flow/install.py | 2 +- cylc/flow/network/authentication.py | 4 +- cylc/flow/scheduler.py | 4 +- cylc/flow/scripts/graph.py | 2 +- cylc/flow/scripts/install.py | 12 +- cylc/flow/task_remote_cmd.py | 2 +- cylc/flow/tui/app.py | 16 +-- cylc/flow/workflow_files.py | 62 +++++------ tests/integration/conftest.py | 26 ++--- tests/integration/graphql/test_root.py | 6 +- .../main_loop/test_auto_restart.py | 4 +- tests/integration/test_client.py | 6 +- tests/integration/test_config.py | 46 ++++---- tests/integration/test_data_store_mgr.py | 4 +- tests/integration/test_examples.py | 32 +++--- tests/integration/test_graphql.py | 6 +- tests/integration/test_install.py | 12 +- tests/integration/test_publisher.py | 4 +- tests/integration/test_resolvers.py | 4 +- tests/integration/test_scan.py | 6 +- tests/integration/test_scan_api.py | 20 ++-- tests/integration/test_scheduler.py | 20 ++-- tests/integration/test_server.py | 4 +- tests/integration/test_task_events_mgr.py | 4 +- tests/integration/test_task_job_mgr.py | 4 +- tests/integration/test_task_pool.py | 8 +- tests/integration/test_workflow_db_mgr.py | 12 +- tests/integration/test_workflow_files.py | 32 +++--- tests/integration/utils/flow_tools.py | 8 +- tests/integration/utils/test_flow_tools.py | 8 +- tests/unit/conftest.py | 12 +- tests/unit/test_clean.py | 104 +++++++++--------- tests/unit/test_config.py | 90 +++++++-------- tests/unit/test_install.py | 8 +- tests/unit/test_pathutil.py | 8 +- tests/unit/test_task_remote_mgr.py | 6 +- tests/unit/test_taskdef.py | 12 +- tests/unit/test_workflow_files.py | 20 ++-- tests/unit/xtriggers/test_workflow_state.py | 6 +- 40 files changed, 340 insertions(+), 340 deletions(-) diff --git a/cylc/flow/clean.py b/cylc/flow/clean.py index 8d980c6cfe6..992fdd24839 100644 --- a/cylc/flow/clean.py +++ b/cylc/flow/clean.py @@ -103,15 +103,15 @@ async def get_contained_workflows(partial_id) -> List[str]: ) -def _clean_check(opts: 'Values', reg: str, run_dir: Path) -> None: +def _clean_check(opts: 'Values', id_: str, run_dir: Path) -> None: """Check whether a workflow can be cleaned. Args: - reg: Workflow name. + id_: Workflow name. run_dir: Path to the workflow run dir on the filesystem. """ - validate_workflow_name(reg) - reg = os.path.normpath(reg) + validate_workflow_name(id_) + id_ = os.path.normpath(id_) # Thing to clean must be a dir or broken symlink: if not run_dir.is_dir() and not run_dir.is_symlink(): raise FileNotFoundError(f"No directory to clean at {run_dir}") @@ -124,10 +124,10 @@ def _clean_check(opts: 'Values', reg: str, run_dir: Path) -> None: # about contact file. return try: - detect_old_contact_file(reg) + detect_old_contact_file(id_) except ContactFileExists as exc: raise ServiceFileError( - f"Cannot clean running workflow {reg}.\n\n{exc}" + f"Cannot clean running workflow {id_}.\n\n{exc}" ) @@ -336,7 +336,7 @@ def _clean_using_glob( def remote_clean( - reg: str, + id_: str, platform_names: Iterable[str], rm_dirs: Optional[List[str]] = None, timeout: str = '120' @@ -345,7 +345,7 @@ def remote_clean( (skip localhost), given a set of platform names to look up. Args: - reg: Workflow name. + id_: Workflow name. platform_names: List of platform names to look up in the global config, in order to determine the install targets to clean on. rm_dirs: Sub dirs to remove instead of the whole run dir. @@ -356,18 +356,18 @@ def remote_clean( get_install_target_to_platforms_map(platform_names)) except PlatformLookupError as exc: raise PlatformLookupError( - f"Cannot clean {reg} on remote platforms as the workflow database " + f"Cannot clean {id_} on remote platforms as the workflow database " f"is out of date/inconsistent with the global config - {exc}") queue: Deque[RemoteCleanQueueTuple] = deque() remote_clean_cmd = partial( - _remote_clean_cmd, reg=reg, rm_dirs=rm_dirs, timeout=timeout + _remote_clean_cmd, id_=id_, rm_dirs=rm_dirs, timeout=timeout ) for target, platforms in install_targets_map.items(): if target == get_localhost_install_target(): continue shuffle(platforms) LOG.info( - f"Cleaning {reg} on install target: " + f"Cleaning {id_} on install target: " f"{platforms[0]['install target']}" ) # Issue ssh command: @@ -417,13 +417,13 @@ def remote_clean( if failed_targets: for target, excp in failed_targets.items(): LOG.error( - f"Could not clean {reg} on install target: {target}\n{excp}" + f"Could not clean {id_} on install target: {target}\n{excp}" ) - raise CylcError(f"Remote clean failed for {reg}") + raise CylcError(f"Remote clean failed for {id_}") def _remote_clean_cmd( - reg: str, + id_: str, platform: Dict[str, Any], rm_dirs: Optional[List[str]], timeout: str @@ -433,7 +433,7 @@ def _remote_clean_cmd( Call "cylc clean --local-only" over ssh and return the subprocess. Args: - reg: Workflow name. + id_: Workflow name. platform: Config for the platform on which to remove the workflow. rm_dirs: Sub dirs to remove instead of the whole run dir. timeout: Number of seconds to wait before cancelling the command. @@ -443,10 +443,10 @@ def _remote_clean_cmd( """ LOG.debug( - f"Cleaning {reg} on install target: {platform['install target']} " + f"Cleaning {id_} on install target: {platform['install target']} " f"(using platform: {platform['name']})" ) - cmd = ['clean', '--local-only', reg] + cmd = ['clean', '--local-only', id_] if rm_dirs is not None: for item in rm_dirs: cmd.extend(['--rm', item]) diff --git a/cylc/flow/install.py b/cylc/flow/install.py index 7db6eacb939..351b080046e 100644 --- a/cylc/flow/install.py +++ b/cylc/flow/install.py @@ -583,7 +583,7 @@ def check_nested_dirs( install dirs. Raises: - WorkflowFilesError if reg dir is nested inside a run dir, or an + WorkflowFilesError if id_ dir is nested inside a run dir, or an install dirs are nested. """ if install_dir is not None: diff --git a/cylc/flow/network/authentication.py b/cylc/flow/network/authentication.py index b30392da84b..d9c571b8bf1 100644 --- a/cylc/flow/network/authentication.py +++ b/cylc/flow/network/authentication.py @@ -24,12 +24,12 @@ remove_keys_on_server) -def key_housekeeping(reg, platform=None, create=True): +def key_housekeeping(id_, platform=None, create=True): """Clean any existing authentication keys and create new ones. If create is set to false, keys will only be cleaned from server.""" - workflow_srv_dir = get_workflow_srv_dir(reg) + workflow_srv_dir = get_workflow_srv_dir(id_) keys = { "client_public_key": KeyInfo( KeyType.PUBLIC, diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 8d27a7387db..10076b7b0a8 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -271,9 +271,9 @@ class Scheduler: time_next_kill: Optional[float] = None - def __init__(self, reg: str, options: Values) -> None: + def __init__(self, id_: str, options: Values) -> None: # flow information - self.workflow = reg + self.workflow = id_ self.workflow_name = get_workflow_name_from_id(self.workflow) self.owner = get_user() self.host = get_host() diff --git a/cylc/flow/scripts/graph.py b/cylc/flow/scripts/graph.py index ce22cfdc2ed..a4721bb6920 100644 --- a/cylc/flow/scripts/graph.py +++ b/cylc/flow/scripts/graph.py @@ -198,7 +198,7 @@ def _get_inheritance_nodes_and_edges( def get_config(workflow_id: str, opts: 'Values', flow_file) -> WorkflowConfig: - """Return a WorkflowConfig object for the provided reg / path.""" + """Return a WorkflowConfig object for the provided id_ / path.""" template_vars = get_template_vars(opts) return WorkflowConfig( workflow_id, flow_file, opts, template_vars=template_vars diff --git a/cylc/flow/scripts/install.py b/cylc/flow/scripts/install.py index 867e83966e4..1d5ba236037 100755 --- a/cylc/flow/scripts/install.py +++ b/cylc/flow/scripts/install.py @@ -265,18 +265,18 @@ async def scan(wf_name: str, ping: bool = True) -> None: def main( _parser: COP, opts: 'Values', - reg: Optional[str] = None + id_: Optional[str] = None ) -> None: """CLI wrapper.""" - install_cli(opts, reg) + install_cli(opts, id_) def install_cli( opts: 'Values', - reg: Optional[str] = None + id_: Optional[str] = None ) -> Tuple[str, str]: """Install workflow and scan for already-running instances.""" - wf_name, wf_id = install(opts, reg) + wf_name, wf_id = install(opts, id_) asyncio.run( scan(wf_name, not opts.no_ping) ) @@ -284,14 +284,14 @@ def install_cli( def install( - opts: 'Values', reg: Optional[str] = None + opts: 'Values', id_: Optional[str] = None ) -> Tuple[str, str]: set_timestamps(LOG, opts.log_timestamp and opts.verbosity > 1) if opts.no_run_name and opts.run_name: raise InputError( "options --no-run-name and --run-name are mutually exclusive." ) - source = get_source_location(reg) + source = get_source_location(id_) # Check deprecation to allow plugins to have access to correct flags # for compatibility mode: diff --git a/cylc/flow/task_remote_cmd.py b/cylc/flow/task_remote_cmd.py index 28774220b55..4f66c0f9b0f 100644 --- a/cylc/flow/task_remote_cmd.py +++ b/cylc/flow/task_remote_cmd.py @@ -60,7 +60,7 @@ def remove_keys_on_client(srvd, install_target, full_clean=False): def create_client_keys(srvd, install_target): - """Create or renew authentication keys for workflow 'reg' in the .service + """Create or renew authentication keys for workflow 'id_' in the .service directory. Generate a pair of ZMQ authentication keys""" diff --git a/cylc/flow/tui/app.py b/cylc/flow/tui/app.py index eb4e248d49c..cf09f75db0e 100644 --- a/cylc/flow/tui/app.py +++ b/cylc/flow/tui/app.py @@ -201,7 +201,7 @@ class TuiApp: tab/selection panel. Arguments: - reg (str): + id_ (str): Workflow registration """ @@ -227,8 +227,8 @@ class TuiApp: for status, spec in WORKFLOW_COLOURS.items() ] - def __init__(self, reg, screen=None): - self.reg = reg + def __init__(self, id_, screen=None): + self.id_ = id_ self.client = None self.loop = None self.screen = None @@ -297,7 +297,7 @@ def get_snapshot(self): """ try: if not self.client: - self.client = get_client(self.reg, timeout=self.CLIENT_TIMEOUT) + self.client = get_client(self.id_, timeout=self.CLIENT_TIMEOUT) data = self.client( 'graphql', { @@ -315,7 +315,7 @@ def get_snapshot(self): except WorkflowStopped: # Distinguish stopped flow from non-existent flow. self.client = None - full_path = Path(get_workflow_run_dir(self.reg)) + full_path = Path(get_workflow_run_dir(self.id_)) if ( (full_path / WorkflowFiles.SUITE_RC).is_file() or (full_path / WorkflowFiles.FLOW_FILE).is_file() @@ -324,12 +324,12 @@ def get_snapshot(self): else: message = ( f"No {WorkflowFiles.SUITE_RC} or {WorkflowFiles.FLOW_FILE}" - f"found in {self.reg}." + f"found in {self.id_}." ) return dummy_flow({ - 'name': self.reg, - 'id': self.reg, + 'name': self.id_, + 'id': self.id_, 'status': message, 'stateTotals': {} }) diff --git a/cylc/flow/workflow_files.py b/cylc/flow/workflow_files.py index 7fc4cddfea2..3f1d011a9da 100644 --- a/cylc/flow/workflow_files.py +++ b/cylc/flow/workflow_files.py @@ -434,7 +434,7 @@ def _is_process_running( def detect_old_contact_file( - reg: str, contact_data=None + id_: str, contact_data=None ) -> None: """Check if the workflow process is still running. @@ -450,7 +450,7 @@ def detect_old_contact_file( ContactFileExists. Args: - reg: workflow name + id_: workflow ID contact_date: Raises: @@ -467,7 +467,7 @@ def detect_old_contact_file( # and can be loaded. if not contact_data: try: - contact_data = load_contact_file(reg) + contact_data = load_contact_file(id_) except (IOError, ValueError, ServiceFileError): # Contact file does not exist or corrupted, workflow should be dead return @@ -487,7 +487,7 @@ def detect_old_contact_file( # NOTE: can raise CylcError process_is_running = _is_process_running(old_host, old_pid, old_cmd) - fname = get_contact_file_path(reg) + fname = get_contact_file_path(id_) if process_is_running: # ... the process is running, raise an exception raise ContactFileExists( @@ -496,7 +496,7 @@ def detect_old_contact_file( "port": old_port, "pid": old_pid, "fname": fname, - "workflow": reg, + "workflow": id_, } ) else: @@ -512,16 +512,16 @@ def detect_old_contact_file( # unexpected error removing the contact file # (note the FileNotFoundError incorporated errno.ENOENT) LOG.error( - f'Failed to remove contact file for {reg}:\n{exc}' + f'Failed to remove contact file for {id_}:\n{exc}' ) else: LOG.info( - f'Removed contact file for {reg}' + f'Removed contact file for {id_}' ' (workflow no longer running).' ) -def dump_contact_file(reg, data): +def dump_contact_file(id_, data): """Create contact file. Data should be a key=value dict.""" # Note: # 1st fsync for writing the content of the contact file to disk. @@ -529,24 +529,24 @@ def dump_contact_file(reg, data): # The double fsync logic ensures that if the contact file is written to # a shared file system e.g. via NFS, it will be immediately visible # from by a process on other hosts after the current process returns. - with open(get_contact_file_path(reg), "wb") as handle: + with open(get_contact_file_path(id_), "wb") as handle: for key, value in sorted(data.items()): handle.write(("%s=%s\n" % (key, value)).encode()) os.fsync(handle.fileno()) - dir_fileno = os.open(get_workflow_srv_dir(reg), os.O_DIRECTORY) + dir_fileno = os.open(get_workflow_srv_dir(id_), os.O_DIRECTORY) os.fsync(dir_fileno) os.close(dir_fileno) -def get_contact_file_path(reg: str) -> str: +def get_contact_file_path(id_: str) -> str: """Return name of contact file.""" return os.path.join( - get_workflow_srv_dir(reg), WorkflowFiles.Service.CONTACT) + get_workflow_srv_dir(id_), WorkflowFiles.Service.CONTACT) -def get_flow_file(reg: str) -> Path: +def get_flow_file(id_: str) -> Path: """Return the path of a workflow's flow.cylc file.""" - run_dir = get_workflow_run_dir(reg) + run_dir = get_workflow_run_dir(id_) path = check_flow_file(run_dir) return path @@ -581,15 +581,15 @@ def get_workflow_source_dir( return None, None -def get_workflow_srv_dir(reg): +def get_workflow_srv_dir(id_): """Return service directory of a workflow.""" run_d = os.getenv("CYLC_WORKFLOW_RUN_DIR") if ( not run_d - or os.getenv("CYLC_WORKFLOW_ID") != reg + or os.getenv("CYLC_WORKFLOW_ID") != id_ or os.getenv("CYLC_WORKFLOW_OWNER") != get_user() ): - run_d = get_workflow_run_dir(reg) + run_d = get_workflow_run_dir(id_) return os.path.join(run_d, WorkflowFiles.Service.DIRNAME) @@ -688,7 +688,7 @@ def is_installed(rund: Union[Path, str]) -> bool: return cylc_install_dir.is_dir() or alt_cylc_install_dir.is_dir() -def get_symlink_dirs(reg: str, run_dir: Union[Path, str]) -> Dict[str, Path]: +def get_symlink_dirs(id_: str, run_dir: Union[Path, str]) -> Dict[str, Path]: """Return the standard symlink dirs and their targets if they exist in the workflow run dir. @@ -706,7 +706,7 @@ def get_symlink_dirs(reg: str, run_dir: Union[Path, str]) -> Dict[str, Path]: raise WorkflowFilesError( f'Invalid symlink at {path}.\n' f'Link target is not a directory: {target}') - expected_end = str(Path('cylc-run', reg, _dir)) + expected_end = str(Path('cylc-run', id_, _dir)) if not str(target).endswith(expected_end): raise WorkflowFilesError( f'Invalid symlink at {path}\n' @@ -729,7 +729,7 @@ def remove_keys_on_server(keys): def create_server_keys(keys, workflow_srv_dir): - """Create or renew authentication keys for workflow 'reg' in the .service + """Create or renew authentication keys for workflow 'id_' in the .service directory. Generate a pair of ZMQ authentication keys""" import zmq.auth @@ -756,7 +756,7 @@ def create_server_keys(keys, workflow_srv_dir): os.umask(old_umask) -def get_workflow_title(reg): +def get_workflow_title(id_): """Return the the workflow title without a full file parse Limitations: @@ -764,7 +764,7 @@ def get_workflow_title(reg): * Assume title is not in an include-file. """ title = NO_TITLE - with open(get_flow_file(reg), 'r') as handle: + with open(get_flow_file(id_), 'r') as handle: for line in handle: if line.lstrip().startswith("[meta]"): # continue : title comes inside [meta] section @@ -842,8 +842,8 @@ def check_reserved_dir_names(name: Union[Path, str]) -> None: def infer_latest_run_from_id(workflow_id: str) -> str: run_dir = Path(get_workflow_run_dir(workflow_id)) - _, reg = infer_latest_run(run_dir) - return reg + _, id_ = infer_latest_run(run_dir) + return id_ def infer_latest_run( @@ -862,7 +862,7 @@ def infer_latest_run( Returns: path: Absolute path of the numbered run dir if applicable, otherwise the input arg path. - reg: The workflow name (including the numbered run if applicable). + id_: The workflow name (including the numbered run if applicable). Raises: - WorkflowFilesError if the runN symlink is not valid. @@ -870,12 +870,12 @@ def infer_latest_run( """ cylc_run_dir = get_cylc_run_dir() try: - reg = str(path.relative_to(cylc_run_dir)) + id_ = str(path.relative_to(cylc_run_dir)) except ValueError: raise ValueError(f"{path} is not in the cylc-run directory") if not path.exists(): raise InputError( - f'Workflow ID not found: {reg}\n(Directory not found: {path})' + f'Workflow ID not found: {id_}\n(Directory not found: {path})' ) if path.name == WorkflowFiles.RUN_N: runN_path = path @@ -888,9 +888,9 @@ def infer_latest_run( elif implicit_runN: runN_path = path / WorkflowFiles.RUN_N if not os.path.lexists(runN_path): - return (path, reg) + return (path, id_) else: - return (path, reg) + return (path, id_) if not runN_path.is_symlink() or not runN_path.is_dir(): raise WorkflowFilesError( f"{runN_path} symlink not valid" @@ -903,8 +903,8 @@ def infer_latest_run( f"{runN_path} symlink target not valid: {numbered_run}" ) path = runN_path.parent / numbered_run - reg = str(path.relative_to(cylc_run_dir)) - return (path, reg) + id_ = str(path.relative_to(cylc_run_dir)) + return (path, id_) def is_valid_run_dir(path: Union[Path, str]) -> bool: diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index b2a4f0dde97..f2f24b09ab5 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -100,7 +100,7 @@ def run_dir(): @pytest.fixture(scope='session') def ses_test_dir(request, run_dir): - """The root reg dir for test flows in this test session.""" + """The root run dir for test flows in this test session.""" timestamp = get_current_time_string(use_basic_format=True) uuid = f'cit-{timestamp}' path = Path(run_dir, uuid) @@ -111,7 +111,7 @@ def ses_test_dir(request, run_dir): @pytest.fixture(scope='module') def mod_test_dir(request, ses_test_dir): - """The root reg dir for test flows in this test module.""" + """The root run dir for test flows in this test module.""" path = Path(ses_test_dir, request.module.__name__) path.mkdir(exist_ok=True) yield path @@ -125,7 +125,7 @@ def mod_test_dir(request, ses_test_dir): @pytest.fixture def test_dir(request, mod_test_dir): - """The root reg dir for test flows in this test function.""" + """The root run dir for test flows in this test function.""" path = Path(mod_test_dir, request.function.__name__) path.mkdir(parents=True, exist_ok=True) yield path @@ -164,7 +164,7 @@ def scheduler(): """Return a Scheduler object for a flow. Args: - reg (str): Workflow name. + id_ (str): Workflow name. **opts (Any): Options to be passed to the Scheduler. """ with _make_scheduler() as _scheduler: @@ -226,15 +226,15 @@ def mod_one_conf(): @pytest.fixture def one(one_conf, flow, scheduler): """Return a Scheduler for the simple "R1 = one" graph.""" - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) return schd @pytest.fixture(scope='module') def mod_one(mod_one_conf, mod_flow, mod_scheduler): - reg = mod_flow(mod_one_conf) - schd = mod_scheduler(reg) + id_ = mod_flow(mod_one_conf) + schd = mod_scheduler(id_) return schd @@ -342,14 +342,14 @@ def validate(run_dir): errors. Args: - reg - The flow to validate + id_ - The flow to validate kwargs - Arguments to pass to ValidateOptions """ - def _validate(reg: Union[str, Path], **kwargs) -> WorkflowConfig: - reg = str(reg) + def _validate(id_: Union[str, Path], **kwargs) -> WorkflowConfig: + id_ = str(id_) return WorkflowConfig( - reg, - str(Path(run_dir, reg, 'flow.cylc')), + id_, + str(Path(run_dir, id_, 'flow.cylc')), ValidateOptions(**kwargs) ) diff --git a/tests/integration/graphql/test_root.py b/tests/integration/graphql/test_root.py index 12b16822abb..27290316cf7 100644 --- a/tests/integration/graphql/test_root.py +++ b/tests/integration/graphql/test_root.py @@ -21,7 +21,7 @@ @pytest.fixture(scope='module') async def harness(mod_flow, mod_scheduler, mod_run): - reg = mod_flow({ + id_ = mod_flow({ 'scheduling': { 'graph': { 'R1': ''' @@ -42,9 +42,9 @@ async def harness(mod_flow, mod_scheduler, mod_run): 'b': {}, }, }) - schd = mod_scheduler(reg) + schd = mod_scheduler(id_) async with mod_run(schd): - client = WorkflowRuntimeClient(reg) + client = WorkflowRuntimeClient(id_) async def _query(query_string): nonlocal client diff --git a/tests/integration/main_loop/test_auto_restart.py b/tests/integration/main_loop/test_auto_restart.py index a83fb7c7932..20cc4ea81c6 100644 --- a/tests/integration/main_loop/test_auto_restart.py +++ b/tests/integration/main_loop/test_auto_restart.py @@ -42,8 +42,8 @@ async def test_no_detach( 'cylc.flow.main_loop.auto_restart._should_auto_restart', Mock(return_value=AutoRestartMode.RESTART_NORMAL) ) - reg: str = flow(one_conf) - schd: Scheduler = scheduler(reg, paused_start=True, no_detach=True) + id_: str = flow(one_conf) + schd: Scheduler = scheduler(id_, paused_start=True, no_detach=True) with pytest.raises(MainLoopPluginException) as exc: async with run(schd) as log: await asyncio.sleep(2) diff --git a/tests/integration/test_client.py b/tests/integration/test_client.py index 07485d2dcf1..013d929e820 100644 --- a/tests/integration/test_client.py +++ b/tests/integration/test_client.py @@ -23,10 +23,10 @@ @pytest.fixture(scope='module') async def harness(mod_flow, mod_scheduler, mod_run, mod_one_conf): - reg = mod_flow(mod_one_conf) - schd = mod_scheduler(reg) + id_ = mod_flow(mod_one_conf) + schd = mod_scheduler(id_) async with mod_run(schd): - client = WorkflowRuntimeClient(reg) + client = WorkflowRuntimeClient(id_) yield schd, client diff --git a/tests/integration/test_config.py b/tests/integration/test_config.py index b3799f213da..acf24d17eaf 100644 --- a/tests/integration/test_config.py +++ b/tests/integration/test_config.py @@ -48,7 +48,7 @@ def test_validate_task_name( valid: bool ): """It should raise errors for invalid task names in the runtime section.""" - reg = flow({ + id_ = flow({ **one_conf, 'runtime': { task_name: {} @@ -56,10 +56,10 @@ def test_validate_task_name( }) if valid: - validate(reg) + validate(id_) else: with pytest.raises(WorkflowConfigError) as exc_ctx: - validate(reg) + validate(id_) assert task_name in str(exc_ctx.value) @@ -82,7 +82,7 @@ def test_validate_implicit_task_name( Here we ensure that names which look like valid graph node names but which are blacklisted get caught and raise errors. """ - reg = flow({ + id_ = flow({ 'scheduler': { 'allow implicit tasks': 'True' }, @@ -99,7 +99,7 @@ def test_validate_implicit_task_name( }) with pytest.raises(WorkflowConfigError) as exc_ctx: - validate(reg) + validate(id_) assert str(exc_ctx.value).splitlines()[0] == ( f'invalid task name "{task_name}"' ) @@ -114,7 +114,7 @@ def test_validate_implicit_task_name( ) def test_validate_env_vars(flow, one_conf, validate, env_var, valid): """It should validate environment variable names.""" - reg = flow({ + id_ = flow({ **one_conf, 'runtime': { 'foo': { @@ -125,10 +125,10 @@ def test_validate_env_vars(flow, one_conf, validate, env_var, valid): } }) if valid: - validate(reg) + validate(id_) else: with pytest.raises(WorkflowConfigError) as exc_ctx: - validate(reg) + validate(id_) assert env_var in str(exc_ctx.value) @@ -147,7 +147,7 @@ def test_validate_param_env_templ( log_filter, ): """It should validate parameter environment templates.""" - reg = flow({ + id_ = flow({ **one_conf, 'runtime': { 'foo': { @@ -157,18 +157,18 @@ def test_validate_param_env_templ( } } }) - validate(reg) + validate(id_) assert log_filter(caplog, contains='bad parameter environment template') assert log_filter(caplog, contains=env_val) def test_no_graph(flow, validate): """It should fail for missing graph sections.""" - reg = flow({ + id_ = flow({ 'scheduling': {}, }) with pytest.raises(WorkflowConfigError) as exc_ctx: - validate(reg) + validate(id_) assert 'missing [scheduling][[graph]] section.' in str(exc_ctx.value) @@ -181,7 +181,7 @@ def test_no_graph(flow, validate): ) def test_parse_special_tasks_invalid(flow, validate, section): """It should fail for invalid "special tasks".""" - reg = flow({ + id_ = flow({ 'scheduler': { 'allow implicit tasks': 'True', }, @@ -196,14 +196,14 @@ def test_parse_special_tasks_invalid(flow, validate, section): } }) with pytest.raises(WorkflowConfigError) as exc_ctx: - validate(reg) + validate(id_) assert f'Illegal {section} spec' in str(exc_ctx.value) assert 'foo' in str(exc_ctx.value) def test_parse_special_tasks_interval(flow, validate): """It should fail for invalid durations in clock-triggers.""" - reg = flow({ + id_ = flow({ 'scheduler': { 'allow implicit tasks': 'True', }, @@ -218,7 +218,7 @@ def test_parse_special_tasks_interval(flow, validate): } }) with pytest.raises(WorkflowConfigError) as exc_ctx: - validate(reg) + validate(id_) assert 'Illegal clock-trigger spec' in str(exc_ctx.value) assert 'PT1Y' in str(exc_ctx.value) @@ -232,7 +232,7 @@ def test_parse_special_tasks_interval(flow, validate): ) def test_parse_special_tasks_families(flow, scheduler, validate, section): """It should expand families in special tasks.""" - reg = flow({ + id_ = flow({ 'scheduling': { 'initial cycle point': 'now', 'special tasks': { @@ -261,10 +261,10 @@ def test_parse_special_tasks_families(flow, scheduler, validate, section): # external triggers cannot be used for multiple tasks so if family # expansion is completed correctly, validation should fail with pytest.raises(WorkflowConfigError) as exc_ctx: - config = validate(reg) + config = validate(id_) assert 'external triggers must be used only once' in str(exc_ctx.value) else: - config = validate(reg) + config = validate(id_) assert set(config.cfg['scheduling']['special tasks'][section]) == { # the family FOO has been expanded to the tasks foo, foot 'foo(P1D)', @@ -277,7 +277,7 @@ def test_queue_treated_as_implicit(flow, validate, caplog): https://github.com/cylc/cylc-flow/issues/5260 """ - reg = flow( + id_ = flow( { "scheduling": { "queues": {"my_queue": {"members": "task1, task2"}}, @@ -286,7 +286,7 @@ def test_queue_treated_as_implicit(flow, validate, caplog): "runtime": {"task2": {}}, } ) - validate(reg) + validate(id_) assert ( 'Queues contain tasks not defined in runtime' in caplog.records[0].message @@ -298,7 +298,7 @@ def test_queue_treated_as_comma_separated(flow, validate): https://github.com/cylc/cylc-flow/issues/5260 """ - reg = flow( + id_ = flow( { "scheduling": { "queues": {"my_queue": {"members": "task1 task2"}}, @@ -308,7 +308,7 @@ def test_queue_treated_as_comma_separated(flow, validate): } ) with pytest.raises(ListValueError, match="cannot contain a space"): - validate(reg) + validate(id_) def test_validate_incompatible_db(one_conf, flow, validate): diff --git a/tests/integration/test_data_store_mgr.py b/tests/integration/test_data_store_mgr.py index 84060e28762..6dbae4b6fd5 100644 --- a/tests/integration/test_data_store_mgr.py +++ b/tests/integration/test_data_store_mgr.py @@ -100,8 +100,8 @@ async def harness(mod_flow, mod_scheduler, mod_start): } } } - reg: str = mod_flow(flow_def) - schd: 'Scheduler' = mod_scheduler(reg) + id_: str = mod_flow(flow_def) + schd: 'Scheduler' = mod_scheduler(id_) async with mod_start(schd): await schd.update_data_structure() data = schd.data_store_mgr.data[schd.data_store_mgr.workflow_id] diff --git a/tests/integration/test_examples.py b/tests/integration/test_examples.py index c5e3439afe0..cd0fa9f5277 100644 --- a/tests/integration/test_examples.py +++ b/tests/integration/test_examples.py @@ -31,7 +31,7 @@ async def test_create_flow(flow, run_dir): """Use the flow fixture to create workflows on the file system.""" # Ensure a flow.cylc file gets written out - reg = flow({ + id_ = flow({ 'scheduler': { 'allow implicit tasks': True }, @@ -41,7 +41,7 @@ async def test_create_flow(flow, run_dir): } } }) - workflow_dir = run_dir / reg + workflow_dir = run_dir / id_ flow_file = workflow_dir / 'flow.cylc' assert workflow_dir.exists() @@ -51,8 +51,8 @@ async def test_create_flow(flow, run_dir): async def test_run(flow, scheduler, run, one_conf): """Create a workflow, initialise the scheduler and run it.""" # Ensure the scheduler can survive for at least one second without crashing - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) async with run(schd): await asyncio.sleep(1) # this yields control to the main loop @@ -60,8 +60,8 @@ async def test_run(flow, scheduler, run, one_conf): async def test_logging(flow, scheduler, start, one_conf, log_filter): """We can capture log records when we run a scheduler.""" # Ensure that the cylc version is logged on startup. - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) async with start(schd) as log: # this returns a list of log records containing __version__ assert log_filter(log, contains=__version__) @@ -76,12 +76,12 @@ async def test_scheduler_arguments(flow, scheduler, start, one_conf): """ # Ensure the paused_start option is obeyed by the scheduler. - reg = flow(one_conf) - schd = scheduler(reg, paused_start=True) + id_ = flow(one_conf) + schd = scheduler(id_, paused_start=True) async with start(schd): assert schd.is_paused - reg = flow(one_conf) - schd = scheduler(reg, paused_start=False) + id_ = flow(one_conf) + schd = scheduler(id_, paused_start=False) async with start(schd): assert not schd.is_paused @@ -95,8 +95,8 @@ async def test_shutdown(flow, scheduler, start, one_conf): """ # Ensure the TCP server shuts down with the scheduler. - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) async with start(schd): pass assert schd.server.replier.socket.closed @@ -110,8 +110,8 @@ async def test_install(flow, scheduler, one_conf, run_dir): """ # Ensure the installation of the job script is completed. - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) await schd.install() assert Path( run_dir, schd.workflow, '.service', 'etc', 'job.sh' @@ -193,8 +193,8 @@ async def myflow(mod_flow, mod_scheduler, mod_one_conf): object you are testing in the tests. """ - reg = mod_flow(mod_one_conf) - schd = mod_scheduler(reg) + id_ = mod_flow(mod_one_conf) + schd = mod_scheduler(id_) return schd diff --git a/tests/integration/test_graphql.py b/tests/integration/test_graphql.py index 81d0c806069..ddad4907660 100644 --- a/tests/integration/test_graphql.py +++ b/tests/integration/test_graphql.py @@ -91,10 +91,10 @@ async def harness(mod_flow, mod_scheduler, mod_run): }, }, } - reg: str = mod_flow(flow_def) - schd: 'Scheduler' = mod_scheduler(reg) + id_: str = mod_flow(flow_def) + schd: 'Scheduler' = mod_scheduler(id_) async with mod_run(schd): - client = WorkflowRuntimeClient(reg) + client = WorkflowRuntimeClient(id_) schd.pool.hold_tasks('*') schd.resume_workflow() # Think this is needed to save the data state at first start (?) diff --git a/tests/integration/test_install.py b/tests/integration/test_install.py index 330eedb12c0..cbac55c5361 100644 --- a/tests/integration/test_install.py +++ b/tests/integration/test_install.py @@ -110,14 +110,14 @@ def test_install_scan_no_ping( opts = InstallOptions() opts.no_ping = True - install_cli(opts, reg='w1') + install_cli(opts, id_='w1') out = capsys.readouterr().out assert INSTALLED_MSG.format(wfrun='w1/run2') in out assert WF_ACTIVE_MSG.format(wf='w1') in out # Empty contact file faked with "touch": assert f"{BAD_CONTACT_MSG} w1/run1" in caplog.text - install_cli(opts, reg='w2') + install_cli(opts, id_='w2') out = capsys.readouterr().out assert WF_ACTIVE_MSG.format(wf='w2') not in out assert INSTALLED_MSG.format(wfrun='w2/run1') in out @@ -136,7 +136,7 @@ def test_install_scan_ping( opts = InstallOptions() opts.no_ping = False - install_cli(opts, reg='w1') + install_cli(opts, id_='w1') out = capsys.readouterr().out assert INSTALLED_MSG.format(wfrun='w1/run2') in out assert WF_ACTIVE_MSG.format(wf='w1') in out @@ -144,7 +144,7 @@ def test_install_scan_ping( # Empty contact file faked with "touch": assert f"{BAD_CONTACT_MSG} w1/run1" in caplog.text - install_cli(opts, reg='w2') + install_cli(opts, id_='w2') out = capsys.readouterr().out assert INSTALLED_MSG.format(wfrun='w2/run1') in out assert WF_ACTIVE_MSG.format(wf='w2') not in out @@ -182,9 +182,9 @@ def raiser(*_, **__): opts = InstallOptions() monkeypatch.setattr('cylc.flow.flags.cylc7_back_compat', False) - install_cli(opts, reg='w1') + install_cli(opts, id_='w1') assert capsys.readouterr()[0].split('\n')[0] == 'Plugin:False' monkeypatch.setattr('cylc.flow.flags.cylc7_back_compat', True) - install_cli(opts, reg='w1') + install_cli(opts, id_='w1') assert capsys.readouterr()[0].split('\n')[0] == 'Plugin:True' diff --git a/tests/integration/test_publisher.py b/tests/integration/test_publisher.py index dadc3b7bea8..c1fe7e1bcd2 100644 --- a/tests/integration/test_publisher.py +++ b/tests/integration/test_publisher.py @@ -24,8 +24,8 @@ async def test_publisher(flow, scheduler, run, one_conf, port_range): """It should publish deltas when the flow starts.""" - reg = flow(one_conf) - schd = scheduler(reg, paused_start=False) + id_ = flow(one_conf) + schd = scheduler(id_, paused_start=False) async with run(schd): # create a subscriber subscriber = WorkflowSubscriber( diff --git a/tests/integration/test_resolvers.py b/tests/integration/test_resolvers.py index b6e32708505..7e1e4afc7d6 100644 --- a/tests/integration/test_resolvers.py +++ b/tests/integration/test_resolvers.py @@ -56,7 +56,7 @@ async def mock_flow( mod_start, ) -> AsyncGenerator[Scheduler, None]: ret = Mock() - ret.reg = mod_flow({ + ret.id_ = mod_flow({ 'scheduler': { 'allow implicit tasks': True }, @@ -69,7 +69,7 @@ async def mock_flow( } }) - ret.schd = mod_scheduler(ret.reg, paused_start=True) + ret.schd = mod_scheduler(ret.id_, paused_start=True) async with mod_start(ret.schd): ret.schd.pool.release_runahead_tasks() ret.schd.data_store_mgr.initiate_data_model() diff --git a/tests/integration/test_scan.py b/tests/integration/test_scan.py index f4c6837067d..26dba076514 100644 --- a/tests/integration/test_scan.py +++ b/tests/integration/test_scan.py @@ -408,8 +408,8 @@ async def test_scan_sigstop( """ # run a workflow - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) async with start(schd): # stop the server to make the flow un-responsive await schd.server.stop('make-unresponsive') @@ -419,7 +419,7 @@ async def test_scan_sigstop( async for flow in pipe: raise Exception("There shouldn't be any scan results") # there should, however, be a warning - name = Path(reg).name + name = Path(id_).name assert ( (30, f'Workflow not running: {name}') in [(level, msg) for _, level, msg in caplog.record_tuples] diff --git a/tests/integration/test_scan_api.py b/tests/integration/test_scan_api.py index b84c4e064ba..ea25035e804 100644 --- a/tests/integration/test_scan_api.py +++ b/tests/integration/test_scan_api.py @@ -251,9 +251,9 @@ async def test_scan_cleans_stuck_contact_files( ): """Ensure scan tidies up contact files from crashed flows.""" # create a flow - reg = flow(one_conf, name='-crashed-') - schd = scheduler(reg) - srv_dir = Path(run_dir, reg, WorkflowFiles.Service.DIRNAME) + id_ = flow(one_conf, name='-crashed-') + schd = scheduler(id_) + srv_dir = Path(run_dir, id_, WorkflowFiles.Service.DIRNAME) tmp_dir = test_dir / 'srv' cont = srv_dir / WorkflowFiles.Service.CONTACT @@ -269,9 +269,9 @@ async def test_scan_cleans_stuck_contact_files( # integration test the process is the pytest process and it is still # running so we need to change the command so that Cylc sees the flow as # having crashed - contact_info = load_contact_file(reg) + contact_info = load_contact_file(id_) contact_info[ContactFileFields.COMMAND] += 'xyz' - dump_contact_file(reg, contact_info) + dump_contact_file(id_, contact_info) # make sure this flow shows for a regular filesystem-only scan opts = ScanOptions(states='running,paused', format='name') @@ -306,9 +306,9 @@ async def test_scan_fail_well_when_client_unreachable( elegently. """ # create a flow - reg = flow(one_conf, name='-crashed-') - schd = scheduler(reg) - srv_dir = Path(run_dir, reg, WorkflowFiles.Service.DIRNAME) + id_ = flow(one_conf, name='-crashed-') + schd = scheduler(id_) + srv_dir = Path(run_dir, id_, WorkflowFiles.Service.DIRNAME) tmp_dir = test_dir / 'srv' # run the flow, copy the contact, stop the flow, copy back the contact @@ -323,9 +323,9 @@ async def test_scan_fail_well_when_client_unreachable( # integration test the process is the pytest process and it is still # running so we need to change the command so that Cylc sees the flow as # having crashed - contact_info = load_contact_file(reg) + contact_info = load_contact_file(id_) contact_info[ContactFileFields.COMMAND] += 'xyz' - dump_contact_file(reg, contact_info) + dump_contact_file(id_, contact_info) # Run Cylc Scan opts = ScanOptions(states='all', format='rich', ping=True) diff --git a/tests/integration/test_scheduler.py b/tests/integration/test_scheduler.py index 05d26a1296f..7358da08a05 100644 --- a/tests/integration/test_scheduler.py +++ b/tests/integration/test_scheduler.py @@ -43,8 +43,8 @@ async def test_is_paused_after_stop( one_conf: Fixture, flow: Fixture, scheduler: Fixture, run: Fixture, db_select: Fixture): """Test the paused status is unset on normal shutdown.""" - reg: str = flow(one_conf) - schd: 'Scheduler' = scheduler(reg, paused_start=True) + id_: str = flow(one_conf) + schd: 'Scheduler' = scheduler(id_, paused_start=True) # Run async with run(schd): assert not schd.is_restart @@ -52,7 +52,7 @@ async def test_is_paused_after_stop( # Stopped assert ('is_paused', '1') not in db_select(schd, False, 'workflow_params') # Restart - schd = scheduler(reg, paused_start=None) + schd = scheduler(id_, paused_start=None) async with run(schd): assert schd.is_restart assert not schd.is_paused @@ -62,8 +62,8 @@ async def test_is_paused_after_crash( one_conf: Fixture, flow: Fixture, scheduler: Fixture, run: Fixture, db_select: Fixture): """Test the paused status is not unset for an interrupted workflow.""" - reg: str = flow(one_conf) - schd: 'Scheduler' = scheduler(reg, paused_start=True) + id_: str = flow(one_conf) + schd: 'Scheduler' = scheduler(id_, paused_start=True) def ctrl_c(): raise asyncio.CancelledError("Mock keyboard interrupt") @@ -81,7 +81,7 @@ def ctrl_c(): # Reset patched method setattr(schd, 'workflow_shutdown', _schd_workflow_shutdown) # Restart - schd = scheduler(reg, paused_start=None) + schd = scheduler(id_, paused_start=None) async with run(schd): assert schd.is_restart assert schd.is_paused @@ -154,8 +154,8 @@ async def test_holding_tasks_whilst_scheduler_paused( See https://github.com/cylc/cylc-flow/issues/4278 """ - reg = flow(one_conf) - one = scheduler(reg, paused_start=True) + id_ = flow(one_conf) + one = scheduler(id_, paused_start=True) # run the workflow async with start(one): @@ -201,10 +201,10 @@ async def test_no_poll_waiting_tasks( See https://github.com/cylc/cylc-flow/issues/4658 """ - reg: str = flow(one_conf) + id_: str = flow(one_conf) # start the scheduler in live mode in order to activate regular polling # logic - one: Scheduler = scheduler(reg, run_mode='live') + one: Scheduler = scheduler(id_, run_mode='live') log: pytest.LogCaptureFixture async with start(one) as log: diff --git a/tests/integration/test_server.py b/tests/integration/test_server.py index 63a6bd0d67c..bc7103b8365 100644 --- a/tests/integration/test_server.py +++ b/tests/integration/test_server.py @@ -26,8 +26,8 @@ @pytest.fixture(scope='module') async def myflow(mod_flow, mod_scheduler, mod_run, mod_one_conf): - reg = mod_flow(mod_one_conf) - schd = mod_scheduler(reg) + id_ = mod_flow(mod_one_conf) + schd = mod_scheduler(id_) async with mod_run(schd): yield schd diff --git a/tests/integration/test_task_events_mgr.py b/tests/integration/test_task_events_mgr.py index 72ab6807714..baa133cd6cd 100644 --- a/tests/integration/test_task_events_mgr.py +++ b/tests/integration/test_task_events_mgr.py @@ -32,8 +32,8 @@ async def test_process_job_logs_retrieval_warns_no_platform( max_size=256, key='skarloey' ) - reg: str = flow(one_conf) - schd: 'Scheduler' = scheduler(reg, paused_start=True) + id_: str = flow(one_conf) + schd: 'Scheduler' = scheduler(id_, paused_start=True) # Run async with run(schd): schd.task_events_mgr._process_job_logs_retrieval( diff --git a/tests/integration/test_task_job_mgr.py b/tests/integration/test_task_job_mgr.py index d0c99abdb56..297ad889df5 100644 --- a/tests/integration/test_task_job_mgr.py +++ b/tests/integration/test_task_job_mgr.py @@ -114,8 +114,8 @@ async def test__run_job_cmd_logs_platform_lookup_fail( db_select: Fixture, caplog: Fixture ) -> None: """TaskJobMg._run_job_cmd handles failure to get platform.""" - reg: str = flow(one_conf) - schd: 'Scheduler' = scheduler(reg, paused_start=True) + id_: str = flow(one_conf) + schd: 'Scheduler' = scheduler(id_, paused_start=True) # Run async with run(schd): from types import SimpleNamespace diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index 39c12029f1d..58909c7c1aa 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -100,8 +100,8 @@ async def mod_example_flow( This is module-scoped so faster than example_flow, but should only be used where the test does not mutate the state of the scheduler or task pool. """ - reg = mod_flow(EXAMPLE_FLOW_CFG) - schd: Scheduler = mod_scheduler(reg, paused_start=True) + id_ = mod_flow(EXAMPLE_FLOW_CFG) + schd: Scheduler = mod_scheduler(id_, paused_start=True) async with mod_run(schd): pass return schd @@ -122,8 +122,8 @@ async def example_flow( # The run(schd) fixture doesn't work for modifying the DB, so have to # set up caplog and do schd.install()/.initialise()/.configure() instead caplog.set_level(logging.INFO, CYLC_LOG) - reg = flow(EXAMPLE_FLOW_CFG) - schd: Scheduler = scheduler(reg) + id_ = flow(EXAMPLE_FLOW_CFG) + schd: Scheduler = scheduler(id_) async with start(schd): yield schd diff --git a/tests/integration/test_workflow_db_mgr.py b/tests/integration/test_workflow_db_mgr.py index 8fb292301e1..ade87722880 100644 --- a/tests/integration/test_workflow_db_mgr.py +++ b/tests/integration/test_workflow_db_mgr.py @@ -26,12 +26,12 @@ async def test_restart_number( flow, one_conf, start, scheduler, log_filter, db_select ): """The restart number should increment correctly.""" - reg = flow(one_conf) + id_ = flow(one_conf) async def test(expected_restart_num: int, do_reload: bool = False): """(Re)start the workflow and check the restart number is as expected. """ - schd: Scheduler = scheduler(reg, paused_start=True) + schd: Scheduler = scheduler(id_, paused_start=True) async with start(schd) as log: if do_reload: schd.command_reload_workflow() @@ -81,10 +81,10 @@ async def test_db_upgrade_pre_803( flow, one_conf, start, scheduler, log_filter, db_select ): """Test scheduler restart with upgrade of pre-8.0.3 DB.""" - reg = flow(one_conf) + id_ = flow(one_conf) # Run a scheduler to create a DB. - schd: Scheduler = scheduler(reg, paused_start=True) + schd: Scheduler = scheduler(id_, paused_start=True) async with start(schd): assert ('n_restart', '0') in db_select(schd, False, 'workflow_params') @@ -92,7 +92,7 @@ async def test_db_upgrade_pre_803( db_remove_column(schd, "task_states", "is_manual_submit") db_remove_column(schd, "task_jobs", "flow_nums") - schd: Scheduler = scheduler(reg, paused_start=True) + schd: Scheduler = scheduler(id_, paused_start=True) # Restart should fail due to the missing column. with pytest.raises(sqlite3.OperationalError): @@ -100,7 +100,7 @@ async def test_db_upgrade_pre_803( pass assert ('n_restart', '1') in db_select(schd, False, 'workflow_params') - schd: Scheduler = scheduler(reg, paused_start=True) + schd: Scheduler = scheduler(id_, paused_start=True) # Run the DB upgrader for version 8.0.2 # (8.0.2 requires upgrade) diff --git a/tests/integration/test_workflow_files.py b/tests/integration/test_workflow_files.py index 91933bd7074..54d60bea4d0 100644 --- a/tests/integration/test_workflow_files.py +++ b/tests/integration/test_workflow_files.py @@ -40,8 +40,8 @@ @pytest.fixture(scope='module') async def myflow(mod_flow, mod_scheduler, mod_run, mod_one_conf): - reg = mod_flow(mod_one_conf) - schd = mod_scheduler(reg) + id_ = mod_flow(mod_one_conf) + schd = mod_scheduler(id_) async with mod_run(schd): yield schd @@ -62,8 +62,8 @@ async def test_load_contact_file_async(myflow): @pytest.fixture async def workflow(flow, scheduler, one_conf, run_dir): - reg = flow(one_conf) - schd = scheduler(reg) + id_ = flow(one_conf) + schd = scheduler(id_) await schd.install() from collections import namedtuple @@ -73,15 +73,15 @@ async def workflow(flow, scheduler, one_conf, run_dir): contact_data = schd.get_contact_data() contact_file = Path( run_dir, - reg, + id_, WorkflowFiles.Service.DIRNAME, WorkflowFiles.Service.CONTACT ) def dump_contact(**kwargs): - nonlocal contact_data, reg + nonlocal contact_data, id_ dump_contact_file( - reg, + id_, { **contact_data, **kwargs @@ -94,20 +94,20 @@ def dump_contact(**kwargs): Fixture = namedtuple( 'TextFixture', [ - 'reg', + 'id_', 'contact_file', 'contact_data', 'dump_contact', ] ) - return Fixture(reg, contact_file, contact_data, dump_contact) + return Fixture(id_, contact_file, contact_data, dump_contact) def test_detect_old_contact_file_running(workflow): """It should raise an error if the workflow is running.""" # the workflow is running so we should get a ServiceFileError with pytest.raises(ContactFileExists): - detect_old_contact_file(workflow.reg) + detect_old_contact_file(workflow.id_) # the contact file is valid so should be left alone assert workflow.contact_file.exists() @@ -124,7 +124,7 @@ def test_detect_old_contact_file_network_issue(workflow): # detect_old_contact_file should report that it can't tell if the workflow # is running or not with pytest.raises(CylcError) as exc_ctx: - detect_old_contact_file(workflow.reg) + detect_old_contact_file(workflow.id_) assert ( 'Cannot determine whether workflow is running' in str(exc_ctx.value) @@ -145,7 +145,7 @@ def test_detect_old_contact_file_old_run(workflow, caplog, log_filter): # the workflow should not appear to be running (according to the contact # data) so detect_old_contact_file should not raise any errors - detect_old_contact_file(workflow.reg) + detect_old_contact_file(workflow.id_) # as a side effect the contact file should have been removed assert not workflow.contact_file.exists() @@ -159,7 +159,7 @@ def test_detect_old_contact_file_none(workflow): assert not workflow.contact_file.exists() # detect_old_contact_file should return - detect_old_contact_file(workflow.reg) + detect_old_contact_file(workflow.id_) # it should not recreate the contact file assert not workflow.contact_file.exists() @@ -236,9 +236,9 @@ def _unlink(*args): if process_running: # this should error if the process is running with pytest.raises(ContactFileExists): - detect_old_contact_file(workflow.reg) + detect_old_contact_file(workflow.id_) else: - detect_old_contact_file(workflow.reg) + detect_old_contact_file(workflow.id_) # decide which log messages we should expect to see if process_running: @@ -264,7 +264,7 @@ def _unlink(*args): assert bool(log_filter( caplog, contains=( - f'Failed to remove contact file for {workflow.reg}:' + f'Failed to remove contact file for {workflow.id_}:' '\nmocked-os-error' ), )) is remove_failed diff --git a/tests/integration/utils/flow_tools.py b/tests/integration/utils/flow_tools.py index e4d7685e997..69cf3e065f1 100644 --- a/tests/integration/utils/flow_tools.py +++ b/tests/integration/utils/flow_tools.py @@ -65,12 +65,12 @@ def _make_flow( name = str(uuid1()) flow_run_dir = (test_dir / name) flow_run_dir.mkdir(parents=True, exist_ok=True) - reg = str(flow_run_dir.relative_to(cylc_run_dir)) + id_ = str(flow_run_dir.relative_to(cylc_run_dir)) if isinstance(conf, dict): conf = flow_config_str(conf) with open((flow_run_dir / WorkflowFiles.FLOW_FILE), 'w+') as flow_file: flow_file.write(conf) - return reg + return id_ @contextmanager @@ -78,7 +78,7 @@ def _make_scheduler(): """Return a scheduler object for a flow registration.""" schd: Scheduler = None # type: ignore[assignment] - def __make_scheduler(reg: str, **opts: Any) -> Scheduler: + def __make_scheduler(id_: str, **opts: Any) -> Scheduler: opts = { # safe n sane defaults for integration tests 'paused_start': True, @@ -88,7 +88,7 @@ def __make_scheduler(reg: str, **opts: Any) -> Scheduler: options = RunOptions(**opts) # create workflow nonlocal schd - schd = Scheduler(reg, options) + schd = Scheduler(id_, options) return schd yield __make_scheduler diff --git a/tests/integration/utils/test_flow_tools.py b/tests/integration/utils/test_flow_tools.py index e3ec926364b..04e292eb4ed 100644 --- a/tests/integration/utils/test_flow_tools.py +++ b/tests/integration/utils/test_flow_tools.py @@ -25,8 +25,8 @@ # test _make_flow via the conftest fixture def test_flow(run_dir, flow, one_conf): """It should create a flow in the run directory.""" - reg = flow(one_conf) - assert Path(run_dir / reg).exists() - assert Path(run_dir / reg / 'flow.cylc').exists() - with open(Path(run_dir / reg / 'flow.cylc'), 'r') as flow_file: + id_ = flow(one_conf) + assert Path(run_dir / id_).exists() + assert Path(run_dir / id_ / 'flow.cylc').exists() + with open(Path(run_dir / id_ / 'flow.cylc'), 'r') as flow_file: assert 'scheduling' in flow_file.read() diff --git a/tests/unit/conftest.py b/tests/unit/conftest.py index 80b64ab604c..68095a5795e 100644 --- a/tests/unit/conftest.py +++ b/tests/unit/conftest.py @@ -70,7 +70,7 @@ def _tmp_run_dir(tmp_path: Path, monkeypatch: pytest.MonkeyPatch): Adds the runN symlink automatically if the workflow ID ends with /run__. Args: - reg: Workflow name. + id_: Workflow name. installed: If True, make it look like the workflow was installed using cylc install (creates _cylc-install dir). named: If True and installed is True, the _cylc-install dir will @@ -83,7 +83,7 @@ def _tmp_run_dir(tmp_path: Path, monkeypatch: pytest.MonkeyPatch): cylc_run_dir = tmp_run_dir() """ def __tmp_run_dir( - reg: Optional[str] = None, + id_: Optional[str] = None, installed: bool = False, named: bool = False ) -> Path: @@ -92,8 +92,8 @@ def __tmp_run_dir( cylc_run_dir = tmp_path / 'cylc-run' cylc_run_dir.mkdir(exist_ok=True) monkeypatch.setattr('cylc.flow.pathutil._CYLC_RUN_DIR', cylc_run_dir) - if reg: - run_dir = cylc_run_dir.joinpath(reg) + if id_: + run_dir = cylc_run_dir.joinpath(id_) run_dir.mkdir(parents=True, exist_ok=True) (run_dir / WorkflowFiles.FLOW_FILE).touch(exist_ok=True) (run_dir / WorkflowFiles.Service.DIRNAME).mkdir(exist_ok=True) @@ -102,8 +102,8 @@ def __tmp_run_dir( link_runN(run_dir) if installed: if named: - if len(Path(reg).parts) < 2: - raise ValueError("Named run requires two-level reg") + if len(Path(id_).parts) < 2: + raise ValueError("Named run requires two-level id_") (run_dir.parent / WorkflowFiles.Install.DIRNAME).mkdir( exist_ok=True) else: diff --git a/tests/unit/test_clean.py b/tests/unit/test_clean.py index f335b2d110d..61c8964fd3c 100644 --- a/tests/unit/test_clean.py +++ b/tests/unit/test_clean.py @@ -86,7 +86,7 @@ def glbl_cfg_max_scan_depth(mock_glbl_cfg: Callable) -> None: @pytest.mark.parametrize( - 'reg, stopped, err, err_msg', + 'id_, stopped, err, err_msg', [ ('foo/..', True, WorkflowFilesError, "cannot be a path that points to the cylc-run directory or above"), @@ -96,7 +96,7 @@ def glbl_cfg_max_scan_depth(mock_glbl_cfg: Callable) -> None: ] ) def test_clean_check__fail( - reg: str, + id_: str, stopped: bool, err: Type[Exception], err_msg: str, @@ -106,7 +106,7 @@ def test_clean_check__fail( """Test that _clean_check() fails appropriately. Params: - reg: Workflow name. + id_: Workflow name. stopped: Whether the workflow is stopped when _clean_check() is called. err: Expected error class. err_msg: Message that is expected to be in the exception. @@ -121,7 +121,7 @@ def mocked_detect_old_contact_file(*a, **k): ) with pytest.raises(err) as exc: - cylc_clean._clean_check(CleanOptions(), reg, tmp_path) + cylc_clean._clean_check(CleanOptions(), id_, tmp_path) assert err_msg in str(exc.value) @@ -162,15 +162,15 @@ def test_init_clean( clean_called: If a local clean is expected to go ahead. remote_clean_called: If a remote clean is expected to go ahead. """ - reg = 'foo/bar/' - rdir = tmp_run_dir(reg, installed=True) + id_ = 'foo/bar/' + rdir = tmp_run_dir(id_, installed=True) Path(rdir, WorkflowFiles.Service.DIRNAME, WorkflowFiles.Service.DB).touch() mock_clean = monkeymock('cylc.flow.clean.clean') mock_remote_clean = monkeymock('cylc.flow.clean.remote_clean') monkeypatch.setattr('cylc.flow.clean.get_platforms_from_db', lambda x: set(db_platforms)) - init_clean(reg, opts=CleanOptions(**opts)) + init_clean(id_, opts=CleanOptions(**opts)) assert mock_clean.called is clean_called assert mock_remote_clean.called is remote_clean_called @@ -261,8 +261,8 @@ def test_init_clean__rm_dirs( expected_remote_clean: The dirs that are expected to be passed to remote_clean(). """ - reg = 'dagobah' - run_dir: Path = tmp_run_dir(reg) + id_ = 'dagobah' + run_dir: Path = tmp_run_dir(id_) Path(run_dir, WorkflowFiles.Service.DIRNAME, WorkflowFiles.Service.DB).touch() mock_clean = monkeymock('cylc.flow.clean.clean') mock_remote_clean = monkeymock('cylc.flow.clean.remote_clean') @@ -271,14 +271,14 @@ def test_init_clean__rm_dirs( lambda x: platforms) opts = CleanOptions(rm_dirs=rm_dirs) if rm_dirs else CleanOptions() - init_clean(reg, opts=opts) - mock_clean.assert_called_with(reg, run_dir, expected_clean) + init_clean(id_, opts=opts) + mock_clean.assert_called_with(id_, run_dir, expected_clean) mock_remote_clean.assert_called_with( - reg, platforms, expected_remote_clean, opts.remote_timeout) + id_, platforms, expected_remote_clean, opts.remote_timeout) @pytest.mark.parametrize( - 'reg, symlink_dirs, rm_dirs, expected_deleted, expected_remaining', + 'id_, symlink_dirs, rm_dirs, expected_deleted, expected_remaining', [ pytest.param( 'foo/bar', @@ -357,7 +357,7 @@ def test_init_clean__rm_dirs( ] ) def test_clean( - reg: str, + id_: str, symlink_dirs: Dict[str, str], rm_dirs: Optional[Set[str]], expected_deleted: List[str], @@ -367,7 +367,7 @@ def test_clean( """Test the clean() function. Params: - reg: Workflow name. + id_: Workflow name. symlink_dirs: As you would find in the global config under [symlink dirs][platform]. rm_dirs: As passed to clean(). @@ -377,16 +377,16 @@ def test_clean( not expected to be cleaned. """ # --- Setup --- - run_dir: Path = tmp_run_dir(reg) + run_dir: Path = tmp_run_dir(id_) if 'run' in symlink_dirs: - target = tmp_path / symlink_dirs['run'] / 'cylc-run' / reg + target = tmp_path / symlink_dirs['run'] / 'cylc-run' / id_ target.mkdir(parents=True) shutil.rmtree(run_dir) run_dir.symlink_to(target) symlink_dirs.pop('run') for symlink_name, target_name in symlink_dirs.items(): - target = tmp_path / target_name / 'cylc-run' / reg / symlink_name + target = tmp_path / target_name / 'cylc-run' / id_ / symlink_name target.mkdir(parents=True) symlink = run_dir / symlink_name symlink.symlink_to(target) @@ -398,7 +398,7 @@ def test_clean( assert (tmp_path / rel_path).exists() # --- The actual test --- - cylc_clean.clean(reg, run_dir, rm_dirs) + cylc_clean.clean(id_, run_dir, rm_dirs) for rel_path in expected_deleted: assert (tmp_path / rel_path).exists() is False assert (tmp_path / rel_path).is_symlink() is False @@ -411,16 +411,16 @@ def test_clean__broken_symlink_run_dir( ) -> None: """Test clean() successfully remove a run dir that is a broken symlink.""" # Setup - reg = 'foo/bar' - run_dir: Path = tmp_run_dir(reg) - target = tmp_path.joinpath('rabbow/cylc-run', reg) + id_ = 'foo/bar' + run_dir: Path = tmp_run_dir(id_) + target = tmp_path.joinpath('rabbow/cylc-run', id_) target.mkdir(parents=True) shutil.rmtree(run_dir) run_dir.symlink_to(target) target.rmdir() assert run_dir.parent.exists() is True # cylc-run/foo should exist # Test - cylc_clean.clean(reg, run_dir) + cylc_clean.clean(id_, run_dir) assert run_dir.parent.exists() is False # cylc-run/foo should be gone assert target.parent.exists() is False # rabbow/cylc-run/foo too @@ -430,16 +430,16 @@ def test_clean__bad_symlink_dir_wrong_type( ) -> None: """Test clean() raises error when a symlink dir actually points to a file instead of a dir""" - reg = 'foo' - run_dir: Path = tmp_run_dir(reg) + id_ = 'foo' + run_dir: Path = tmp_run_dir(id_) symlink = run_dir.joinpath('log') - target = tmp_path.joinpath('sym-log', 'cylc-run', reg, 'meow.txt') + target = tmp_path.joinpath('sym-log', 'cylc-run', id_, 'meow.txt') target.parent.mkdir(parents=True) target.touch() symlink.symlink_to(target) with pytest.raises(WorkflowFilesError) as exc: - cylc_clean.clean(reg, run_dir) + cylc_clean.clean(id_, run_dir) assert "Invalid symlink at" in str(exc.value) assert symlink.exists() is True @@ -465,13 +465,13 @@ def test_clean__bad_symlink_dir_wrong_form( def test_clean__rm_dir_not_file(pattern: str, tmp_run_dir: Callable): """Test clean() does not remove a file when the rm_dir glob pattern would match a dir only.""" - reg = 'foo' - run_dir: Path = tmp_run_dir(reg) + id_ = 'foo' + run_dir: Path = tmp_run_dir(id_) a_file = run_dir.joinpath('thing') a_file.touch() rm_dirs = parse_rm_dirs([pattern]) - cylc_clean.clean(reg, run_dir, rm_dirs) + cylc_clean.clean(id_, run_dir, rm_dirs) assert a_file.exists() @@ -483,7 +483,7 @@ def filetree_for_testing_cylc_clean(tmp_path: Path): See tests/unit/filetree.py Args: - reg: Workflow name. + id_: Workflow name. initial_filetree: The filetree before cleaning. filetree_left_behind: The filetree that is expected to be left behind after cleaning, excluding the 'you-shall-not-pass/' directory, @@ -495,7 +495,7 @@ def filetree_for_testing_cylc_clean(tmp_path: Path): files_not_to_delete: List of files that are not expected to be deleted. """ def _filetree_for_testing_cylc_clean( - reg: str, + id_: str, initial_filetree: Dict[str, Any], filetree_left_behind: Dict[str, Any] ) -> Tuple[Path, List[str], List[str]]: @@ -512,7 +512,7 @@ def _filetree_for_testing_cylc_clean( files_not_to_delete ) ) - run_dir = tmp_path / 'cylc-run' / reg + run_dir = tmp_path / 'cylc-run' / id_ return run_dir, files_to_delete, files_not_to_delete return _filetree_for_testing_cylc_clean @@ -753,16 +753,16 @@ def test_clean__targeted( # --- Setup --- caplog.set_level(logging.DEBUG, CYLC_LOG) tmp_run_dir() - reg = 'foo/bar' + id_ = 'foo/bar' run_dir: Path files_to_delete: List[str] files_not_to_delete: List[str] run_dir, files_to_delete, files_not_to_delete = ( filetree_for_testing_cylc_clean( - reg, initial_filetree, filetree_left_behind) + id_, initial_filetree, filetree_left_behind) ) # --- Test --- - cylc_clean.clean(reg, run_dir, rm_dirs) + cylc_clean.clean(id_, run_dir, rm_dirs) for file in files_not_to_delete: assert os.path.exists(file) is True for file in files_to_delete: @@ -920,7 +920,7 @@ def test_remote_clean( # Remove randomness: monkeymock('cylc.flow.clean.shuffle') - def mocked_remote_clean_cmd_side_effect(reg, platform, rm_dirs, timeout): + def mocked_remote_clean_cmd_side_effect(id_, platform, rm_dirs, timeout): proc_ret_code = 0 if failed_platforms and platform['name'] in failed_platforms: proc_ret_code = failed_platforms[platform['name']] @@ -936,23 +936,23 @@ def mocked_remote_clean_cmd_side_effect(reg, platform, rm_dirs, timeout): side_effect=mocked_remote_clean_cmd_side_effect) rm_dirs = ["whatever"] # ----- Test ----- - reg = 'foo' + id_ = 'foo' platform_names = ( "This arg bypassed as we provide the install targets map in the test") if exc_expected: with pytest.raises(CylcError) as exc: cylc_clean.remote_clean( - reg, platform_names, rm_dirs, timeout='irrelevant') + id_, platform_names, rm_dirs, timeout='irrelevant') assert "Remote clean failed" in str(exc.value) else: cylc_clean.remote_clean( - reg, platform_names, rm_dirs, timeout='irrelevant') + id_, platform_names, rm_dirs, timeout='irrelevant') for msg in expected_err_msgs: assert log_filter(caplog, level=logging.ERROR, contains=msg) if expected_platforms: for p_name in expected_platforms: mocked_remote_clean_cmd.assert_any_call( - reg, PLATFORMS[p_name], rm_dirs, 'irrelevant') + id_, PLATFORMS[p_name], rm_dirs, 'irrelevant') else: mocked_remote_clean_cmd.assert_not_called() if failed_platforms: @@ -980,7 +980,7 @@ def test_remote_clean_cmd( expected_args: Expected CLI arguments of the cylc clean command that gets constructed. """ - reg = 'jean/luc/picard' + id_ = 'jean/luc/picard' platform = { 'name': 'enterprise', 'install target': 'mars', @@ -991,30 +991,30 @@ def test_remote_clean_cmd( 'cylc.flow.clean.construct_ssh_cmd', return_value=['blah']) monkeymock('cylc.flow.clean.Popen') - cylc_clean._remote_clean_cmd(reg, platform, rm_dirs, timeout='dunno') + cylc_clean._remote_clean_cmd(id_, platform, rm_dirs, timeout='dunno') args, kwargs = mock_construct_ssh_cmd.call_args constructed_cmd = args[0] - assert constructed_cmd == ['clean', '--local-only', reg, *expected_args] + assert constructed_cmd == ['clean', '--local-only', id_, *expected_args] def test_clean_top_level(tmp_run_dir: Callable): """Test that cleaning last remaining run dir inside a workflow dir removes the top level dir if it's empty (excluding _cylc-install).""" # Setup - reg = 'blue/planet/run1' - run_dir: Path = tmp_run_dir(reg, installed=True, named=True) + id_ = 'blue/planet/run1' + run_dir: Path = tmp_run_dir(id_, installed=True, named=True) cylc_install_dir = run_dir.parent / WorkflowFiles.Install.DIRNAME assert cylc_install_dir.is_dir() runN_symlink = run_dir.parent / WorkflowFiles.RUN_N assert runN_symlink.exists() # Test - clean(reg, run_dir) + clean(id_, run_dir) assert not run_dir.parent.parent.exists() # Now check that if the top level dir is not empty, it doesn't get removed - run_dir: Path = tmp_run_dir(reg, installed=True, named=True) + run_dir: Path = tmp_run_dir(id_, installed=True, named=True) jellyfish_file = (run_dir.parent / 'jellyfish.txt') jellyfish_file.touch() - clean(reg, run_dir) + clean(id_, run_dir) assert cylc_install_dir.is_dir() assert jellyfish_file.exists() @@ -1088,10 +1088,10 @@ def test_glob_in_run_dir( """ # Setup cylc_run_dir: Path = tmp_run_dir() - reg = 'foo/bar' - run_dir = cylc_run_dir / reg + id_ = 'foo/bar' + run_dir = cylc_run_dir / id_ create_filetree(filetree, tmp_path, tmp_path) - symlink_dirs = [run_dir / i for i in get_symlink_dirs(reg, run_dir)] + symlink_dirs = [run_dir / i for i in get_symlink_dirs(id_, run_dir)] expected = [tmp_path / i for i in expected_matches] # Test assert glob_in_run_dir(run_dir, pattern, symlink_dirs) == expected diff --git a/tests/unit/test_config.py b/tests/unit/test_config.py index 6d746dc6de5..4e9f579c937 100644 --- a/tests/unit/test_config.py +++ b/tests/unit/test_config.py @@ -54,13 +54,13 @@ def _tmp_flow_config(tmp_run_dir: Callable): """Create a temporary flow config file for use in init'ing WorkflowConfig. Args: - reg: Workflow name. + id_: Workflow name. config: The flow file content. Returns the path to the flow file. """ - def __tmp_flow_config(reg: str, config: str) -> Path: - run_dir: Path = tmp_run_dir(reg) + def __tmp_flow_config(id_: str, config: str) -> Path: + run_dir: Path = tmp_run_dir(id_) flow_file = run_dir / WorkflowFiles.FLOW_FILE flow_file.write_text(config) return flow_file @@ -239,8 +239,8 @@ def test_family_inheritance_and_quotes( hosts = localhost ''' ) - reg = 'test' - file_path = tmp_flow_config(reg, f''' + id_ = 'test' + file_path = tmp_flow_config(id_, f''' [scheduler] allow implicit tasks = True [task parameters] @@ -259,7 +259,7 @@ def test_family_inheritance_and_quotes( inherit = 'MAINFAM', {fam_txt} ''') config = WorkflowConfig( - reg, file_path, template_vars={}, options=Values() + id_, file_path, template_vars={}, options=Values() ) assert ('goodbye_0_major1_minor10' in config.runtime['descendants']['MAINFAM_major1_minor10']) @@ -780,8 +780,8 @@ def test_stopcp_after_fcp( """Test that setting a stop after cycle point that is beyond the final cycle point is handled correctly.""" caplog.set_level(logging.WARNING, CYLC_LOG) - reg = 'cassini' - flow_file: Path = tmp_flow_config(reg, f""" + id_ = 'cassini' + flow_file: Path = tmp_flow_config(id_, f""" [scheduler] allow implicit tasks = True [scheduling] @@ -791,7 +791,7 @@ def test_stopcp_after_fcp( [[graph]] P1Y = huygens """) - cfg = WorkflowConfig(reg, flow_file, options=RunOptions(**opts)) + cfg = WorkflowConfig(id_, flow_file, options=RunOptions(**opts)) msg = "will have no effect as it is after the final cycle point" if warning_expected: assert msg in caplog.text @@ -992,8 +992,8 @@ def _test(cp_tz, utc_mode, expected, expected_warnings=0): def test_rsync_includes_will_not_accept_sub_directories(tmp_flow_config): - reg = 'rsynctest' - flow_file = tmp_flow_config(reg, """ + id_ = 'rsynctest' + flow_file = tmp_flow_config(id_, """ [scheduling] initial cycle point = 2020-01-01 [[dependencies]] @@ -1004,15 +1004,15 @@ def test_rsync_includes_will_not_accept_sub_directories(tmp_flow_config): with pytest.raises(WorkflowConfigError) as exc: WorkflowConfig( - workflow=reg, fpath=flow_file, options=Values() + workflow=id_, fpath=flow_file, options=Values() ) assert "Directories can only be from the top level" in str(exc.value) def test_valid_rsync_includes_returns_correct_list(tmp_flow_config): """Test that the rsync includes in the correct """ - reg = 'rsynctest' - flow_file = tmp_flow_config(reg, """ + id_ = 'rsynctest' + flow_file = tmp_flow_config(id_, """ [scheduling] initial cycle point = 2020-01-01 [[dependencies]] @@ -1023,7 +1023,7 @@ def test_valid_rsync_includes_returns_correct_list(tmp_flow_config): """) config = WorkflowConfig( - workflow=reg, fpath=flow_file, options=Values() + workflow=id_, fpath=flow_file, options=Values() ) rsync_includes = WorkflowConfig.get_validated_rsync_includes(config) @@ -1078,8 +1078,8 @@ def test_check_circular(opt, monkeypatch, caplog, tmp_flow_config): if opt: setattr(options, opt, True) - reg = 'circular' - flow_file = tmp_flow_config(reg, """ + id_ = 'circular' + flow_file = tmp_flow_config(id_, """ [scheduling] cycling mode = integer [[graph]] @@ -1091,7 +1091,7 @@ def test_check_circular(opt, monkeypatch, caplog, tmp_flow_config): def WorkflowConfig__assert_err_raised(): with pytest.raises(WorkflowConfigError) as exc: - WorkflowConfig(workflow=reg, fpath=flow_file, options=options) + WorkflowConfig(workflow=id_, fpath=flow_file, options=options) assert "circular edges detected" in str(exc.value) # ----- The actual test ----- @@ -1110,8 +1110,8 @@ def WorkflowConfig__assert_err_raised(): def test_undefined_custom_output(tmp_flow_config: Callable): """Test error on undefined custom output referenced in graph.""" - reg = 'custom_out1' - flow_file = tmp_flow_config(reg, """ + id_ = 'custom_out1' + flow_file = tmp_flow_config(id_, """ [scheduling] [[graph]] R1 = "foo:x => bar" @@ -1120,14 +1120,14 @@ def test_undefined_custom_output(tmp_flow_config: Callable): """) with pytest.raises(WorkflowConfigError) as cm: - WorkflowConfig(workflow=reg, fpath=flow_file, options=Values()) + WorkflowConfig(workflow=id_, fpath=flow_file, options=Values()) assert "Undefined custom output" in str(cm.value) def test_invalid_custom_output_msg(tmp_flow_config: Callable): """Test invalid output message (colon not allowed).""" - reg = 'invalid_output' - flow_file = tmp_flow_config(reg, """ + id_ = 'invalid_output' + flow_file = tmp_flow_config(id_, """ [scheduling] [[graph]] R1 = "foo:x => bar" @@ -1140,7 +1140,7 @@ def test_invalid_custom_output_msg(tmp_flow_config: Callable): with pytest.raises(WorkflowConfigError) as cm: WorkflowConfig( - workflow=reg, fpath=flow_file, options=Values()) + workflow=id_, fpath=flow_file, options=Values()) assert ( 'Invalid task message "[runtime][foo][outputs]x = ' 'the quick: brown fox"' @@ -1155,8 +1155,8 @@ def test_c7_back_compat_optional_outputs(tmp_flow_config, monkeypatch): """ monkeypatch.setattr('cylc.flow.flags.cylc7_back_compat', True) - reg = 'custom_out2' - flow_file = tmp_flow_config(reg, ''' + id_ = 'custom_out2' + flow_file = tmp_flow_config(id_, ''' [scheduling] [[graph]] R1 = """ @@ -1171,7 +1171,7 @@ def test_c7_back_compat_optional_outputs(tmp_flow_config, monkeypatch): x = x ''') - cfg = WorkflowConfig(workflow=reg, fpath=flow_file, options=None) + cfg = WorkflowConfig(workflow=id_, fpath=flow_file, options=None) for taskdef in cfg.taskdefs.values(): for output, (_, required) in taskdef.outputs.items(): @@ -1191,8 +1191,8 @@ def test_c7_back_compat_optional_outputs(tmp_flow_config, monkeypatch): ) def test_implicit_success_required(tmp_flow_config, graph): """Check foo:succeed is required if success/fail not used in the graph.""" - reg = 'blargh' - flow_file = tmp_flow_config(reg, f""" + id_ = 'blargh' + flow_file = tmp_flow_config(id_, f""" [scheduling] [[graph]] R1 = {graph} @@ -1202,7 +1202,7 @@ def test_implicit_success_required(tmp_flow_config, graph): [[[outputs]]] x = "the quick brown fox" """) - cfg = WorkflowConfig(workflow=reg, fpath=flow_file, options=None) + cfg = WorkflowConfig(workflow=id_, fpath=flow_file, options=None) assert cfg.taskdefs['foo'].outputs[TASK_OUTPUT_SUCCEEDED][1] @@ -1215,8 +1215,8 @@ def test_implicit_success_required(tmp_flow_config, graph): ) def test_success_after_optional_submit(tmp_flow_config, graph): """Check foo:succeed is not required if foo:submit is optional.""" - reg = 'blargh' - flow_file = tmp_flow_config(reg, f""" + id_ = 'blargh' + flow_file = tmp_flow_config(id_, f""" [scheduling] [[graph]] R1 = {graph} @@ -1224,7 +1224,7 @@ def test_success_after_optional_submit(tmp_flow_config, graph): [[bar]] [[foo]] """) - cfg = WorkflowConfig(workflow=reg, fpath=flow_file, options=None) + cfg = WorkflowConfig(workflow=id_, fpath=flow_file, options=None) assert not cfg.taskdefs['foo'].outputs[TASK_OUTPUT_SUCCEEDED][1] @@ -1281,8 +1281,8 @@ def test_implicit_tasks( implicit tasks in the err msg. """ # Setup - reg = 'rincewind' - flow_file: Path = tmp_flow_config(reg, f""" + id_ = 'rincewind' + flow_file: Path = tmp_flow_config(id_, f""" [scheduler] { f'allow implicit tasks = {allow_implicit_tasks}' @@ -1302,7 +1302,7 @@ def test_implicit_tasks( expected_exc = WorkflowConfigError extra_msg_expected &= (allow_implicit_tasks is None) # Test - args: dict = {'workflow': reg, 'fpath': flow_file, 'options': None} + args: dict = {'workflow': id_, 'fpath': flow_file, 'options': None} expected_msg = r"implicit tasks detected.*" if expected_exc: with pytest.raises(expected_exc, match=expected_msg) as excinfo: @@ -1385,8 +1385,8 @@ def test_zero_interval( ): """Test that a zero-duration recurrence with >1 repetition gets an appropriate warning.""" - reg = 'ordinary' - flow_file: Path = tmp_flow_config(reg, f""" + id_ = 'ordinary' + flow_file: Path = tmp_flow_config(id_, f""" [scheduler] UTC mode = True allow implicit tasks = True @@ -1396,7 +1396,7 @@ def test_zero_interval( [[graph]] {recurrence} = slidescape36 """) - WorkflowConfig(reg, flow_file, options=opts) + WorkflowConfig(id_, flow_file, options=opts) logged = log_filter( caplog, level=logging.WARNING, @@ -1433,8 +1433,8 @@ def test_chain_expr( Note the order matters when "nominal" units (years, months) are used. """ - reg = 'osgiliath' - flow_file: Path = tmp_flow_config(reg, f""" + id_ = 'osgiliath' + flow_file: Path = tmp_flow_config(id_, f""" [scheduler] UTC mode = True allow implicit tasks = True @@ -1444,7 +1444,7 @@ def test_chain_expr( [[graph]] P1D = faramir """) - cfg = WorkflowConfig(reg, flow_file, options=ValidateOptions()) + cfg = WorkflowConfig(id_, flow_file, options=ValidateOptions()) assert cfg.final_point == ISO8601Point(expected_fcp).standardise() @@ -1497,8 +1497,8 @@ def test_check_for_owner(runtime_cfg): @pytest.fixture(scope='module') def awe_config(mod_tmp_flow_config: Callable) -> WorkflowConfig: """Return a workflow config object.""" - reg = 'awe' - flow_file = mod_tmp_flow_config(reg, ''' + id_ = 'awe' + flow_file = mod_tmp_flow_config(id_, ''' [scheduling] cycling mode = integer [[graph]] @@ -1512,7 +1512,7 @@ def awe_config(mod_tmp_flow_config: Callable) -> WorkflowConfig: inherit = MOON ''') return WorkflowConfig( - workflow=reg, fpath=flow_file, options=ValidateOptions() + workflow=id_, fpath=flow_file, options=ValidateOptions() ) diff --git a/tests/unit/test_install.py b/tests/unit/test_install.py index e3dc8c876da..1b8ad505d38 100644 --- a/tests/unit/test_install.py +++ b/tests/unit/test_install.py @@ -136,8 +136,8 @@ def test_install_workflow__symlink_target_exists( ): """Test that you can't install workflow when run dir symlink dir target already exists.""" - reg = 'smeagol' - src_dir: Path = tmp_src_dir(reg) + id_ = 'smeagol' + src_dir: Path = tmp_src_dir(id_) tmp_run_dir() sym_run = tmp_path / 'sym-run' sym_log = tmp_path / 'sym-log' @@ -153,13 +153,13 @@ def test_install_workflow__symlink_target_exists( ) msg = "Symlink dir target already exists: .*{}" # Test: - (sym_run / 'cylc-run' / reg / 'run1').mkdir(parents=True) + (sym_run / 'cylc-run' / id_ / 'run1').mkdir(parents=True) with pytest.raises(WorkflowFilesError, match=msg.format(sym_run)): install_workflow(src_dir) shutil.rmtree(sym_run) ( - sym_log / 'cylc-run' / reg / 'run1' / WorkflowFiles.LogDir.DIRNAME + sym_log / 'cylc-run' / id_ / 'run1' / WorkflowFiles.LogDir.DIRNAME ).mkdir(parents=True) with pytest.raises(WorkflowFilesError, match=msg.format(sym_log)): install_workflow(src_dir) diff --git a/tests/unit/test_pathutil.py b/tests/unit/test_pathutil.py index d42b26a2dbe..6ed9e9ec120 100644 --- a/tests/unit/test_pathutil.py +++ b/tests/unit/test_pathutil.py @@ -438,21 +438,21 @@ def test_remove_empty_parents(tmp_path: Path): """Test that _remove_empty_parents() doesn't remove parents containing a sibling.""" # -- Setup -- - reg = 'foo/bar/baz/qux' - path = tmp_path.joinpath(reg) + id_ = 'foo/bar/baz/qux' + path = tmp_path.joinpath(id_) tmp_path.joinpath('foo/bar/baz').mkdir(parents=True) # Note qux does not exist, but that shouldn't matter sibling_reg = 'foo/darmok' sibling_path = tmp_path.joinpath(sibling_reg) sibling_path.mkdir() # -- Test -- - remove_empty_parents(path, reg) + remove_empty_parents(path, id_) assert tmp_path.joinpath('foo/bar').exists() is False assert tmp_path.joinpath('foo').exists() is True # Check it skips non-existent dirs, and stops at the right place too tmp_path.joinpath('foo/bar').mkdir() sibling_path.rmdir() - remove_empty_parents(path, reg) + remove_empty_parents(path, id_) assert tmp_path.joinpath('foo').exists() is False assert tmp_path.exists() is True diff --git a/tests/unit/test_task_remote_mgr.py b/tests/unit/test_task_remote_mgr.py index dab97b2d7ce..c41e415eba3 100644 --- a/tests/unit/test_task_remote_mgr.py +++ b/tests/unit/test_task_remote_mgr.py @@ -43,9 +43,9 @@ def test__remote_init_items(comms_meth: CommsMeth, expected: bool): Should only includes files under .service/ """ - reg = 'barclay' - mock_mgr = Mock(workflow=reg) - srv_dir = get_workflow_srv_dir(reg) + id_ = 'barclay' + mock_mgr = Mock(workflow=id_) + srv_dir = get_workflow_srv_dir(id_) items = TaskRemoteMgr._remote_init_items(mock_mgr, comms_meth) if expected: assert items diff --git a/tests/unit/test_taskdef.py b/tests/unit/test_taskdef.py index b58bab8529f..550e01f5f15 100644 --- a/tests/unit/test_taskdef.py +++ b/tests/unit/test_taskdef.py @@ -24,9 +24,9 @@ def test_generate_graph_parents_1(tmp_flow_config): """Test that parents are only generated from valid recurrences.""" - reg = 'pan-galactic' + id_ = 'pan-galactic' flow_file = tmp_flow_config( - reg, + id_, f""" [scheduler] UTC mode = True @@ -41,7 +41,7 @@ def test_generate_graph_parents_1(tmp_flow_config): [[every_cycle, run_once_at_midnight]] """ ) - cfg = WorkflowConfig(workflow=reg, fpath=flow_file, options=None) + cfg = WorkflowConfig(workflow=id_, fpath=flow_file, options=None) # Each instance of every_cycle should have a parent only at T00. for point in [ @@ -65,9 +65,9 @@ def test_generate_graph_parents_1(tmp_flow_config): def test_generate_graph_parents_2(tmp_flow_config): """Test inferred parents are valid w.r.t to their own recurrences.""" - reg = 'gargle-blaster' + id_ = 'gargle-blaster' flow_file = tmp_flow_config( - reg, + id_, f""" [scheduling] cycling mode = integer @@ -77,7 +77,7 @@ def test_generate_graph_parents_2(tmp_flow_config): [[foo]] """ ) - cfg = WorkflowConfig(workflow=reg, fpath=flow_file, options=None) + cfg = WorkflowConfig(workflow=id_, fpath=flow_file, options=None) # Each instance of every_cycle should have a parent only at T00. parents = generate_graph_parents( diff --git a/tests/unit/test_workflow_files.py b/tests/unit/test_workflow_files.py index 90fe3327845..b2b33e495aa 100644 --- a/tests/unit/test_workflow_files.py +++ b/tests/unit/test_workflow_files.py @@ -93,7 +93,7 @@ def test_is_valid_run_dir(is_abs_path: bool, tmp_run_dir: Callable): @pytest.mark.parametrize( - 'reg, expected_err, expected_msg', + 'id_, expected_err, expected_msg', [('foo/bar/', None, None), ('/foo/bar', WorkflowFilesError, "cannot be an absolute path"), ('$HOME/alone', WorkflowFilesError, "invalid workflow name"), @@ -101,14 +101,14 @@ def test_is_valid_run_dir(is_abs_path: bool, tmp_run_dir: Callable): ('meow/..', WorkflowFilesError, "cannot be a path that points to the cylc-run directory or above")] ) -def test_validate_workflow_name(reg, expected_err, expected_msg): +def test_validate_workflow_name(id_, expected_err, expected_msg): if expected_err: with pytest.raises(expected_err) as exc: - validate_workflow_name(reg) + validate_workflow_name(id_) if expected_msg: assert expected_msg in str(exc.value) else: - validate_workflow_name(reg) + validate_workflow_name(id_) @pytest.mark.parametrize( @@ -169,7 +169,7 @@ def test_infer_latest_run( Params: path: Input arg. implicit_runN: Input arg. - expected_reg: The reg part of the expected returned tuple. + expected_reg: The id_ part of the expected returned tuple. """ # Setup cylc_run_dir: Path = tmp_run_dir() @@ -307,11 +307,11 @@ def test_get_symlink_dirs( # Setup cylc_run_dir = tmp_run_dir() create_filetree(filetree, tmp_path, tmp_path) - reg = 'foo/bar' + id_ = 'foo/bar' for k, v in expected.items(): expected[k] = Path(tmp_path / v) # Test - assert get_symlink_dirs(reg, cylc_run_dir / reg) == expected + assert get_symlink_dirs(id_, cylc_run_dir / id_) == expected @@ -484,14 +484,14 @@ def test_check_flow_file_symlink( @pytest.mark.parametrize( - 'reg, installed, named, expected', + 'id_, installed, named, expected', [('reg1/run1', True, True, True), ('reg2', True, False, True), ('reg3', False, False, False)] ) -def test_is_installed(tmp_run_dir: Callable, reg, installed, named, expected): +def test_is_installed(tmp_run_dir: Callable, id_, installed, named, expected): """Test is_installed correctly identifies presence of _cylc-install dir""" - cylc_run_dir: Path = tmp_run_dir(reg, installed=installed, named=named) + cylc_run_dir: Path = tmp_run_dir(id_, installed=installed, named=named) actual = is_installed(cylc_run_dir) assert actual == expected diff --git a/tests/unit/xtriggers/test_workflow_state.py b/tests/unit/xtriggers/test_workflow_state.py index 99a8c6d5fe0..4ea6ee53a58 100644 --- a/tests/unit/xtriggers/test_workflow_state.py +++ b/tests/unit/xtriggers/test_workflow_state.py @@ -24,8 +24,8 @@ def test_inferred_run(tmp_run_dir: Callable, monkeymock: MonkeyMock): """Test that the workflow_state xtrigger infers the run number""" - reg = 'isildur' - expected_workflow_id = f'{reg}/run1' + id_ = 'isildur' + expected_workflow_id = f'{id_}/run1' cylc_run_dir = str(tmp_run_dir()) tmp_run_dir(expected_workflow_id, installed=True, named=True) mock_db_checker = monkeymock( @@ -35,6 +35,6 @@ def test_inferred_run(tmp_run_dir: Callable, monkeymock: MonkeyMock): ) ) - _, results = workflow_state(reg, task='precious', point='3000') + _, results = workflow_state(id_, task='precious', point='3000') mock_db_checker.assert_called_once_with(cylc_run_dir, expected_workflow_id) assert results['workflow'] == expected_workflow_id From bac6c96e07c4bec57fbd33ae984f508a41f32af6 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Tue, 11 Jul 2023 11:52:58 +0100 Subject: [PATCH 19/24] terminology: workflow-name => workflow-id * Workflow name still exists as an install concept in relation to the source directory, and in the run directory where it is the workflow ID with the run name removed. --- cylc/flow/cfgspec/globalcfg.py | 30 ++++++++++++++--------------- cylc/flow/cfgspec/workflow.py | 2 +- cylc/flow/install.py | 2 +- cylc/flow/network/__init__.py | 2 +- cylc/flow/network/client_factory.py | 2 +- cylc/flow/pathutil.py | 22 ++++++++++----------- cylc/flow/scripts/clean.py | 2 +- cylc/flow/scripts/install.py | 4 ++-- cylc/flow/scripts/message.py | 2 +- cylc/flow/scripts/remote_init.py | 2 +- cylc/flow/scripts/scan.py | 2 +- cylc/flow/task_message.py | 2 +- 12 files changed, 37 insertions(+), 37 deletions(-) diff --git a/cylc/flow/cfgspec/globalcfg.py b/cylc/flow/cfgspec/globalcfg.py index c50845310db..d6d62159660 100644 --- a/cylc/flow/cfgspec/globalcfg.py +++ b/cylc/flow/cfgspec/globalcfg.py @@ -1024,11 +1024,11 @@ def default_for( Alternative location for the run dir. If specified, the workflow run directory will - be created in ``/cylc-run/`` + be created in ``/cylc-run/`` and a symbolic link will be created from - ``$HOME/cylc-run/``. + ``$HOME/cylc-run/``. If not specified the workflow run directory will be created - in ``$HOME/cylc-run/``. + in ``$HOME/cylc-run/``. All the workflow files and the ``.service`` directory get installed into this directory. @@ -1038,11 +1038,11 @@ def default_for( Alternative location for the log dir. If specified the workflow log directory will be created in - ``/cylc-run//log`` and a + ``/cylc-run//log`` and a symbolic link will be created from - ``$HOME/cylc-run//log``. If not specified + ``$HOME/cylc-run//log``. If not specified the workflow log directory will be created in - ``$HOME/cylc-run//log``. + ``$HOME/cylc-run//log``. .. versionadded:: 8.0.0 """) @@ -1050,11 +1050,11 @@ def default_for( Alternative location for the share dir. If specified the workflow share directory will be - created in ``/cylc-run//share`` + created in ``/cylc-run//share`` and a symbolic link will be created from - ``<$HOME/cylc-run//share``. If not specified + ``<$HOME/cylc-run//share``. If not specified the workflow share directory will be created in - ``$HOME/cylc-run//share``. + ``$HOME/cylc-run//share``. .. versionadded:: 8.0.0 """) @@ -1063,11 +1063,11 @@ def default_for( If specified the workflow share/cycle directory will be created in - ``/cylc-run//share/cycle`` + ``/cylc-run//share/cycle`` and a symbolic link will be created from - ``$HOME/cylc-run//share/cycle``. If not + ``$HOME/cylc-run//share/cycle``. If not specified the workflow share/cycle directory will be - created in ``$HOME/cylc-run//share/cycle``. + created in ``$HOME/cylc-run//share/cycle``. .. versionadded:: 8.0.0 """) @@ -1075,11 +1075,11 @@ def default_for( Alternative directory for the work dir. If specified the workflow work directory will be created in - ``/cylc-run//work`` and a + ``/cylc-run//work`` and a symbolic link will be created from - ``$HOME/cylc-run//work``. If not specified + ``$HOME/cylc-run//work``. If not specified the workflow work directory will be created in - ``$HOME/cylc-run//work``. + ``$HOME/cylc-run//work``. .. versionadded:: 8.0.0 """) diff --git a/cylc/flow/cfgspec/workflow.py b/cylc/flow/cfgspec/workflow.py index 77e7254c706..c78fd65f574 100644 --- a/cylc/flow/cfgspec/workflow.py +++ b/cylc/flow/cfgspec/workflow.py @@ -1530,7 +1530,7 @@ def get_script_common_text(this: str, example: Optional[str] = None): The items in this section reflect options and defaults of the ``cylc workflow-state`` command, - except that the target workflow name and the + except that the target workflow ID and the ``--task``, ``--cycle``, and ``--status`` options are taken from the graph notation. diff --git a/cylc/flow/install.py b/cylc/flow/install.py index 351b080046e..0e55ac90c8c 100644 --- a/cylc/flow/install.py +++ b/cylc/flow/install.py @@ -548,7 +548,7 @@ def detect_flow_exists( Args: run_path_base: Absolute path of workflow directory, - i.e ~/cylc-run/ + i.e ~/cylc-run/ numbered: If True, will detect if numbered runs exist. If False, will detect if non-numbered runs exist, i.e. runs installed by --run-name. diff --git a/cylc/flow/network/__init__.py b/cylc/flow/network/__init__.py index 448b35fb65d..916b129e244 100644 --- a/cylc/flow/network/__init__.py +++ b/cylc/flow/network/__init__.py @@ -67,7 +67,7 @@ def get_location(workflow: str) -> Tuple[str, int, int]: NB: if it fails to load the workflow contact file, it will exit. Args: - workflow: workflow name + workflow: workflow ID Returns: Tuple (host name, port number, publish port number) Raises: diff --git a/cylc/flow/network/client_factory.py b/cylc/flow/network/client_factory.py index 161bbcbbc82..c387f3d1699 100644 --- a/cylc/flow/network/client_factory.py +++ b/cylc/flow/network/client_factory.py @@ -46,7 +46,7 @@ def get_runtime_client( Args: comm_method: communication method - workflow: workflow name + workflow: workflow ID """ if comms_method == CommsMeth.SSH: from cylc.flow.network.ssh_client import WorkflowRuntimeClient diff --git a/cylc/flow/pathutil.py b/cylc/flow/pathutil.py index c08b638e1f4..6b105c7de04 100644 --- a/cylc/flow/pathutil.py +++ b/cylc/flow/pathutil.py @@ -56,19 +56,19 @@ def expand_path(*args: Union[Path, str]) -> str: def get_remote_workflow_run_dir( - workflow_name: Union[Path, str], *args: Union[Path, str] + workflow_id: Union[Path, str], *args: Union[Path, str] ) -> str: """Return remote workflow run directory, joining any extra args, NOT expanding vars or user.""" - return os.path.join(_CYLC_RUN_DIR, workflow_name, *args) + return os.path.join(_CYLC_RUN_DIR, workflow_id, *args) def get_remote_workflow_run_job_dir( - workflow_name: Union[Path, str], *args: Union[Path, str] + workflow_id: Union[Path, str], *args: Union[Path, str] ) -> str: """Return remote workflow job log directory, joining any extra args, NOT expanding vars or user.""" - return get_remote_workflow_run_dir(workflow_name, 'log', 'job', *args) + return get_remote_workflow_run_dir(workflow_id, 'log', 'job', *args) def get_cylc_run_dir() -> str: @@ -77,14 +77,14 @@ def get_cylc_run_dir() -> str: def get_workflow_run_dir( - workflow_name: Union[Path, str], *args: Union[Path, str] + workflow_id: Union[Path, str], *args: Union[Path, str] ) -> str: """Return local workflow run directory, joining any extra args, and expanding vars and user. Does not check that the directory exists. """ - return expand_path(_CYLC_RUN_DIR, workflow_name, *args) + return expand_path(_CYLC_RUN_DIR, workflow_id, *args) def get_workflow_run_job_dir(workflow, *args): @@ -157,7 +157,7 @@ def make_localhost_symlinks( """Creates symlinks for any configured symlink dirs from glbl_cfg. Args: rund: the entire run directory path - named_sub_dir: e.g workflow_name/run1 + named_sub_dir: e.g workflow_id/run1 symlink_conf: Symlinks dirs configuration passed from cli Returns: @@ -195,7 +195,7 @@ def make_localhost_symlinks( def get_dirs_to_symlink( install_target: str, - workflow_name: str, + workflow_id: str, symlink_conf: Optional[Dict[str, Dict[str, Any]]] = None ) -> Dict[str, str]: """Returns dictionary of directories to symlink. @@ -204,7 +204,7 @@ def get_dirs_to_symlink( Args: install_target: Symlinks to be created on this install target - flow_name: full name of the run, e.g. myflow/run1 + flow_id: full id of the run, e.g. myflow/run1 symlink_conf: Symlink dirs, if sent on the cli. Defaults to None, in which case global config symlink dirs will be applied. @@ -220,13 +220,13 @@ def get_dirs_to_symlink( base_dir = symlink_conf[install_target]['run'] if base_dir: dirs_to_symlink['run'] = os.path.join( - base_dir, 'cylc-run', workflow_name) + base_dir, 'cylc-run', workflow_id) for dir_ in ['log', 'share', 'share/cycle', 'work']: link = symlink_conf[install_target].get(dir_, None) if (not link) or link == base_dir: continue dirs_to_symlink[dir_] = os.path.join( - link, 'cylc-run', workflow_name, dir_) + link, 'cylc-run', workflow_id, dir_) return dirs_to_symlink diff --git a/cylc/flow/scripts/clean.py b/cylc/flow/scripts/clean.py index 368c1094cba..c2d2ee3032a 100644 --- a/cylc/flow/scripts/clean.py +++ b/cylc/flow/scripts/clean.py @@ -29,7 +29,7 @@ It will also remove any symlink directory targets. -Workflow names can be hierarchical, corresponding to the path under ~/cylc-run. +Workflow IDs can be hierarchical, corresponding to the path under ~/cylc-run. Examples: # Remove the workflow at ~/cylc-run/foo/bar diff --git a/cylc/flow/scripts/install.py b/cylc/flow/scripts/install.py index 1d5ba236037..df03be41e20 100755 --- a/cylc/flow/scripts/install.py +++ b/cylc/flow/scripts/install.py @@ -27,13 +27,13 @@ If a SOURCE_NAME is supplied, Cylc will search for the workflow source in the list of directories given by "global.cylc[install]source dirs", and install -the first match. The installed workflow name will be the same as SOURCE_NAME, +the first match. The installed workflow ID will be the same as SOURCE_NAME, unless --workflow-name is used. If a PATH is supplied, Cylc will install the workflow from the source directory given by the path. Relative paths must start with "./" to avoid ambiguity with SOURCE_NAME (i.e. "foo/bar" will be interpreted as a source name, whereas -"./foo/bar" will be interpreted as a path). The installed workflow name will +"./foo/bar" will be interpreted as a path). The installed workflow ID will be the basename of the path, unless --workflow-name is used. If no argument is supplied, Cylc will install the workflow from the source diff --git a/cylc/flow/scripts/message.py b/cylc/flow/scripts/message.py index b2fb01f431c..8e16eebe776 100755 --- a/cylc/flow/scripts/message.py +++ b/cylc/flow/scripts/message.py @@ -78,7 +78,7 @@ For backward compatibility, if number of arguments is less than or equal to 2, the command assumes the classic interface, where all arguments are messages. -Otherwise, the first 2 arguments are assumed to be workflow name and job +Otherwise, the first 2 arguments are assumed to be workflow ID and job identifier. """ diff --git a/cylc/flow/scripts/remote_init.py b/cylc/flow/scripts/remote_init.py index 7706cecfa8f..8af5ed42e2f 100755 --- a/cylc/flow/scripts/remote_init.py +++ b/cylc/flow/scripts/remote_init.py @@ -21,7 +21,7 @@ Initialise an install target. Initialisation creates a workflow run directory on the install target, -"$HOME/cylc-run//". The .service directory is also created and +"$HOME/cylc-run//". The .service directory is also created and populated with the install target authentication files and the contact file. Symlinks are created for run, work, share, share/cycle, log directories, diff --git a/cylc/flow/scripts/scan.py b/cylc/flow/scripts/scan.py index 73f6eb199d7..66e7e76a86d 100644 --- a/cylc/flow/scripts/scan.py +++ b/cylc/flow/scripts/scan.py @@ -210,7 +210,7 @@ def get_option_parser() -> COP: '--format', '-t', help=( 'Output data and format (default "plain").' - ' ("name": list the workflow names only)' + ' ("name": list the workflow IDs only)' ' ("plain": name,host:port,PID on one line)' ' ("tree": name,host:port,PID in tree format)' ' ("json": full contact data in JSON format)' diff --git a/cylc/flow/task_message.py b/cylc/flow/task_message.py index eb50d66d1ac..3ea753e7335 100644 --- a/cylc/flow/task_message.py +++ b/cylc/flow/task_message.py @@ -77,7 +77,7 @@ def record_messages(workflow: str, job_id: str, messages: List[list]) -> None: Send the messages to the workflow, if possible. Arguments: - workflow: Workflow name. + workflow: Workflow ID. job_id: Job identifier "CYCLE/TASK_NAME/SUBMIT_NUM". messages: List of messages "[[severity, message], ...]". """ From c1da8e5098f4f4d7d285b6d6dfc2664cc4634c53 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Tue, 11 Jul 2023 11:59:17 +0100 Subject: [PATCH 20/24] config: fill small coverage gap --- cylc/flow/config.py | 8 +++++--- tests/unit/test_config.py | 10 ++++++++++ 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/cylc/flow/config.py b/cylc/flow/config.py index 2b94b699127..79d305da57a 100644 --- a/cylc/flow/config.py +++ b/cylc/flow/config.py @@ -59,11 +59,11 @@ from cylc.flow.cycling.iso8601 import ingest_time, ISO8601Interval from cylc.flow.exceptions import ( CylcError, - WorkflowConfigError, + InputError, IntervalParsingError, - TaskDefError, ParamExpandError, - InputError + TaskDefError, + WorkflowConfigError, ) import cylc.flow.flags from cylc.flow.graph_parser import GraphParser @@ -209,6 +209,8 @@ def dequote(string): 'foo' >>> dequote('"f') '"f' + >>> dequote('f') + 'f' """ if len(string) < 2: diff --git a/tests/unit/test_config.py b/tests/unit/test_config.py index 4e9f579c937..736bd34ed9d 100644 --- a/tests/unit/test_config.py +++ b/tests/unit/test_config.py @@ -341,6 +341,16 @@ def test_family_inheritance_and_quotes( (WorkflowConfigError, "does not meet the constraints"), id="Violated constraints" ), + pytest.param( + ISO8601_CYCLING_TYPE, + { + 'initial cycle point': 'a', + }, + None, + None, + (WorkflowConfigError, 'Invalid ISO 8601 date representation: a'), + id="invalid" + ), ] ) def test_process_icp( From 258ad0fd28ded5fff9c7e9fb34968a012d629964 Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Tue, 4 Jul 2023 10:27:01 +0100 Subject: [PATCH 21/24] pool: prevent no-flow tasks from spawning downstreams in compat mode * Closes #5613 --- CHANGES.md | 4 ++ cylc/flow/task_pool.py | 12 ++--- tests/integration/test_task_pool.py | 70 +++++++++++++++++++++++++++-- 3 files changed, 78 insertions(+), 8 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 9ef9abe91e4..3e877de6c35 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -85,6 +85,10 @@ in a timely fashion when tasks completed. Task outputs and messages are now validated to avoid conflicts with built-in outputs, messages, qualifiers and Cylc keywords. +[#5614](https://github.com/cylc/cylc-flow/pull/5614) - +Fix a bug in Cylc 7 compatibility mode where tasks running in the `none` flow +(e.g. via `cylc trigger --flow=none`) would trigger downstream tasks. + [#5604](https://github.com/cylc/cylc-flow/pull/5604) - Fix a possible issue where workflows started using `cylc play --start-cycle-point` could hang during startup. diff --git a/cylc/flow/task_pool.py b/cylc/flow/task_pool.py index a129c2b7a5a..55fffad7fae 100644 --- a/cylc/flow/task_pool.py +++ b/cylc/flow/task_pool.py @@ -1270,11 +1270,11 @@ def spawn_on_output(self, itask, output, forced=False): and itask.identity not in self.expected_failed_tasks ): self.abort_task_failed = True - try: - children = itask.graph_children[output] - except KeyError: - # No children depend on this output - children = [] + + children = [] + if itask.flow_nums or forced: + with suppress(KeyError): + children = itask.graph_children[output] suicide = [] for c_name, c_point, is_abs in children: @@ -1404,6 +1404,8 @@ def spawn_on_all_outputs( associated prerequisites of spawned children to satisifed. """ + if not itask.flow_nums: + return if completed_only: outputs = itask.state.outputs.get_completed() else: diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index 39c12029f1d..9064809d026 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -14,17 +14,18 @@ # You should have received a copy of the GNU General Public License # along with this program. If not, see . -from cylc.flow import CYLC_LOG from copy import deepcopy import logging +from typing import AsyncGenerator, Callable, Iterable, List, Tuple, Union + import pytest from pytest import param -from typing import AsyncGenerator, Callable, Iterable, List, Tuple, Union +from cylc.flow import CYLC_LOG from cylc.flow.cycling import PointBase from cylc.flow.cycling.integer import IntegerPoint -from cylc.flow.exceptions import PlatformLookupError from cylc.flow.data_store_mgr import TASK_PROXIES +from cylc.flow.task_outputs import TASK_OUTPUT_SUCCEEDED from cylc.flow.scheduler import Scheduler from cylc.flow.flow_mgr import FLOW_ALL from cylc.flow.task_state import ( @@ -1051,3 +1052,66 @@ async def test_db_update_on_removal( # the task should be gone from the DB assert list_pool_from_db(schd) == [] + + +async def test_no_flow_tasks_dont_spawn( + flow, + scheduler, + start, +): + """Ensure no-flow tasks don't spawn downstreams. + + No-flow tasks (i.e `--flow=none`) are one-offs which are not attached to + any "flow". + + See https://github.com/cylc/cylc-flow/issues/5613 + """ + id_ = flow({ + 'scheduling': { + 'graph': { + 'R1': 'a => b => c' + } + }, + 'scheduler': { + 'allow implicit tasks': 'true', + }, + }) + + schd = scheduler(id_) + async with start(schd): + # mark task 1/a as succeeded + task_a = schd.pool.get_tasks()[0] + task_a.state_reset(TASK_OUTPUT_SUCCEEDED) + + for flow_nums, force, pool in ( + # outputs yielded from a no-flow task should not spawn downstreams + (set(), False, []), + # forced spawning downstream of a no-flow task should spawn + # downstreams with flow_nums={} + (set(), True, [('1/b', set())]), + # outputs yielded from a task with flow numbers should spawn + # downstreams in the same flow + ({1}, False, [('1/b', {1})]), + # forced spawning should work in the same way + ({1}, True, [('1/b', {1})]), + ): + # set the flow-nums on 1/a + task_a.flow_nums = flow_nums + + # spawn on the succeeded output + schd.pool.spawn_on_output( + task_a, + TASK_OUTPUT_SUCCEEDED, + forced=force, + ) + schd.pool.spawn_on_all_outputs(task_a) + + # ensure the pool is as expected + assert [ + (itask.identity, itask.flow_nums) + for pool in [ + schd.pool.get_tasks(), + schd.pool.get_hidden_tasks(), + ] + for itask in pool + ] == pool From 80ef1ca2a837c832231f45342b9bfb22586d4310 Mon Sep 17 00:00:00 2001 From: David Sutherland Date: Wed, 12 Jul 2023 21:24:06 +1200 Subject: [PATCH 22/24] blacklist setuptools v67 (#5625) --- CHANGES.md | 3 +++ conda-environment.yml | 2 +- setup.cfg | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index bcc2f3be888..fa943c62867 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -107,6 +107,9 @@ xtrigger function signatures. Fix a bug when rapidly issuing the same/opposite commands e.g. pausing & resuming a workflow. +[#5625](https://github.com/cylc/cylc-flow/pull/5625) - Exclude `setuptools` +version (v67) which results in dependency check failure with editable installs. + ## __cylc-8.1.4 (Released 2023-05-04)__ ### Fixes diff --git a/conda-environment.yml b/conda-environment.yml index 4942f35aeed..11dd58f7521 100644 --- a/conda-environment.yml +++ b/conda-environment.yml @@ -15,7 +15,7 @@ dependencies: - psutil >=5.6.0 - python - pyzmq >=22 - - setuptools >=49 + - setuptools >=49,!=67.* - importlib_metadata # [py<3.8] - urwid >=2,<3 - tomli >=2 # [py<3.11] diff --git a/setup.cfg b/setup.cfg index d8ef91ee7f0..cb5244a9509 100644 --- a/setup.cfg +++ b/setup.cfg @@ -73,7 +73,7 @@ install_requires = psutil>=5.6.0 pyzmq>=22 # https://github.com/pypa/setuptools/issues/3802 - setuptools>=49 + setuptools>=49,!=67.* importlib_metadata; python_version < "3.8" urwid==2.* # unpinned transient dependencies used for type checking From 79d7d2f568156e27f8b0f37350bceda28efd6f65 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Jul 2023 09:32:11 +0000 Subject: [PATCH 23/24] Bump pypa/gh-action-pypi-publish from 1.8.7 to 1.8.8 Bumps [pypa/gh-action-pypi-publish](https://github.com/pypa/gh-action-pypi-publish) from 1.8.7 to 1.8.8. - [Release notes](https://github.com/pypa/gh-action-pypi-publish/releases) - [Commits](https://github.com/pypa/gh-action-pypi-publish/compare/v1.8.7...v1.8.8) --- updated-dependencies: - dependency-name: pypa/gh-action-pypi-publish dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- .github/workflows/2_auto_publish_release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/2_auto_publish_release.yml b/.github/workflows/2_auto_publish_release.yml index d3886eedddf..99aac4991d6 100644 --- a/.github/workflows/2_auto_publish_release.yml +++ b/.github/workflows/2_auto_publish_release.yml @@ -38,7 +38,7 @@ jobs: uses: cylc/release-actions/build-python-package@v1 - name: Publish distribution to PyPI - uses: pypa/gh-action-pypi-publish@v1.8.7 + uses: pypa/gh-action-pypi-publish@v1.8.8 with: user: __token__ # uses the API token feature of PyPI - least permissions possible password: ${{ secrets.PYPI_TOKEN }} From d7fc524c87b80c8175c31ede570aba1a7c96b2bd Mon Sep 17 00:00:00 2001 From: Oliver Sanders Date: Thu, 13 Jul 2023 16:19:35 +0100 Subject: [PATCH 24/24] Apply suggestions from code review Co-authored-by: Ronnie Dutta <61982285+MetRonnie@users.noreply.github.com> --- cylc/flow/install.py | 8 ++++---- cylc/flow/pathutil.py | 4 ++-- cylc/flow/scripts/install.py | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/cylc/flow/install.py b/cylc/flow/install.py index 0e55ac90c8c..27810f72e97 100644 --- a/cylc/flow/install.py +++ b/cylc/flow/install.py @@ -547,8 +547,8 @@ def detect_flow_exists( """Returns True if installed flow already exists. Args: - run_path_base: Absolute path of workflow directory, - i.e ~/cylc-run/ + run_path_base: Absolute path of the parent of the workflow's run dir, + i.e ~/cylc-run/ numbered: If True, will detect if numbered runs exist. If False, will detect if non-numbered runs exist, i.e. runs installed by --run-name. @@ -583,8 +583,8 @@ def check_nested_dirs( install dirs. Raises: - WorkflowFilesError if id_ dir is nested inside a run dir, or an - install dirs are nested. + WorkflowFilesError if run_dir is nested inside an existing run dir, + or install dirs are nested. """ if install_dir is not None: install_dir = Path(os.path.normpath(install_dir)) diff --git a/cylc/flow/pathutil.py b/cylc/flow/pathutil.py index 6b105c7de04..7d220e43145 100644 --- a/cylc/flow/pathutil.py +++ b/cylc/flow/pathutil.py @@ -157,7 +157,7 @@ def make_localhost_symlinks( """Creates symlinks for any configured symlink dirs from glbl_cfg. Args: rund: the entire run directory path - named_sub_dir: e.g workflow_id/run1 + named_sub_dir: e.g workflow_name/run1 symlink_conf: Symlinks dirs configuration passed from cli Returns: @@ -204,7 +204,7 @@ def get_dirs_to_symlink( Args: install_target: Symlinks to be created on this install target - flow_id: full id of the run, e.g. myflow/run1 + workflow_id: full id of the run, e.g. myflow/run1 symlink_conf: Symlink dirs, if sent on the cli. Defaults to None, in which case global config symlink dirs will be applied. diff --git a/cylc/flow/scripts/install.py b/cylc/flow/scripts/install.py index df03be41e20..45010e4c714 100755 --- a/cylc/flow/scripts/install.py +++ b/cylc/flow/scripts/install.py @@ -27,14 +27,14 @@ If a SOURCE_NAME is supplied, Cylc will search for the workflow source in the list of directories given by "global.cylc[install]source dirs", and install -the first match. The installed workflow ID will be the same as SOURCE_NAME, +the first match. The installed workflow ID will start with SOURCE_NAME, unless --workflow-name is used. If a PATH is supplied, Cylc will install the workflow from the source directory given by the path. Relative paths must start with "./" to avoid ambiguity with SOURCE_NAME (i.e. "foo/bar" will be interpreted as a source name, whereas "./foo/bar" will be interpreted as a path). The installed workflow ID will -be the basename of the path, unless --workflow-name is used. +start with the basename of the path, unless --workflow-name is used. If no argument is supplied, Cylc will install the workflow from the source in the current working directory.