diff --git a/changes.d/7024.feat.md b/changes.d/7024.feat.md new file mode 100644 index 0000000000..78f790b3ad --- /dev/null +++ b/changes.d/7024.feat.md @@ -0,0 +1 @@ +Ensure `next()/previous()` syntax works for the `cylc play --startcp` option. diff --git a/changes.d/7048.fix.md b/changes.d/7048.fix.md new file mode 100644 index 0000000000..0d7c6fc025 --- /dev/null +++ b/changes.d/7048.fix.md @@ -0,0 +1 @@ +Fixed `initial cycle point = now` changing on reload/restart. diff --git a/cylc/flow/commands.py b/cylc/flow/commands.py index 01982f233d..46f374d64b 100644 --- a/cylc/flow/commands.py +++ b/cylc/flow/commands.py @@ -568,6 +568,10 @@ async def reload_workflow(schd: 'Scheduler', reload_global: bool = False): schd.reload_pending = 'loading the workflow definition' schd.update_data_store() # update workflow status msg schd._update_workflow_state() + # Things that can't change on workflow reload: + schd._set_workflow_params( + schd.workflow_db_mgr.pri_dao.select_workflow_params() + ) LOG.info("Reloading the workflow definition.") config = schd.load_flow_file(is_reload=True) except (ParsecError, CylcConfigError) as exc: @@ -589,10 +593,7 @@ async def reload_workflow(schd: 'Scheduler', reload_global: bool = False): else: schd.reload_pending = 'applying the new config' old_tasks = set(schd.config.get_task_name_list()) - # Things that can't change on workflow reload: - schd._set_workflow_params( - schd.workflow_db_mgr.pri_dao.select_workflow_params() - ) + schd.apply_new_config(config, is_reload=True) schd.broadcast_mgr.linearized_ancestors = ( schd.config.get_linearized_ancestors() diff --git a/cylc/flow/config.py b/cylc/flow/config.py index df5d7026a8..7a46e2245a 100644 --- a/cylc/flow/config.py +++ b/cylc/flow/config.py @@ -42,6 +42,7 @@ Dict, Iterable, List, + Literal, Mapping, Optional, Set, @@ -249,6 +250,17 @@ def interpolate_template(tmpl, params_dict): raise ParamExpandError('bad template syntax') from None +def _parse_iso_cycle_point(value: str) -> str: + """Helper for parsing initial/start cycle point option in + datetime cycling mode.""" + if value == 'now': + return get_current_time_string() + try: + return ingest_time(value, get_current_time_string()) + except IsodatetimeError as exc: + raise WorkflowConfigError(str(exc)) from None + + class WorkflowConfig: """Class for workflow configuration items and derived quantities.""" @@ -367,7 +379,8 @@ def __init__( raise WorkflowConfigError("missing [scheduling][[graph]] section.") # (The check that 'graph' is defined is below). - # Override the workflow defn with an initial point from the CLI. + # Override the workflow defn with an initial point from the CLI + # or from reload/restart: icp_str = getattr(self.options, 'icp', None) if icp_str is not None: self.cfg['scheduling']['initial cycle point'] = icp_str @@ -468,7 +481,9 @@ def __init__( # after the call to init_cyclers, we can start getting proper points. init_cyclers(self.cfg) - self.cycling_type = get_interval_cls().get_null().TYPE + self.cycling_type: Literal['integer', 'iso8601'] = ( + get_interval_cls().get_null().TYPE + ) self.cycle_point_dump_format = get_dump_format(self.cycling_type) # Initial point from workflow definition (or CLI override above). @@ -550,7 +565,7 @@ def __init__( self._upg_wflow_event_names() self.mem_log("config.py: before load_graph()") - self.load_graph() + self._load_graph() self.mem_log("config.py: after load_graph()") self._set_completion_expressions() @@ -670,10 +685,10 @@ def prelim_process_graph(self) -> None: all(item in ['graph', '1', 'R1'] for item in graphdict) ): # Pure acyclic graph, assume integer cycling mode with '1' cycle - self.cfg['scheduling']['cycling mode'] = INTEGER_CYCLING_TYPE for key in ('initial cycle point', 'final cycle point'): if key not in self.cfg['scheduling']: self.cfg['scheduling'][key] = '1' + self.cfg['scheduling']['cycling mode'] = INTEGER_CYCLING_TYPE def process_utc_mode(self): """Set UTC mode from config or from stored value on restart. @@ -747,7 +762,6 @@ def process_initial_cycle_point(self) -> None: Sets: self.initial_point self.cfg['scheduling']['initial cycle point'] - self.evaluated_icp Raises: WorkflowConfigError - if it fails to validate """ @@ -756,22 +770,11 @@ def process_initial_cycle_point(self) -> None: if orig_icp is None: orig_icp = '1' icp = orig_icp - elif self.cycling_type == ISO8601_CYCLING_TYPE: + else: if orig_icp is None: raise WorkflowConfigError( "This workflow requires an initial cycle point.") - if orig_icp == "now": - icp = get_current_time_string() - else: - try: - icp = ingest_time(orig_icp, get_current_time_string()) - except IsodatetimeError as exc: - raise WorkflowConfigError(str(exc)) from None - self.evaluated_icp = None - if icp != orig_icp: - # now/next()/previous() was used, need to store - # evaluated point in DB - self.evaluated_icp = icp + icp = _parse_iso_cycle_point(orig_icp) self.initial_point = get_point(icp).standardise() self.cfg['scheduling']['initial cycle point'] = str(self.initial_point) @@ -807,8 +810,7 @@ def process_start_cycle_point(self) -> None: ) if startcp: # Start from a point later than initial point. - if self.options.startcp == 'now': - self.options.startcp = get_current_time_string() + self.options.startcp = _parse_iso_cycle_point(startcp) self.start_point = get_point(self.options.startcp).standardise() elif starttask: # Start from designated task(s). @@ -2294,7 +2296,7 @@ def _close_families(l_id, r_id, clf_map): return lret, rret - def load_graph(self): + def _load_graph(self): """Parse and load dependency graph.""" LOG.debug("Parsing the dependency graph") @@ -2318,18 +2320,14 @@ def load_graph(self): section = get_sequence_cls().get_async_expr() graphdict[section] = graphdict.pop('graph') - icp = self.cfg['scheduling']['initial cycle point'] + icp = str(self.initial_point) fcp = self.cfg['scheduling']['final cycle point'] # Make a stack of sections and graphs [(sec1, graph1), ...] sections = [] for section, value in self.cfg['scheduling']['graph'].items(): # Substitute initial and final cycle points. - if icp: - section = section.replace("^", icp) - elif "^" in section: - raise WorkflowConfigError("Initial cycle point referenced" - " (^) but not defined.") + section = section.replace("^", icp) if fcp: section = section.replace("$", fcp) elif "$" in section: diff --git a/cylc/flow/cycling/integer.py b/cylc/flow/cycling/integer.py index ce81ba55a9..125afbd36a 100644 --- a/cylc/flow/cycling/integer.py +++ b/cylc/flow/cycling/integer.py @@ -594,7 +594,7 @@ def init_from_cfg(_): pass -def get_dump_format(cycling_type=None): +def get_dump_format() -> None: """Return cycle point string dump format.""" # Not used for integer cycling. return None diff --git a/cylc/flow/cycling/iso8601.py b/cylc/flow/cycling/iso8601.py index 832f18b794..75842c97eb 100644 --- a/cylc/flow/cycling/iso8601.py +++ b/cylc/flow/cycling/iso8601.py @@ -649,6 +649,9 @@ def __lt__(self, other): def __str__(self): return self.value + def __repr__(self) -> str: + return f"<{type(self).__name__} {self.value}>" + def __hash__(self) -> int: return hash(self.value) @@ -902,7 +905,7 @@ def init(num_expanded_year_digits=0, custom_dump_format=None, time_zone=None, return WorkflowSpecifics -def get_dump_format(): +def get_dump_format() -> str: """Return cycle point string dump format.""" return WorkflowSpecifics.DUMP_FORMAT diff --git a/cylc/flow/cycling/loader.py b/cylc/flow/cycling/loader.py index 11b86f0819..deb57befaf 100644 --- a/cylc/flow/cycling/loader.py +++ b/cylc/flow/cycling/loader.py @@ -19,11 +19,21 @@ Each task may have multiple sequences, e.g. 12-hourly and 6-hourly. """ -from typing import Optional, Type, overload +from typing import ( + Literal, + Optional, + Type, + overload, +) -from cylc.flow.cycling import PointBase, integer, iso8601 from metomi.isodatetime.data import Calendar +from cylc.flow.cycling import ( + PointBase, + integer, + iso8601, +) + ISO8601_CYCLING_TYPE = iso8601.CYCLER_TYPE_ISO8601 INTEGER_CYCLING_TYPE = integer.CYCLER_TYPE_INTEGER @@ -88,8 +98,18 @@ def get_point_cls(cycling_type: Optional[str] = None) -> Type[PointBase]: return POINTS[cycling_type] -def get_dump_format(cycling_type=None): - """Return cycle point dump format, or None.""" +@overload +def get_dump_format(cycling_type: Literal["integer"]) -> None: + ... + + +@overload +def get_dump_format(cycling_type: Literal["iso8601"]) -> str: + ... + + +def get_dump_format(cycling_type: Literal["integer", "iso8601"]) -> str | None: + """Return cycle point dump format (None for integer mode).""" return DUMP_FORMAT_GETTERS[cycling_type]() diff --git a/cylc/flow/option_parsers.py b/cylc/flow/option_parsers.py index 95b6313b21..e460e6cfc7 100644 --- a/cylc/flow/option_parsers.py +++ b/cylc/flow/option_parsers.py @@ -123,10 +123,11 @@ def _update_sources(self, other): ICP_OPTION = OptionSettings( ["--initial-cycle-point", "--icp"], help=( - "Set the initial cycle point." - " Required if not defined in flow.cylc." - "\nMay be either an absolute point or an offset: See" - f" {SHORTLINK_TO_ICP_DOCS} (Cylc documentation link)." + "Set the initial cycle point. " + "Required if not defined in flow.cylc.\n" + "Can be an absolute point or an offset relative to the " + "current time - see " + f"{SHORTLINK_TO_ICP_DOCS} (Cylc documentation link)." ), metavar="CYCLE_POINT or OFFSET", action='store', diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index ea690ff9ea..6606b7ae07 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -1228,7 +1228,7 @@ def apply_new_config(self, config, is_reload=False): }) def _set_workflow_params( - self, params: Iterable[Tuple[str, Optional[str]]] + self, params: Iterable[tuple[str, str | None]] ) -> None: """Set workflow params on restart/reload. @@ -1240,20 +1240,20 @@ def _set_workflow_params( * A flag to indicate if the workflow should be paused or not. * Original workflow run time zone. """ - LOG.info('LOADING workflow parameters') + LOG.info("LOADING saved workflow parameters") for key, value in params: if key == self.workflow_db_mgr.KEY_RUN_MODE: self.options.run_mode = value or RunMode.LIVE.value LOG.info(f"+ run mode = {value}") if value is None: continue - if key in self.workflow_db_mgr.KEY_INITIAL_CYCLE_POINT_COMPATS: + if key == self.workflow_db_mgr.KEY_INITIAL_CYCLE_POINT: self.options.icp = value LOG.info(f"+ initial point = {value}") - elif key in self.workflow_db_mgr.KEY_START_CYCLE_POINT_COMPATS: + elif key == self.workflow_db_mgr.KEY_START_CYCLE_POINT: self.options.startcp = value LOG.info(f"+ start point = {value}") - elif key in self.workflow_db_mgr.KEY_FINAL_CYCLE_POINT_COMPATS: + elif key == self.workflow_db_mgr.KEY_FINAL_CYCLE_POINT: if self.is_restart and self.options.fcp == 'reload': LOG.debug(f"- final point = {value} (ignored)") elif self.options.fcp is None: diff --git a/cylc/flow/scheduler_cli.py b/cylc/flow/scheduler_cli.py index af55f66d2e..8c08d7e496 100644 --- a/cylc/flow/scheduler_cli.py +++ b/cylc/flow/scheduler_cli.py @@ -15,7 +15,6 @@ # along with this program. If not, see . """Common logic for "cylc play" CLI.""" -from ansimarkup import parse as cparse import asyncio from copy import deepcopy from functools import lru_cache @@ -24,50 +23,62 @@ from pathlib import Path from shlex import quote import sys -from typing import TYPE_CHECKING, Tuple +from typing import ( + TYPE_CHECKING, + Tuple, +) +from ansimarkup import parse as cparse from packaging.version import Version -from cylc.flow import LOG, __version__ +from cylc.flow import ( + LOG, + __version__, +) from cylc.flow.exceptions import ( CylcError, ServiceFileError, WorkflowStopped, ) -from cylc.flow.scripts.ping import run as cylc_ping import cylc.flow.flags -from cylc.flow.id import upgrade_legacy_ids from cylc.flow.host_select import select_workflow_host from cylc.flow.hostuserutil import is_remote_host +from cylc.flow.id import upgrade_legacy_ids from cylc.flow.id_cli import parse_ids_async from cylc.flow.loggingutil import ( - close_log, RotatingLogFileHandler, + close_log, ) from cylc.flow.network.client import WorkflowRuntimeClient from cylc.flow.network.log_stream_handler import ProtobufStreamHandler from cylc.flow.option_parsers import ( + ICP_OPTION, + SHORTLINK_TO_ICP_DOCS, WORKFLOW_ID_ARG_DOC, CylcOptionParser as COP, - OptionSettings, Options, - ICP_OPTION, + OptionSettings, ) from cylc.flow.pathutil import get_workflow_run_scheduler_log_path from cylc.flow.remote import cylc_server_cmd -from cylc.flow.scheduler import Scheduler, SchedulerError -from cylc.flow.scripts.common import cylc_header from cylc.flow.run_modes import WORKFLOW_RUN_MODES -from cylc.flow.workflow_db_mgr import WorkflowDatabaseManager -from cylc.flow.workflow_files import ( - SUITERC_DEPR_MSG, - get_workflow_srv_dir, +from cylc.flow.scheduler import ( + Scheduler, + SchedulerError, ) +from cylc.flow.scripts.common import cylc_header +from cylc.flow.scripts.ping import run as cylc_ping from cylc.flow.terminal import ( cli_function, is_terminal, prompt, ) +from cylc.flow.workflow_db_mgr import WorkflowDatabaseManager +from cylc.flow.workflow_files import ( + SUITERC_DEPR_MSG, + get_workflow_srv_dir, +) + if TYPE_CHECKING: from optparse import Values @@ -162,10 +173,14 @@ OptionSettings( ["--start-cycle-point", "--startcp"], help=( - "Set the start cycle point, which may be after" - " the initial cycle point. If the specified start point is" - " not in the sequence, the next on-sequence point will" - " be used. (Not to be confused with the initial cycle point)"), + "Set the start cycle point, which may be after " + "the initial cycle point. " + "If the specified start point is not in the sequence, the next " + "on-sequence point will be used.\n" + "Can be an absolute point or an offset relative to the " + "current time, like the --initial-cycle-point option - see " + f"{SHORTLINK_TO_ICP_DOCS} (Cylc documentation link)." + ), metavar="CYCLE_POINT", action='store', dest="startcp", diff --git a/cylc/flow/workflow_db_mgr.py b/cylc/flow/workflow_db_mgr.py index 5bd9889a22..f7e8df8d8c 100644 --- a/cylc/flow/workflow_db_mgr.py +++ b/cylc/flow/workflow_db_mgr.py @@ -95,13 +95,8 @@ class WorkflowDatabaseManager: """Manage the workflow runtime private and public databases.""" KEY_INITIAL_CYCLE_POINT = 'icp' - KEY_INITIAL_CYCLE_POINT_COMPATS = ( - KEY_INITIAL_CYCLE_POINT, 'initial_point') KEY_START_CYCLE_POINT = 'startcp' - KEY_START_CYCLE_POINT_COMPATS = ( - KEY_START_CYCLE_POINT, 'start_point') KEY_FINAL_CYCLE_POINT = 'fcp' - KEY_FINAL_CYCLE_POINT_COMPATS = (KEY_FINAL_CYCLE_POINT, 'final_point') KEY_STOP_CYCLE_POINT = 'stopcp' KEY_UUID_STR = 'uuid_str' KEY_CYLC_VERSION = 'cylc_version' @@ -337,7 +332,7 @@ def put_workflow_params(self, schd: 'Scheduler') -> None: This method queues the relevant insert statements. Arguments: - schd (cylc.flow.scheduler.Scheduler): scheduler object. + schd: scheduler object. """ self.db_deletes_map[self.TABLE_WORKFLOW_PARAMS].append({}) self.db_inserts_map[self.TABLE_WORKFLOW_PARAMS].extend([ @@ -353,11 +348,8 @@ def put_workflow_params(self, schd: 'Scheduler') -> None: ]) # Store raw initial cycle point in the DB. - value = schd.config.evaluated_icp - value = None if value == 'reload' else value self.put_workflow_params_1( - self.KEY_INITIAL_CYCLE_POINT, - value or str(schd.config.initial_point) + self.KEY_INITIAL_CYCLE_POINT, str(schd.config.initial_point) ) for key in ( diff --git a/tests/functional/cylc-combination-scripts/09-vr-icp-now.t b/tests/functional/cylc-combination-scripts/09-vr-icp-now.t index 932e735fff..6874d3925d 100644 --- a/tests/functional/cylc-combination-scripts/09-vr-icp-now.t +++ b/tests/functional/cylc-combination-scripts/09-vr-icp-now.t @@ -26,14 +26,12 @@ WORKFLOW_ID=$(workflow_id) cp -r "${TEST_SOURCE_DIR}/${TEST_NAME_BASE}/flow.cylc" . -run_ok "${TEST_NAME_BASE}-vip" \ - cylc vip . \ - --workflow-name "${WORKFLOW_ID}" \ - --no-detach \ - --no-run-name +run_ok "${TEST_NAME_BASE}-vip" cylc vip . \ + --workflow-name "${WORKFLOW_ID}" \ + --no-detach \ + --no-run-name \ + --stop-cycle-point 2020 echo "# Some Comment" >> flow.cylc -run_ok "${TEST_NAME_BASE}-vr" \ - cylc vr "${WORKFLOW_ID}" \ - --stop-cycle-point 2020-01-01T00:02Z +run_ok "${TEST_NAME_BASE}-vr" cylc vr "${WORKFLOW_ID}" --no-detach diff --git a/tests/functional/cylc-combination-scripts/09-vr-icp-now/flow.cylc b/tests/functional/cylc-combination-scripts/09-vr-icp-now/flow.cylc index e9f6284769..0866416c7d 100644 --- a/tests/functional/cylc-combination-scripts/09-vr-icp-now/flow.cylc +++ b/tests/functional/cylc-combination-scripts/09-vr-icp-now/flow.cylc @@ -1,8 +1,8 @@ [scheduling] initial cycle point = 2020 - stop after cycle point = 2020-01-01T00:01Z + final cycle point = 2021 [[graph]] - PT1M = foo + P1Y = foo [runtime] [[foo]] [[[simulation]]] diff --git a/tests/functional/param_expand/01-basic.t b/tests/functional/param_expand/01-basic.t index e058224649..a7c34d4282 100644 --- a/tests/functional/param_expand/01-basic.t +++ b/tests/functional/param_expand/01-basic.t @@ -381,9 +381,9 @@ cmp_ok '19.cylc' <<'__FLOW_CONFIG__' [[templates]] lang = %(lang)s [scheduling] - cycling mode = integer initial cycle point = 1 final cycle point = 1 + cycling mode = integer [[graph]] R1 = => [runtime] diff --git a/tests/functional/reload/23-cycle-point-time-zone.t b/tests/functional/reload/23-cycle-point-time-zone.t index d9bf216656..a6d181b9a6 100644 --- a/tests/functional/reload/23-cycle-point-time-zone.t +++ b/tests/functional/reload/23-cycle-point-time-zone.t @@ -50,7 +50,7 @@ poll_grep_workflow_log "Reload completed" cylc stop --now --now "${WORKFLOW_NAME}" log_scan "${TEST_NAME_BASE}-log-scan" "${WORKFLOW_RUN_DIR}/log/scheduler/log" 1 0 \ - 'LOADING workflow parameters' \ + 'LOADING saved workflow parameters' \ '+ cycle point time zone = +0100' purge diff --git a/tests/functional/restart/52-cycle-point-time-zone.t b/tests/functional/restart/52-cycle-point-time-zone.t index 9b928ca721..53430c2c25 100644 --- a/tests/functional/restart/52-cycle-point-time-zone.t +++ b/tests/functional/restart/52-cycle-point-time-zone.t @@ -55,7 +55,7 @@ poll_workflow_running cylc stop "${WORKFLOW_NAME}" log_scan "${TEST_NAME_BASE}-log-scan" "${WORKFLOW_RUN_DIR}/log/scheduler/log" 1 0 \ - 'LOADING workflow parameters' \ + 'LOADING saved workflow parameters' \ '+ cycle point time zone = +0100' purge diff --git a/tests/integration/test_config.py b/tests/integration/test_config.py index 440e2c9814..0d71cbe6bd 100644 --- a/tests/integration/test_config.py +++ b/tests/integration/test_config.py @@ -21,6 +21,7 @@ from typing import Any import pytest +from cylc.flow import commands from cylc.flow.cfgspec.glbl_cfg import glbl_cfg from cylc.flow.cfgspec.globalcfg import GlobalConfig from cylc.flow.exceptions import ( @@ -33,6 +34,7 @@ from cylc.flow.parsec.exceptions import ListValueError from cylc.flow.parsec.fileparse import read_and_proc from cylc.flow.pathutil import get_workflow_run_pub_db_path +from cylc.flow.scheduler import Scheduler Fixture = Any @@ -739,3 +741,60 @@ async def test_task_event_bad_custom_template( with pytest.raises(WorkflowConfigError, match=exception): async with start(schd): pass + + +async def test_icp_now_reload( + flow, scheduler, start, monkeypatch: pytest.MonkeyPatch, log_filter +): + """initial cycle point = 'now' should not change from original value on + reload/restart, and sequences should remain intact. + + https://github.com/cylc/cylc-flow/issues/7047 + """ + def set_time(value): + monkeypatch.setattr( + 'cylc.flow.config.get_current_time_string', + lambda *a, **k: f"2005-01-01T{value}Z", + ) + + wid = flow({ + 'scheduling': { + 'initial cycle point': 'now', + 'graph': { + 'R1': 'cold => foo', + 'PT15M': 'foo[-PT15M] => foo', + }, + }, + }) + schd: Scheduler = scheduler(wid) + + def main_check(icp): + assert str(schd.config.initial_point) == icp + assert schd.pool.get_task_ids() == { + f'{icp}/cold', + } + assert {str(seq) for seq in schd.config.sequences} == { + f'R1/{icp}/P0Y', + f'R/{icp}/PT15M', + } + + set_time('06:00') + async with start(schd): + expected_icp = '20050101T0600Z' + main_check(expected_icp) + + set_time('06:03') + await commands.run_cmd(commands.reload_workflow(schd)) + + main_check(expected_icp) + + await commands.run_cmd( + commands.set_prereqs_and_outputs( + schd, [f'{expected_icp}/cold'], [] + ) + ) + # Downstream task should have spawned on sequence: + assert schd.pool.get_task_ids() == { + f'{expected_icp}/foo', + } + assert not log_filter(level=logging.WARNING) diff --git a/tests/unit/test_config.py b/tests/unit/test_config.py index eb236d7240..377cada18b 100644 --- a/tests/unit/test_config.py +++ b/tests/unit/test_config.py @@ -263,14 +263,12 @@ def test_family_inheritance_and_quotes( @pytest.mark.parametrize( - ('cycling_type', 'scheduling_cfg', 'expected_icp', 'expected_eval_icp', - 'expected_err'), + ('cycling_type', 'scheduling_cfg', 'expected_icp', 'expected_err'), [ pytest.param( ISO8601_CYCLING_TYPE, {'initial cycle point': None}, None, - None, (WorkflowConfigError, "requires an initial cycle point"), id="Lack of icp" ), @@ -279,14 +277,12 @@ def test_family_inheritance_and_quotes( {'initial cycle point': None}, '1', None, - None, id="Default icp for integer cycling type" ), pytest.param( INTEGER_CYCLING_TYPE, {'initial cycle point': "now"}, None, - None, (PointParsingError, "invalid literal for int()"), id="Non-integer ICP for integer cycling type" ), @@ -294,7 +290,6 @@ def test_family_inheritance_and_quotes( INTEGER_CYCLING_TYPE, {'initial cycle point': "20500808T0000Z"}, None, - None, (PointParsingError, "invalid literal for int()"), id="More non-integer ICP for integer cycling type" ), @@ -302,7 +297,6 @@ def test_family_inheritance_and_quotes( ISO8601_CYCLING_TYPE, {'initial cycle point': "1"}, None, - None, (PointParsingError, "Invalid ISO 8601 date representation"), id="Non-ISO8601 ICP for ISO8601 cycling type" ), @@ -310,10 +304,16 @@ def test_family_inheritance_and_quotes( ISO8601_CYCLING_TYPE, {'initial cycle point': 'now'}, '20050102T0615+0530', - '20050102T0615+0530', None, id="ICP = now" ), + pytest.param( + ISO8601_CYCLING_TYPE, + {'initial cycle point': 'previous(T00)'}, + '20050102T0000+0530', + None, + id="ICP = prev" + ), pytest.param( ISO8601_CYCLING_TYPE, { @@ -322,7 +322,6 @@ def test_family_inheritance_and_quotes( }, '20130101T0000+0530', None, - None, id="Constraints" ), pytest.param( @@ -332,7 +331,6 @@ def test_family_inheritance_and_quotes( 'initial cycle point constraints': ['--01-19', '--01-21'] }, None, - None, (WorkflowConfigError, "does not meet the constraints"), id="Violated constraints" ), @@ -342,7 +340,6 @@ def test_family_inheritance_and_quotes( 'initial cycle point': 'a', }, None, - None, (WorkflowConfigError, 'Invalid ISO 8601 date representation: a'), id="invalid" ), @@ -352,7 +349,6 @@ def test_process_icp( cycling_type: str, scheduling_cfg: Dict[str, Any], expected_icp: Optional[str], - expected_eval_icp: Optional[str], expected_err: Optional[Tuple[Type[Exception], str]], monkeypatch: pytest.MonkeyPatch, set_cycling_type: 'Fixture' ) -> None: @@ -364,8 +360,6 @@ def test_process_icp( cycling_type: Workflow cycling type. scheduling_cfg: 'scheduling' section of workflow config. expected_icp: The expected icp value that gets set. - expected_eval_icp: The expected value of options.icp that gets set - (this gets stored in the workflow DB). expected_err: Exception class expected to be raised plus the message. """ set_cycling_type(cycling_type, time_zone="+0530") @@ -392,10 +386,6 @@ def test_process_icp( assert mocked_config.cfg[ 'scheduling']['initial cycle point'] == expected_icp assert str(mocked_config.initial_point) == expected_icp - eval_icp = mocked_config.evaluated_icp - if eval_icp is not None: - eval_icp = str(loader.get_point(eval_icp).standardise()) - assert eval_icp == expected_eval_icp @pytest.mark.parametrize( @@ -413,6 +403,12 @@ def test_process_icp( '20050102T0615+0530', None ), + ( + 'previous(T00)', + None, + '20050102T0000+0530', + None, + ), ( None, None,