diff --git a/changes.d/5658.feat.md b/changes.d/5658.feat.md new file mode 100644 index 00000000000..ed33a09b529 --- /dev/null +++ b/changes.d/5658.feat.md @@ -0,0 +1 @@ +New "cylc set" command for setting task prerequisites and outputs. diff --git a/cylc/flow/config.py b/cylc/flow/config.py index d80456266bf..93a49a435c3 100644 --- a/cylc/flow/config.py +++ b/cylc/flow/config.py @@ -88,7 +88,7 @@ ) from cylc.flow.task_id import TaskID from cylc.flow.task_outputs import ( - TASK_OUTPUT_SUCCEEDED, + TASK_OUTPUT_EXPIRED, TaskOutputs ) from cylc.flow.task_trigger import TaskTrigger, Dependency @@ -1649,7 +1649,7 @@ def generate_triggers(self, lexpression, left_nodes, right, seq, offset_is_irregular, offset_is_absolute) = ( GraphNodeParser.get_inst().parse(left)) - # Qualifier. + # Qualifier. Note ":succeeded" made explicit by the graph parser. outputs = self.cfg['runtime'][name]['outputs'] if outputs and (output in outputs): # Qualifier is a custom task message. @@ -1660,9 +1660,6 @@ def generate_triggers(self, lexpression, left_nodes, right, seq, f"Undefined custom output: {name}:{output}" ) qualifier = output - else: - # No qualifier specified => use "succeeded". - qualifier = TASK_OUTPUT_SUCCEEDED # Generate TaskTrigger if not already done. key = (name, offset, qualifier, @@ -2131,7 +2128,11 @@ def set_required_outputs( Args: task_output_opt: {(task, output): (is-optional, default, is_set)} + """ + # task_output_opt: outputs parsed from graph triggers + # taskdef.outputs: outputs listed under runtime + for name, taskdef in self.taskdefs.items(): for output in taskdef.outputs: try: @@ -2141,6 +2142,28 @@ def set_required_outputs( continue taskdef.set_required_output(output, not optional) + # Add expired outputs to taskdefs if flagged in the graph. + graph_exp = set() + for (task, output) in task_output_opt.keys(): + if output == TASK_OUTPUT_EXPIRED: + graph_exp.add(task) + self.taskdefs[task].add_output( + TASK_OUTPUT_EXPIRED, TASK_OUTPUT_EXPIRED + ) + + # clock-expire must be flagged in the graph for visibility + bad_exp = set() + for task in self.expiration_offsets: + if task not in graph_exp: + bad_exp.add(task) + + if bad_exp: + msg = '\n '.join( + [t + f":{TASK_OUTPUT_EXPIRED}?" for t in bad_exp]) + raise WorkflowConfigError( + f"Clock-expire must be visible in the graph:\n {msg}" + ) + def find_taskdefs(self, name: str) -> Set[TaskDef]: """Find TaskDef objects in family "name" or matching "name". diff --git a/cylc/flow/data_store_mgr.py b/cylc/flow/data_store_mgr.py index ef77105b3a4..7a59e76d7a4 100644 --- a/cylc/flow/data_store_mgr.py +++ b/cylc/flow/data_store_mgr.py @@ -2185,8 +2185,9 @@ def update_workflow(self, reloaded=False): w_delta.n_edge_distance = self.n_edge_distance delta_set = True - if self.schd.pool.main_pool: - pool_points = set(self.schd.pool.main_pool) + if self.schd.pool.active_tasks: + pool_points = set(self.schd.pool.active_tasks) + oldest_point = str(min(pool_points)) if w_data.oldest_active_cycle_point != oldest_point: w_delta.oldest_active_cycle_point = oldest_point diff --git a/cylc/flow/etc/cylc b/cylc/flow/etc/cylc index 48a91c0603a..facf27e4291 100755 --- a/cylc/flow/etc/cylc +++ b/cylc/flow/etc/cylc @@ -2,7 +2,7 @@ # 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 diff --git a/cylc/flow/flow_mgr.py b/cylc/flow/flow_mgr.py index 41aec947a80..1a2bdb7343e 100644 --- a/cylc/flow/flow_mgr.py +++ b/cylc/flow/flow_mgr.py @@ -20,18 +20,99 @@ import datetime from cylc.flow import LOG +from cylc.flow.exceptions import InputError if TYPE_CHECKING: from cylc.flow.workflow_db_mgr import WorkflowDatabaseManager - FlowNums = Set[int] # Flow constants FLOW_ALL = "all" FLOW_NEW = "new" FLOW_NONE = "none" +# For flow-related CLI options: +ERR_OPT_FLOW_VAL = "Flow values must be an integer, or 'all', 'new', or 'none'" +ERR_OPT_FLOW_INT = "Multiple flow options must all be integer valued" +ERR_OPT_FLOW_WAIT = ( + f"--wait is not compatible with --flow={FLOW_NEW} or --flow={FLOW_NONE}" +) + + +def add_flow_opts(parser): + parser.add_option( + "--flow", action="append", dest="flow", metavar="FLOW", + help=f'Assign new tasks to all active flows ("{FLOW_ALL}");' + f' no flow ("{FLOW_NONE}"); a new flow ("{FLOW_NEW}");' + f' or a specific flow (e.g. "2"). The default is "{FLOW_ALL}".' + ' Specific flow numbers can be new or existing.' + ' Reuse the option to assign multiple flow numbers.' + ) + + parser.add_option( + "--meta", metavar="DESCRIPTION", action="store", + dest="flow_descr", default=None, + help=f"description of new flow (with --flow={FLOW_NEW})." + ) + + parser.add_option( + "--wait", action="store_true", default=False, dest="flow_wait", + help="Wait for merge with current active flows before flowing on." + ) + + +def validate_flow_opts(options): + """Check validity of flow-related CLI options.""" + if options.flow is None: + # Default to all active flows + options.flow = [FLOW_ALL] + + for val in options.flow: + val = val.strip() + if val in [FLOW_NONE, FLOW_NEW, FLOW_ALL]: + if len(options.flow) != 1: + raise InputError(ERR_OPT_FLOW_INT) + else: + try: + int(val) + except ValueError: + raise InputError(ERR_OPT_FLOW_VAL.format(val)) + + if options.flow_wait and options.flow[0] in [FLOW_NEW, FLOW_NONE]: + raise InputError(ERR_OPT_FLOW_WAIT) + + +def stringify_flow_nums(flow_nums: Set[int], full: bool = False) -> str: + """Return a string representation of a set of flow numbers + + If the set contains only the original flow 1, return an empty string + so that users can disregard flows unless they trigger new ones. + + Otherwise return e.g. "(1,2,3)". + + Examples: + >>> stringify_flow_nums({}) + '(none)' + + >>> stringify_flow_nums({1}) + '' + + >>> stringify_flow_nums({1}, True) + '(1)' + + >>> stringify_flow_nums({1,2,3}) + '(1,2,3)' + + """ + if not full and flow_nums == {1}: + return "" + return ( + "(" + f"{','.join(str(i) for i in flow_nums) or 'none'}" + ")" + ) + class FlowMgr: """Logic to manage flow counter and flow metadata.""" @@ -42,28 +123,54 @@ def __init__(self, db_mgr: "WorkflowDatabaseManager") -> None: self.flows: Dict[int, Dict[str, str]] = {} self.counter: int = 0 - def get_new_flow(self, description: Optional[str] = None) -> int: - """Increment flow counter, record flow metadata.""" - self.counter += 1 - # record start time to nearest second - now = datetime.datetime.now() - now_sec: str = str( - now - datetime.timedelta(microseconds=now.microsecond)) - description = description or "no description" - self.flows[self.counter] = { - "description": description, - "start_time": now_sec - } - LOG.info( - f"New flow: {self.counter} " - f"({description}) " - f"{now_sec}" - ) - self.db_mgr.put_insert_workflow_flows( - self.counter, - self.flows[self.counter] - ) - return self.counter + def get_flow_num( + self, + flow_num: Optional[int] = None, + meta: Optional[str] = None + ) -> int: + """Return a valid flow number, and record a new flow if necessary. + + If asked for a new flow: + - increment the automatic counter until we find an unused number + + If given a flow number: + - record a new flow if the number is unused + - else return it, as an existing flow number. + + The metadata string is only used if it is a new flow. + + """ + if flow_num is None: + self.counter += 1 + while self.counter in self.flows: + # Skip manually-created out-of-sequence flows. + self.counter += 1 + flow_num = self.counter + + if flow_num in self.flows: + if meta is not None: + LOG.warning( + f'Ignoring flow metadata "{meta}":' + f' {flow_num} is not a new flow' + ) + else: + # Record a new flow. + now = datetime.datetime.now() + now_sec: str = str( + now - datetime.timedelta(microseconds=now.microsecond)) + meta = meta or "no description" + self.flows[flow_num] = { + "description": meta, + "start_time": now_sec + } + LOG.info( + f"New flow: {flow_num} ({meta}) {now_sec}" + ) + self.db_mgr.put_insert_workflow_flows( + flow_num, + self.flows[flow_num] + ) + return flow_num def load_from_db(self, flow_nums: FlowNums) -> None: """Load flow data for scheduler restart. diff --git a/cylc/flow/graph_parser.py b/cylc/flow/graph_parser.py index ad0ec280a3d..090ebae1c5a 100644 --- a/cylc/flow/graph_parser.py +++ b/cylc/flow/graph_parser.py @@ -33,6 +33,7 @@ from cylc.flow.task_id import TaskID from cylc.flow.task_trigger import TaskTrigger from cylc.flow.task_outputs import ( + TASK_OUTPUT_EXPIRED, TASK_OUTPUT_SUCCEEDED, TASK_OUTPUT_STARTED, TASK_OUTPUT_FAILED, @@ -41,6 +42,8 @@ TASK_OUTPUT_SUBMIT_FAILED ) from cylc.flow.task_qualifiers import ( + QUAL_FAM_EXPIRE_ALL, + QUAL_FAM_EXPIRE_ANY, QUAL_FAM_SUCCEED_ALL, QUAL_FAM_SUCCEED_ANY, QUAL_FAM_FAIL_ALL, @@ -58,6 +61,7 @@ class Replacement: """A class to remember match group information in re.sub() calls""" + def __init__(self, replacement): self.replacement = replacement self.substitutions = [] @@ -124,6 +128,8 @@ class GraphParser: # E.g. QUAL_FAM_START_ALL: (TASK_OUTPUT_STARTED, True) simply maps # "FAM:start-all" to "MEMBER:started" and "-all" (all members). fam_to_mem_trigger_map: Dict[str, Tuple[str, bool]] = { + QUAL_FAM_EXPIRE_ALL: (TASK_OUTPUT_EXPIRED, True), + QUAL_FAM_EXPIRE_ANY: (TASK_OUTPUT_EXPIRED, False), QUAL_FAM_START_ALL: (TASK_OUTPUT_STARTED, True), QUAL_FAM_START_ANY: (TASK_OUTPUT_STARTED, False), QUAL_FAM_SUCCEED_ALL: (TASK_OUTPUT_SUCCEEDED, True), @@ -140,6 +146,8 @@ class GraphParser: # Map family pseudo triggers to affected member outputs. fam_to_mem_output_map: Dict[str, List[str]] = { + QUAL_FAM_EXPIRE_ANY: [TASK_OUTPUT_EXPIRED], + QUAL_FAM_EXPIRE_ALL: [TASK_OUTPUT_EXPIRED], QUAL_FAM_START_ANY: [TASK_OUTPUT_STARTED], QUAL_FAM_START_ALL: [TASK_OUTPUT_STARTED], QUAL_FAM_SUCCEED_ANY: [TASK_OUTPUT_SUCCEEDED], @@ -738,6 +746,10 @@ def _set_output_opt( if suicide: return + if output == TASK_OUTPUT_EXPIRED and not optional: + raise GraphParseError( + f"Output {name}:{output} must be optional (append '?')") + if output == TASK_OUTPUT_FINISHED: # Interpret :finish pseudo-output if optional: diff --git a/cylc/flow/id.py b/cylc/flow/id.py index db6d5ca8d80..af7802ae04c 100644 --- a/cylc/flow/id.py +++ b/cylc/flow/id.py @@ -31,6 +31,7 @@ ) from cylc.flow import LOG +from cylc.flow.task_outputs import TASK_OUTPUT_SUCCEEDED class IDTokens(Enum): @@ -356,6 +357,14 @@ def is_null(self) -> bool: self[key] for key in self._REGULAR_KEYS ) + def to_prereq_tuple(self) -> Tuple[str, str, str]: + """Return (cycle, task, selector) as used for task prerequisites.""" + return ( + self['cycle'], + self['task'], + self['task_sel'] or TASK_OUTPUT_SUCCEEDED + ) + def duplicate( self, *tokens_list, diff --git a/cylc/flow/id_match.py b/cylc/flow/id_match.py index bbe087d7fe8..de3fa21cb16 100644 --- a/cylc/flow/id_match.py +++ b/cylc/flow/id_match.py @@ -76,7 +76,7 @@ def filter_ids( - pools: 'List[Pool]', + pool: 'Pool', ids: 'Iterable[str]', *, warn: 'bool' = True, @@ -145,8 +145,6 @@ def filter_ids( if tokens.get(lowest_token.value): break - # This needs to be a set to avoid getting two copies of matched tasks - # in cycle points that appear in both pools: cycles = set() tasks = [] @@ -154,19 +152,18 @@ def filter_ids( if lowest_token == IDTokens.Cycle: cycle = tokens[IDTokens.Cycle.value] cycle_sel = tokens.get(IDTokens.Cycle.value + '_sel') or '*' - for pool in pools: - for icycle, itasks in pool.items(): - if not itasks: - continue - if not point_match(icycle, cycle, pattern_match): - continue - if cycle_sel == '*': + for icycle, itasks in pool.items(): + if not itasks: + continue + if not point_match(icycle, cycle, pattern_match): + continue + if cycle_sel == '*': + cycles.add(icycle) + continue + for itask in itasks.values(): + if match(itask.state.status, cycle_sel): cycles.add(icycle) - continue - for itask in itasks.values(): - if match(itask.state.status, cycle_sel): - cycles.add(icycle) - break + break # filter by task elif lowest_token == IDTokens.Task: # noqa SIM106 @@ -176,36 +173,35 @@ def filter_ids( task = tokens[IDTokens.Task.value] task_sel_raw = tokens.get(IDTokens.Task.value + '_sel') task_sel = task_sel_raw or '*' - for pool in pools: - for icycle, itasks in pool.items(): - if not point_match(icycle, cycle, pattern_match): - continue - for itask in itasks.values(): - if ( - # check cycle selector + for icycle, itasks in pool.items(): + if not point_match(icycle, cycle, pattern_match): + continue + for itask in itasks.values(): + if ( + # check cycle selector + ( ( - ( - # disable cycle_sel if not defined if - # pattern matching is turned off - pattern_match is False - and cycle_sel_raw is None - ) - or match(itask.state.status, cycle_sel) + # disable cycle_sel if not defined if + # pattern matching is turned off + pattern_match is False + and cycle_sel_raw is None ) - # check namespace name - and itask.name_match(task, match_func=match) - # check task selector - and ( - ( - # disable task_sel if not defined if - # pattern matching is turned off - pattern_match is False - and task_sel_raw is None - ) - or match(itask.state.status, task_sel) + or match(itask.state.status, cycle_sel) + ) + # check namespace name + and itask.name_match(task, match_func=match) + # check task selector + and ( + ( + # disable task_sel if not defined if + # pattern matching is turned off + pattern_match is False + and task_sel_raw is None ) - ): - tasks.append(itask) + or match(itask.state.status, task_sel) + ) + ): + tasks.append(itask) else: raise NotImplementedError @@ -226,10 +222,9 @@ def filter_ids( }) ret = _cycles elif out == IDTokens.Task: - for pool in pools: - for icycle in _cycles: - if icycle in pool: - _tasks.extend(pool[icycle].values()) + for icycle in _cycles: + if icycle in pool: + _tasks.extend(pool[icycle].values()) ret = _tasks return ret, _not_matched diff --git a/cylc/flow/network/resolvers.py b/cylc/flow/network/resolvers.py index 5af50b0a4f8..a694b092802 100644 --- a/cylc/flow/network/resolvers.py +++ b/cylc/flow/network/resolvers.py @@ -26,7 +26,6 @@ from typing import ( Any, Dict, - Iterable, List, NamedTuple, Optional, @@ -58,7 +57,6 @@ from graphql import ResolveInfo from cylc.flow.data_store_mgr import DataStoreMgr from cylc.flow.scheduler import Scheduler - from cylc.flow.workflow_status import StopMode class TaskMsg(NamedTuple): @@ -706,32 +704,40 @@ async def mutator( 'response': (False, f'No matching workflow in {workflows}')}] w_id = w_ids[0] result = await self._mutation_mapper(command, kwargs, meta) - if result is None: - result = (True, 'Command queued') return [{'id': w_id, 'response': result}] - def _log_command(self, command: str, user: str) -> None: - """Log receipt of command, with user name if not owner.""" - is_owner = user == self.schd.owner - if command == 'put_messages' and is_owner: - # Logging put_messages is overkill. - return - log_msg = f"[command] {command}" - if not is_owner: - log_msg += (f" (issued by {user})") - LOG.info(log_msg) - async def _mutation_mapper( self, command: str, kwargs: Dict[str, Any], meta: Dict[str, Any] - ) -> Optional[Tuple[bool, str]]: - """Map between GraphQL resolvers and internal command interface.""" + ) -> Tuple[bool, str]: + """Map to internal command interface. + + Some direct methods are in this module. + Others go to the scheduler command queue. + + """ + user = meta.get('auth_user', self.schd.owner) + if user == self.schd.owner: + log_user = "" # don't log user name if owner + else: + log_user = f" from {user}" - self._log_command( - command, - meta.get('auth_user', self.schd.owner) + log_msg1 = f'Command "{command}" received{log_user}.' + + log_msg2 = ( + f"{command}(" + + ", ".join( + f"{key}={value}" for key, value in kwargs.items()) + + ")" ) + method = getattr(self, command, None) if method is not None: + if ( + command != "put_messages" + or user != self.schd.owner + ): + # Logging task messages as commands is overkill. + LOG.info(f"{log_msg1}\n{log_msg2}") return method(**kwargs) try: @@ -739,11 +745,15 @@ async def _mutation_mapper( except AttributeError: raise ValueError(f"Command '{command}' not found") - self.schd.queue_command( - command, - kwargs + return ( + True, + self.schd.queue_command( + command, + [], + kwargs, + [log_msg1, log_msg2], + ) ) - return None def broadcast( self, @@ -846,107 +856,3 @@ def set_graph_window_extent(self, n_edge_distance): return (True, f'Maximum edge distance set to {n_edge_distance}') else: return (False, 'Edge distance cannot be negative') - - def force_spawn_children( - self, - tasks: Iterable[str], - outputs: Optional[Iterable[str]] = None, - flow_num: Optional[int] = None - ) -> Tuple[bool, str]: - """Spawn children of given task outputs. - - User-facing method name: set_outputs. - - Args: - tasks: List of identifiers or task globs. - outputs: List of outputs to spawn on. - flow_num: Flow number to attribute the outputs. - """ - self.schd.command_queue.put( - ( - "force_spawn_children", - (tasks,), - { - "outputs": outputs, - "flow_num": flow_num - }, - ) - ) - return (True, 'Command queued') - - def stop( - self, - mode: Union[str, 'StopMode'], - cycle_point: Optional[str] = None, - clock_time: Optional[str] = None, - task: Optional[str] = None, - flow_num: Optional[int] = None, - ) -> Tuple[bool, str]: - """Stop the workflow or specific flow from spawning any further. - - Args: - mode: Stop mode to set - cycle_point: Cycle point after which to stop. - clock_time: Wallclock time after which to stop. - task: Stop after this task succeeds. - flow_num: The flow to stop. - ): - - Returns: - outcome: True if command successfully queued. - message: Information about outcome. - - """ - self.schd.command_queue.put(( - "stop", - (), - filter_none({ - 'mode': mode, - 'cycle_point': cycle_point, - 'clock_time': clock_time, - 'task': task, - 'flow_num': flow_num, - }), - ) - ) - return (True, 'Command queued') - - def force_trigger_tasks( - self, - tasks: Iterable[str], - flow: Iterable[str], - flow_wait: bool, - flow_descr: Optional[str] = None, - ): - """Trigger submission of task jobs where possible. - - Args: - tasks (list): - List of identifiers or task globs. - flow (list): - Flow ownership of triggered tasks. - flow_wait (bool): - Wait for flows before continuing - flow_descr (str): - Description of new flow. - - Returns: - tuple: (outcome, message) - outcome (bool) - True if command successfully queued. - message (str) - Information about outcome. - - """ - self.schd.command_queue.put( - ( - "force_trigger_tasks", - (tasks or [],), - { - "flow": flow, - "flow_wait": flow_wait, - "flow_descr": flow_descr - } - ), - ) - return (True, 'Command queued') diff --git a/cylc/flow/network/schema.py b/cylc/flow/network/schema.py index f9ed95c7158..8f0b88b5035 100644 --- a/cylc/flow/network/schema.py +++ b/cylc/flow/network/schema.py @@ -43,7 +43,6 @@ from cylc.flow.id import Tokens from cylc.flow.task_outputs import SORT_ORDERS from cylc.flow.task_state import ( - TASK_OUTPUT_SUCCEEDED, TASK_STATUSES_ORDERED, TASK_STATUS_DESC, TASK_STATUS_WAITING, @@ -1009,6 +1008,12 @@ class Meta: time = Float() +class OutputLabel(String): + class Meta: + description = """Task output label, as used in the graph.""" + label = String() + + class XTrigger(ObjectType): class Meta: description = """Task trigger""" @@ -2101,27 +2106,33 @@ class Meta: resolver = partial(mutator, command='remove_tasks') -class SetOutputs(Mutation, TaskMutation): +class Set(Mutation, TaskMutation): class Meta: - description = sstrip(''' - Artificially mark task outputs as completed. + description = sstrip(""" + Set task prerequisites or outputs. - This allows you to manually intervene with Cylc's scheduling - algorithm by artificially satisfying outputs of tasks. + By default, set all required outputs for target task(s). - By default this makes tasks appear as if they succeeded. + Setting prerequisites contributes to the task's readiness to run. - Valid for: paused, running workflows. - ''') - resolver = partial(mutator, command='force_spawn_children') + Setting outputs contributes to the task's completion, sets the + corresponding prerequisites of child tasks, and sets any implied + outputs: + - started implies submitted + - succeeded and failed imply started + - custom outputs and expired do not imply any other outputs + """) + resolver = partial(mutator, command='set') - class Arguments(TaskMutation.Arguments): + class Arguments(TaskMutation.Arguments, FlowMutationArguments): outputs = graphene.List( + OutputLabel, + description='List of task outputs to set complete.' + ) + prerequisites = graphene.List( String, - default_value=[TASK_OUTPUT_SUCCEEDED], - description='List of task outputs to satisfy.' + description='List of task prerequisites to set satisfied.' ) - flow_num = Int() class Trigger(Mutation, TaskMutation): @@ -2181,7 +2192,7 @@ class Mutations(ObjectType): poll = _mut_field(Poll) release = _mut_field(Release) remove = _mut_field(Remove) - set_outputs = _mut_field(SetOutputs) + set = _mut_field(Set) # noqa: A003 trigger = _mut_field(Trigger) # job actions diff --git a/cylc/flow/option_parsers.py b/cylc/flow/option_parsers.py index b04b226c171..14191962c22 100644 --- a/cylc/flow/option_parsers.py +++ b/cylc/flow/option_parsers.py @@ -300,7 +300,7 @@ class CylcOptionParser(OptionParser): action='store_false', dest='log_timestamp', default=True, useif='all'), OptionSettings( - ['--color', '--color'], metavar='WHEN', action='store', + ['--color', '--colour'], metavar='WHEN', action='store', default='auto', choices=['never', 'auto', 'always'], help=( "When to use color/bold text in terminal output." diff --git a/cylc/flow/prerequisite.py b/cylc/flow/prerequisite.py index 31b5db83486..b4abfe737dc 100644 --- a/cylc/flow/prerequisite.py +++ b/cylc/flow/prerequisite.py @@ -18,6 +18,7 @@ import math import re +from typing import Iterable, Set, TYPE_CHECKING from cylc.flow.cycling.loader import get_point from cylc.flow.exceptions import TriggerExpressionError @@ -27,6 +28,9 @@ ) from cylc.flow.id import quick_relative_detokenise +if TYPE_CHECKING: + from cylc.flow.id import Tokens + class Prerequisite: """The concrete result of an abstract logical trigger expression. @@ -75,6 +79,10 @@ def __init__(self, point): # * `False` (prerequisite unsatisfied). self._all_satisfied = None + def __str__(self): + """Useful for debugging; could do with better formatting.""" + return f"{self.point}: {self.satisfied}, {self.conditional_expression}" + def instantaneous_hash(self): """Generate a hash of this prerequisite in its current state. @@ -194,20 +202,25 @@ def _conditional_is_satisfied(self): '"%s":\n%s' % (self.get_raw_conditional_expression(), err_msg)) return res - def satisfy_me(self, all_task_outputs): - """Evaluate prerequisite against known outputs. + def satisfy_me(self, outputs: Iterable['Tokens']) -> Set[str]: + """Attempt to satisfy me with given outputs. - Updates cache with the evaluation result. + Updates cache with the result. + Return outputs that match. """ - relevant_messages = all_task_outputs & set(self.satisfied) - for message in relevant_messages: - self.satisfied[message] = self.DEP_STATE_SATISFIED + valid = set() + for output in outputs: + prereq = output.to_prereq_tuple() + if prereq not in self.satisfied: + continue + valid.add(output.relative_id_with_selectors) + self.satisfied[prereq] = self.DEP_STATE_SATISFIED if self.conditional_expression is None: self._all_satisfied = all(self.satisfied.values()) else: self._all_satisfied = self._conditional_is_satisfied() - return relevant_messages + return valid def api_dump(self): """Return list of populated Protobuf data objects.""" diff --git a/cylc/flow/rundb.py b/cylc/flow/rundb.py index 256da3b3129..ce82b1a625f 100644 --- a/cylc/flow/rundb.py +++ b/cylc/flow/rundb.py @@ -21,7 +21,15 @@ from pprint import pformat import sqlite3 import traceback -from typing import TYPE_CHECKING, Iterable, List, Optional, Tuple, Union +from typing import ( + TYPE_CHECKING, + Iterable, + List, + Set, + Tuple, + Optional, + Union +) from cylc.flow import LOG from cylc.flow.exceptions import PlatformLookupError @@ -314,6 +322,7 @@ class CylcWorkflowDAO: ["status"], ["flow_wait", {"datatype": "INTEGER"}], ["is_manual_submit", {"datatype": "INTEGER"}], + ["is_complete", {"datatype": "INTEGER"}] ], TABLE_TASK_TIMEOUT_TIMERS: [ ["cycle", {"is_primary_key": True}], @@ -776,29 +785,32 @@ def select_task_job_platforms(self): ''' # nosec (table name is code constant) return {i[0] for i in self.connect().execute(stmt)} - def select_submit_nums(self, name, point): - """Select submit_num and flow_nums from task_states table. - - Fetch submit number and flow_nums for spawning tasks. - - Return: {submit_num: (flow_wait, flow_nums)} - - Args: - name: task name - point: task cycle point (str) + def select_prev_instances( + self, name: str, point: str + ) -> List[Tuple[int, bool, Set[int], bool]]: + """Select task_states table info about previous instances of a task. + Flow merge results in multiple entries for the same submit number. """ # Ignore bandit false positive: B608: hardcoded_sql_expressions # Not an injection, simply putting the table name in the SQL query # expression as a string constant local to this module. stmt = ( # nosec - r"SELECT flow_nums,submit_num,flow_wait FROM %(name)s" + r"SELECT flow_nums,submit_num,flow_wait,is_complete FROM %(name)s" r" WHERE name==? AND cycle==?" ) % {"name": self.TABLE_TASK_STATES} - ret = {} - for flow_nums_str, submit_num, flow_wait in self.connect().execute( - stmt, (name, point,)): - ret[submit_num] = (flow_wait == 1, deserialise(flow_nums_str)) + ret = [] + for flow_nums_str, submit_num, flow_wait, is_complete in ( + self.connect().execute(stmt, (name, point,)) + ): + ret.append( + ( + submit_num, + flow_wait == 1, + deserialise(flow_nums_str), + is_complete == 1 + ) + ) return ret def select_latest_flow_nums(self): diff --git a/cylc/flow/scheduler.py b/cylc/flow/scheduler.py index 0e02f9384d9..d66512d660b 100644 --- a/cylc/flow/scheduler.py +++ b/cylc/flow/scheduler.py @@ -223,7 +223,7 @@ class Scheduler: flow_mgr: FlowMgr # queues - command_queue: 'Queue[Tuple[str, tuple, dict]]' + command_queue: 'Queue[Tuple[str, str, list, dict]]' message_queue: 'Queue[TaskMsg]' ext_trigger_queue: Queue @@ -482,7 +482,8 @@ async def configure(self, params): self.workflow_db_mgr, self.task_events_mgr, self.data_store_mgr, - self.flow_mgr + self.flow_mgr, + self.task_job_mgr ) self.data_store_mgr.initiate_data_model() @@ -523,7 +524,7 @@ async def configure(self, params): timer.reset() self.timers[event] = timer - if self.is_restart and not self.pool.get_all_tasks(): + if self.is_restart and not self.pool.get_tasks(): # This workflow completed before restart; wait for intervention. with suppress(KeyError): self.timers[self.EVENT_RESTART_TIMEOUT].reset() @@ -603,6 +604,9 @@ def log_start(self) -> None: # Note that the following lines must be present at the top of # the workflow log file for use in reference test runs. + LOG.info( + "Task log prefix: CYCLE/TASK[/JOB][(FLOWS)]:STATUS" + ) LOG.info( f'Run mode: {self.config.run_mode()}', extra=RotatingLogFileHandler.header_extra @@ -633,7 +637,7 @@ async def run_scheduler(self) -> None: self.task_job_mgr.task_remote_mgr.is_restart = True self.task_job_mgr.task_remote_mgr.rsync_includes = ( self.config.get_validated_rsync_includes()) - if self.pool.get_all_tasks(): + if self.pool.get_tasks(): # (If we're not restarting a finished workflow) self.restart_remote_init() self.command_poll_tasks(['*/*']) @@ -901,6 +905,7 @@ def process_queued_task_messages(self) -> None: if message_items is None: continue should_poll = False + del messages[itask.identity] for submit_num, tm in message_items: if self.task_events_mgr.process_message( itask, tm.severity, tm.message, tm.event_time, @@ -909,18 +914,43 @@ def process_queued_task_messages(self) -> None: should_poll = True if should_poll: to_poll_tasks.append(itask) - self.task_job_mgr.poll_task_jobs( - self.workflow, to_poll_tasks) + if to_poll_tasks: + self.task_job_mgr.poll_task_jobs(self.workflow, to_poll_tasks) + # Remaining messages don't have a receiving task in the pool. + # E.g., after manually setting a running task to finished. + for _id, tms in messages.items(): + warn = "Undeliverable task messages received and ignored:\n" + for _, msg in tms: + warn += f'{msg.job_id}: {msg.severity} - "{msg.message}"' + LOG.warning(warn) def get_command_method(self, command_name: str) -> Callable: """Return a command processing method or raise AttributeError.""" return getattr(self, f'command_{command_name}') - def queue_command(self, command: str, kwargs: dict) -> None: - self.command_queue.put(( - command, - tuple(kwargs.values()), {} - )) + def queue_command( + self, + name: str, + args: list, + kwargs: dict, + log_lines: List[str], + ) -> str: + """Queue a command for action by the scheduler. + + Return a unique command ID for provenance tracking. + + """ + uuid = str(uuid4()) + LOG.info(f"{log_lines[0]} ID={uuid}\n{log_lines[1]}") + self.command_queue.put( + ( + uuid, + name, + args, + kwargs, + ) + ) + return uuid async def process_command_queue(self) -> None: """Process queued commands.""" @@ -929,17 +959,15 @@ async def process_command_queue(self) -> None: return LOG.debug(f"Processing {qsize} queued command(s)") while True: + uuid: str + name: str + args: list + kwargs: dict try: - command = self.command_queue.get(False) - name, args, kwargs = command + uuid, name, args, kwargs = self.command_queue.get(False) except Empty: break - args_string = ', '.join(str(a) for a in args) - kwargs_string = ', '.join( - f"{key}={value}" for key, value in kwargs.items() - ) - sep = ', ' if kwargs_string and args_string else '' - cmdstr = f"{name}({args_string}{sep}{kwargs_string})" + msg = f'Command "{name}" ' + '{result}' + f'. ID={uuid}' try: fcn = self.get_command_method(name) n_warnings: Optional[int] @@ -954,16 +982,20 @@ async def process_command_queue(self) -> None: not isinstance(exc, CommandFailedError) ): LOG.error(traceback.format_exc()) - LOG.error(f"Command failed: {cmdstr}\n{exc}") + LOG.error( + msg.format(result="failed") + f"\n{exc}" + ) else: if n_warnings: LOG.info( - f"Command actioned with {n_warnings} warning(s): " - f"{cmdstr}" + msg.format( + result=f"actioned with {n_warnings} warnings" + ) ) else: - LOG.info(f"Command actioned: {cmdstr}") + LOG.info(msg.format(result="actioned")) self.is_updated = True + self.command_queue.task_done() def info_get_graph_raw(self, cto, ctn, grouping=None): @@ -1025,9 +1057,9 @@ def _set_stop(self, stop_mode: Optional[StopMode] = None) -> None: self.stop_mode = stop_mode self.update_data_store() - def command_release(self, task_globs: Iterable[str]) -> int: + def command_release(self, tasks: Iterable[str]) -> int: """Release held tasks.""" - return self.pool.release_held_tasks(task_globs) + return self.pool.release_held_tasks(tasks) def command_release_hold_point(self) -> None: """Release all held tasks and unset workflow hold after cycle point, @@ -1039,29 +1071,29 @@ def command_resume(self) -> None: """Resume paused workflow.""" self.resume_workflow() - def command_poll_tasks(self, items: List[str]) -> int: + def command_poll_tasks(self, tasks: Iterable[str]) -> int: """Poll pollable tasks or a task or family if options are provided.""" if self.config.run_mode('simulation'): return 0 - itasks, _, bad_items = self.pool.filter_task_proxies(items) + itasks, _, bad_items = self.pool.filter_task_proxies(tasks) self.task_job_mgr.poll_task_jobs(self.workflow, itasks) return len(bad_items) - def command_kill_tasks(self, items: List[str]) -> int: + def command_kill_tasks(self, tasks: Iterable[str]) -> int: """Kill all tasks or a task/family if options are provided.""" - itasks, _, bad_items = self.pool.filter_task_proxies(items) + itasks, _, bad_items = self.pool.filter_task_proxies(tasks) if self.config.run_mode('simulation'): for itask in itasks: if itask.state(*TASK_STATUSES_ACTIVE): itask.state_reset(TASK_STATUS_FAILED) self.data_store_mgr.delta_task_state(itask) return len(bad_items) - self.task_job_mgr.kill_task_jobs(self.workflow, itasks) + self.task_job_mgr.kill_task_jobs(itasks) return len(bad_items) - def command_hold(self, task_globs: Iterable[str]) -> int: + def command_hold(self, tasks: Iterable[str]) -> int: """Hold specified tasks.""" - return self.pool.hold_tasks(task_globs) + return self.pool.hold_tasks(tasks) def command_set_hold_point(self, point: str) -> None: """Hold all tasks after the specified cycle point.""" @@ -1078,18 +1110,18 @@ def command_pause(self) -> None: self.pause_workflow() @staticmethod - def command_set_verbosity(lvl: Union[int, str]) -> None: + def command_set_verbosity(level: Union[int, str]) -> None: """Set workflow verbosity.""" try: - lvl = int(lvl) + lvl = int(level) LOG.setLevel(lvl) except (TypeError, ValueError) as exc: raise CommandFailedError(exc) cylc.flow.flags.verbosity = log_level_to_verbosity(lvl) - def command_remove_tasks(self, items) -> int: + def command_remove_tasks(self, tasks: Iterable[str]) -> int: """Remove tasks.""" - return self.pool.remove_tasks(items) + return self.pool.remove_tasks(tasks) async def command_reload_workflow(self) -> None: """Reload workflow configuration.""" @@ -1700,6 +1732,7 @@ async def main_loop(self) -> None: tinit = time() # Useful for debugging core scheduler issues: + # import logging # self.pool.log_task_pool(logging.CRITICAL) if self.incomplete_ri_map: self.manage_remote_init() @@ -1736,7 +1769,7 @@ async def main_loop(self) -> None: if self.xtrigger_mgr.do_housekeeping: self.xtrigger_mgr.housekeep(self.pool.get_tasks()) - self.pool.set_expired_tasks() + self.pool.clock_expire_tasks() self.release_queued_tasks() if ( @@ -2065,9 +2098,11 @@ def check_auto_shutdown(self): if itask.state( TASK_STATUS_PREPARING, TASK_STATUS_SUBMITTED, - TASK_STATUS_RUNNING - ) - or ( + TASK_STATUS_RUNNING, + ) or ( + # This is because runahead limit gets truncated + # to stop_point if there is one, so tasks spawned + # beyond the stop_point must be runahead limited. itask.state(TASK_STATUS_WAITING) and not itask.state.is_runahead ) @@ -2094,7 +2129,7 @@ def pause_workflow(self, msg: Optional[str] = None) -> None: if self.is_paused: LOG.info("Workflow is already paused") return - _msg = "PAUSING the workflow now" + _msg = "Pausing the workflow" if msg: _msg += f': {msg}' LOG.info(_msg) @@ -2124,18 +2159,33 @@ def resume_workflow(self, quiet: bool = False) -> None: self.workflow_db_mgr.put_workflow_paused(False) self.update_data_store() - def command_force_trigger_tasks(self, items, flow, flow_wait, flow_descr): + def command_force_trigger_tasks( + self, + tasks: Iterable[str], + flow: List[str], + flow_wait: bool = False, + flow_descr: Optional[str] = None + ): """Manual task trigger.""" return self.pool.force_trigger_tasks( - items, flow, flow_wait, flow_descr) + tasks, flow, flow_wait, flow_descr) - def command_force_spawn_children(self, items, outputs, flow_num): + def command_set( + self, + tasks: List[str], + flow: List[str], + outputs: Optional[List[str]] = None, + prerequisites: Optional[List[str]] = None, + flow_wait: bool = False, + flow_descr: Optional[str] = None + ): """Force spawn task successors. - User-facing method name: set_outputs. - + User-facing method name: reset. """ - return self.pool.force_spawn_children(items, outputs, flow_num) + return self.pool.set( # noqa: A003 + tasks, outputs, prerequisites, flow, flow_wait, flow_descr + ) def _update_profile_info(self, category, amount, amount_format="%s"): """Update the 1, 5, 15 minute dt averages for a given category.""" diff --git a/cylc/flow/scripts/completion_server.py b/cylc/flow/scripts/completion_server.py index 1677adb0d1e..da1ef14556a 100644 --- a/cylc/flow/scripts/completion_server.py +++ b/cylc/flow/scripts/completion_server.py @@ -40,6 +40,8 @@ # Which provide possible values to the completion functions. import asyncio +from contextlib import suppress +import inspect import os from pathlib import Path import select @@ -50,6 +52,7 @@ from packaging.specifiers import SpecifierSet from cylc.flow.cfgspec.glbl_cfg import glbl_cfg +from cylc.flow.exceptions import CylcError from cylc.flow.id import tokenise, IDTokens, Tokens from cylc.flow.network.scan import scan from cylc.flow.option_parsers import CylcOptionParser as COP @@ -193,7 +196,12 @@ async def complete_cylc(_root: str, *items: str) -> t.List[str]: if ret is not None: return ret if previous and previous.startswith('-'): - ret = await complete_option_value(command, previous, partial) + ret = await complete_option_value( + command, + previous, + partial, + items=items, + ) if ret is not None: return ret @@ -256,10 +264,11 @@ async def complete_option( async def complete_option_value( command: str, option: str, - partial: t.Optional[str] = None + partial: t.Optional[str] = None, + items: t.Optional[t.Iterable[str]] = None, ) -> t.Optional[t.List[str]]: """Complete values for --options.""" - vals = await list_option_values(command, option, partial) + vals = await list_option_values(command, option, partial, items=items) if vals is not None: return complete(partial, vals) return None @@ -331,9 +340,21 @@ async def list_option_values( command: str, option: str, partial: t.Optional[str] = '', + items: t.Optional[t.Iterable[str]] = None, ) -> t.Optional[t.List[str]]: """List values for an option in a Cylc command. + Args: + command: + The Cylc sub-command. + option: + The --option to list possible values for. + partial: + The part of the command the user is completing. + items: + The CLI context, i.e. everything that has been typed on the CLI + before the partial. + E.G. --flow ['all', 'new', 'none'] """ if option in OPTION_MAP: @@ -341,7 +362,22 @@ async def list_option_values( if not list_option: # do not perform completion for this option return [] - return await list_option(None, partial) + kwargs = {} + if 'tokens_list' in inspect.getfullargspec(list_option).args: + # the function requires information about tokens already specified + # on the CLI + # (e.g. the workflow//cycle/task the command is operating on) + tokens_list = [] + for item in items or []: + # pull out things from the command which look like IDs + if '//' in item: + with suppress(ValueError): + tokens_list.append(Tokens(item)) + continue + with suppress(ValueError): + tokens_list.append(Tokens(item, relative=True)) + kwargs['tokens_list'] = tokens_list + return await list_option(partial, **kwargs) return None @@ -412,7 +448,6 @@ async def list_resources(_partial: str) -> t.List[str]: async def list_dir( - _workflow: t.Optional[str], partial: t.Optional[str] ) -> t.List[str]: """List an arbitrary dir on the filesystem. @@ -459,7 +494,6 @@ def list_rel_dir(path: Path, base: Path) -> t.List[str]: async def list_flows( - _workflow: t.Optional[str], _partial: t.Optional[str] ) -> t.List[str]: """List values for the --flow option.""" @@ -467,13 +501,96 @@ async def list_flows( async def list_colours( - _workflow: t.Optional[str], _partial: t.Optional[str] ) -> t.List[str]: """List values for the --color option.""" return ['never', 'auto', 'always'] +async def list_outputs( + _partial: t.Optional[str], + tokens_list: t.Optional[t.List[Tokens]], +): + """List task outputs.""" + return (await _list_prereqs_and_outputs(tokens_list))[1] + + +async def list_prereqs( + _partial: t.Optional[str], + tokens_list: t.Optional[t.List[Tokens]], +): + """List task prerequisites.""" + return (await _list_prereqs_and_outputs(tokens_list))[0] + ['all'] + + +async def _list_prereqs_and_outputs( + tokens_list: t.Optional[t.List[Tokens]], +) -> t.Tuple[t.List[str], t.List[str]]: + """List task prerequisites and outputs. + + Returns: + tuple - (prereqs, outputs) + + """ + if not tokens_list: + # no context information available on the CLI + # we can't list prereqs/outputs + return ([], []) + + # dynamic import for this relatively unlikely case to avoid slowing down + # server startup unnecessarily + from cylc.flow.network.client_factory import get_client + from cylc.flow.scripts.show import prereqs_and_outputs_query + from types import SimpleNamespace + + workflows: t.Dict[str, t.List[Tokens]] = {} + current_workflow = None + for tokens in tokens_list: + workflow = tokens['workflow'] + task = tokens['task'] + if workflow: + workflows.setdefault(workflow, []) + current_workflow = workflow + if current_workflow and task: + workflows[current_workflow].append(tokens.task) + + clients = {} + for workflow in workflows: + with suppress(CylcError): + clients[workflow] = get_client(workflow) + + if not workflows: + return ([], []) + + json: dict = {} + await asyncio.gather(*( + prereqs_and_outputs_query( + workflow, + workflows[workflow], + pclient, + SimpleNamespace(json=True), + json, + ) + for workflow, pclient in clients.items() + )) + + if not json: + return ([], []) + return ( + [ + f"{cond['taskId']}:{cond['reqState']}" + for value in json.values() + for prerequisite in value['prerequisites'] + for cond in prerequisite['conditions'] + ], + [ + output['label'] + for value in json.values() + for output in value['outputs'] + ], + ) + + # non-exhaustive list of Cylc commands which take non-workflow arguments COMMAND_MAP: t.Dict[str, t.Optional[t.Callable]] = { # register commands which have special positional arguments @@ -512,6 +629,8 @@ async def list_colours( '--flow': list_flows, '--colour': list_colours, '--color': list_colours, + '--out': list_outputs, + '--pre': list_prereqs, # options for which we should not attempt to complete values for '--rm': None, '--run-name': None, @@ -527,7 +646,7 @@ async def list_colours( } -def cli_detokenise(tokens: Tokens) -> str: +def cli_detokenise(tokens: Tokens, relative=False) -> str: """Format tokens for use on the command line. I.E. add the trailing slash[es] onto the end. @@ -535,9 +654,13 @@ def cli_detokenise(tokens: Tokens) -> str: if tokens.is_null: # shouldn't happen but prevents possible error return '' + if relative: + id_ = tokens.relative_id + else: + id_ = tokens.id if tokens.lowest_token == IDTokens.Workflow.value: - return f'{tokens.id}//' - return f'{tokens.id}/' + return f'{id_}//' + return f'{id_}/' def next_token(tokens: Tokens) -> t.Optional[str]: diff --git a/cylc/flow/scripts/cylc.py b/cylc/flow/scripts/cylc.py index 700a70f8d7b..cabcec553e9 100644 --- a/cylc/flow/scripts/cylc.py +++ b/cylc/flow/scripts/cylc.py @@ -293,7 +293,9 @@ def get_version(long=False): 'register': 'cylc register has been removed; use cylc install or cylc play', 'reset': - 'cylc reset has been replaced by cylc set-outputs', + 'cylc reset has been replaced by cylc set', + 'set-outputs': + 'cylc set-outputs (cylc 8.0-8.2) has been replaced by cylc set', 'restart': 'cylc run & cylc restart have been replaced by cylc play', 'review': @@ -311,6 +313,8 @@ def get_version(long=False): 'cylc submit has been removed', 'start': 'cylc start & cylc restart have been replaced by cylc play', + 'set-verbosity': + 'cylc set-verbosity has been replaced by cylc verbosity', 'warranty': 'cylc warranty has been replaced by cylc help license', } diff --git a/cylc/flow/scripts/dump.py b/cylc/flow/scripts/dump.py index d3ffda59fce..1df02112730 100755 --- a/cylc/flow/scripts/dump.py +++ b/cylc/flow/scripts/dump.py @@ -18,23 +18,23 @@ """cylc dump [OPTIONS] ARGS -Print information about a running workflow. +Print current state information for a running workflow, including tasks +in the scheduler's active (n=0) window. -This command can provide information about active tasks, e.g. running or queued -tasks. For more detailed view of the workflow see `cylc tui` or `cylc gui`. +For a more detailed view of the workflow see `cylc tui` or `cylc gui`. + +For more information about a specific task, such as the current state of +its prerequisites and outputs, see 'cylc show'. For command line monitoring: * `cylc tui` * `watch cylc dump WORKFLOW_ID` works for small simple workflows -For more information about a specific task, such as the current state of -its prerequisites and outputs, see 'cylc show'. - Examples: # Display the state of all active tasks, sorted by cycle point: $ cylc dump --tasks --sort WORKFLOW_ID | grep running - # Display the state of all active in a particular cycle point: + # Display the state of all active tasks in a particular cycle point: $ cylc dump -t WORKFLOW_ID | grep 2010082406 """ diff --git a/cylc/flow/scripts/set.py b/cylc/flow/scripts/set.py new file mode 100755 index 00000000000..5de3c61d86c --- /dev/null +++ b/cylc/flow/scripts/set.py @@ -0,0 +1,315 @@ +#!/usr/bin/env python3 + +# 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 . + +"""cylc set [OPTIONS] ARGS + +Manually set task prerequisites or outputs in a running workflow. + +Default: set all required outputs (note "succeeded" can be optional). + +Prerequitistes: + Setting prerequisites contributes to a task's readiness and promotes it + to the scheduler's active window, where clock and xtriggers become active. + +Outputs: + Setting outputs affects task completion and spawns dependent tasks. + + Some implied outputs may be set automatically: + - started implies submitted + - succeeded and failed imply started + - custom outputs and expired do not imply other outputs + + Setting final outputs (succeeded, failed, expired) also sets task state. + + WARNING: setting a task's "expired" output, like clock-expire, means the + workflow can successfully complete without running the task and any + (non-expire) sub-graph that depends onit. This is dangerous, so potential + expiry must be flagged in the graph with ":expire?", for visibility. + If you expire an active task its job will be killed. + +CLI Completion: + Cylc can auto-complete prerequisites and outputs for active tasks if you + specify the task in the command before attempting TAB-completion. + +Examples: + # complete all required outputs of 3/bar: + $ cylc set my_workflow//3/bar + + # complete the succeeded output of 3/bar: + $ cylc set --out=succeeded my_workflow//3/bar + + # satisfy the `3/foo:succeeded` prerequisite of 3/bar: + $ cylc set --pre=3/foo my_workflow//3/bar + # or: + $ cylc set --pre=3/foo:succeeded my_workflow//3/bar + + # satisfy all prerequisites of 3/bar and start checking its xtriggers: + $ cylc set --pre=all my_workflow//3/bar + + # complete the ":file1" custom output of 3/bar: + $ cylc set --out=file1 my_workflow//3/bar + # or use the associated output message from the task definition: + $ cylc set --out="file 1 ready" my_workflow//3/bar + + # set multiple outputs at once: + $ cylc set --out=a --out=b,c my_workflow//3/bar + + # set multiple prerequisites at once: + $ cylc set --pre=3/foo:x --pre=3/foo:y,3/foo:z my_workflow//3/bar +""" + +from functools import partial +from typing import TYPE_CHECKING, Set, List + +from cylc.flow.exceptions import InputError +from cylc.flow.network.client_factory import get_client +from cylc.flow.network.multi import call_multi +from cylc.flow.option_parsers import ( + FULL_ID_MULTI_ARG_DOC, + CylcOptionParser as COP, +) +from cylc.flow.id import Tokens +from cylc.flow.task_outputs import TASK_OUTPUT_EXPIRED +from cylc.flow.terminal import cli_function +from cylc.flow.flow_mgr import ( + add_flow_opts, + validate_flow_opts +) + + +if TYPE_CHECKING: + from optparse import Values + + +MUTATION = ''' +mutation ( + $wFlows: [WorkflowID]!, + $tasks: [NamespaceIDGlob]!, + $prerequisites: [String], + $outputs: [OutputLabel], + $flow: [Flow!], + $flowWait: Boolean, + $flowDescr: String, +) { + set ( + workflows: $wFlows, + tasks: $tasks, + prerequisites: $prerequisites, + outputs: $outputs, + flow: $flow, + flowWait: $flowWait, + flowDescr: $flowDescr + ) { + result + } +} +''' + + +def get_option_parser() -> COP: + parser = COP( + __doc__, + comms=True, + multitask=True, + multiworkflow=True, + argdoc=[FULL_ID_MULTI_ARG_DOC], + ) + + parser.add_option( + "-o", "--out", "--output", metavar="OUTPUT(s)", + help=( + "Set task outputs complete, along with any implied outputs." + " Specify OUTPUT labels (as used in the graph) or associated" + " messages. Multiple use allowed, items may be comma separated." + ), + action="append", default=None, dest="outputs" + ) + + parser.add_option( + "-p", "--pre", "--prerequisite", metavar="PREREQUISITE(s)", + help=( + "Set task prerequisites satisfied." + " PREREQUISITE format: 'point/task:message'." + " Multiple use allowed, items may be comma separated." + " Use 'all' to satisfy all prerequisites, if any." + ), + action="append", default=None, dest="prerequisites" + ) + + add_flow_opts(parser) + return parser + + +def validate_prereq(prereq: str) -> bool: + """Return True if prereq is valid, else False. + + Examples: + >>> validate_prereq('1/foo:succeeded') + True + + >>> validate_prereq('1/foo::succeeded') + False + + >>> validate_prereq('all') + True + + >>> validate_prereq('fish') + False + + """ + try: + tokens = Tokens(prereq, relative=True) + except ValueError: + return False + if ( + tokens["cycle"] == prereq + and prereq not in ["all"] + ): + # Error: --pre= other than "all" + return False + return True + + +def split_opts(options: List[str]) -> Set[str]: + """Split comma-separated single-use options. + + Examples: + # --out='a,b,c' + >>> sorted(split_opts(['a,b,c'])) + ['a', 'b', 'c'] + + # --out='a' --out='a,b' + >>> sorted(split_opts(['a', 'b,c'])) + ['a', 'b', 'c'] + + # --out='a' --out='a,b' + >>> sorted(split_opts(['a', 'a,b'])) + ['a', 'b'] + + """ + if options is None: + return [] + splat: Set[str] = set() # (past tense of split) + for p in options: + splat.update(p.split(',')) + return splat + + +def get_prereq_opts(prereq_options: List[str]) -> Set[str]: + """Validate prerequisite inputs and return them as a flat set. + + Examples: + >>> sorted(get_prereq_opts(['1/foo:bar', '2/foo:baz,3/foo:qux'])) + ['1/foo:bar', '2/foo:baz', '3/foo:qux'] + + >>> get_prereq_opts(['all']) + {'all'} + + >>> get_prereq_opts(['fish']) + Traceback (most recent call last): + ... + InputError: + + >>> get_prereq_opts(['all', '2/foo:baz']) + Traceback (most recent call last): + ... + InputError: + + >>> get_prereq_opts(['1/foo::bar']) + Traceback (most recent call last): + ... + InputError: + + """ + prereqs = split_opts(prereq_options) + if not prereqs: + return prereqs + + msg = '\n'.join( + [ + p for p in prereqs + if not validate_prereq(p) + ] + ) + if msg: + raise InputError(f"Invalid prerequisite(s):\n{msg}") + + if len(prereqs) > 1: # noqa SIM102 (anticipates "cylc set --pre=cycle") + if "all" in prereqs: + raise InputError("--pre=all must be used alone") + + return prereqs + + +def get_output_opts(output_options: List[str]) -> Set[str]: + """Convert outputs options to a single set, and validate. + + Examples: + >>> sorted(get_output_opts(['a', 'b,c'])) + ['a', 'b', 'c'] + + >>> get_output_opts(['expire', 'expired']) + {'expired'} + + """ + outputs = set() + for output in split_opts(output_options): + if output == "expire": + # should be "expired"; and "expire" is not a legal output. + outputs.add(TASK_OUTPUT_EXPIRED) + else: + outputs.add(output) + return outputs + + +async def run( + options: 'Values', + workflow_id: str, + *tokens_list +) -> None: + + pclient = get_client(workflow_id, timeout=options.comms_timeout) + + mutation_kwargs = { + 'request_string': MUTATION, + 'variables': { + 'wFlows': [workflow_id], + 'tasks': [ + tokens.relative_id_with_selectors + for tokens in tokens_list + ], + 'outputs': list(get_output_opts(options.outputs)), + 'prerequisites': list(get_prereq_opts(options.prerequisites)), + 'flow': options.flow, + 'flowWait': options.flow_wait, + 'flowDescr': options.flow_descr + } + } + + await pclient.async_request('graphql', mutation_kwargs) + + +@cli_function(get_option_parser) +def main(parser: COP, options: 'Values', *ids) -> None: + if options.outputs and options.prerequisites: + raise InputError("Use --prerequisite or --output, not both.") + validate_flow_opts(options) + call_multi( + partial(run, options), + *ids, + ) diff --git a/cylc/flow/scripts/set_outputs.py b/cylc/flow/scripts/set_outputs.py deleted file mode 100755 index b45ddbff268..00000000000 --- a/cylc/flow/scripts/set_outputs.py +++ /dev/null @@ -1,129 +0,0 @@ -#!/usr/bin/env python3 - -# 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 . - -"""cylc set-outputs [OPTIONS] ARGS - -Artificially satisfy task outputs. - -Mark task outputs as completed and spawn downstream tasks that depend on those -outputs. By default it marks tasks as succeeded. - -This allows you to manually intervene with Cylc's scheduling algorithm by -artificially satisfying outputs of tasks. - -If a flow number is given, the child tasks will start (or continue) that flow. - -Examples: - # For example, for the following dependency graph: - R1 = ''' - a => b & c => d - foo:x => bar => baz - ''' - - # spawn 1/b and 1/c, but 1/d will not subsequently run - $ cylc set-outputs my_workflow//1/a - - # spawn 1/b and 1/c as flow 2, followed by 1/d - $ cylc set-outputs --flow=2 my_workflow//1/a - - # spawn 1/bar as flow 3, followed by 1/baz - $ cylc set-outputs --flow=3 --output=x my_workflow//1/foo - -Use --output multiple times to spawn off of several outputs at once. -""" - -from functools import partial -from typing import TYPE_CHECKING - -from cylc.flow.network.client_factory import get_client -from cylc.flow.network.multi import call_multi -from cylc.flow.option_parsers import ( - FULL_ID_MULTI_ARG_DOC, - CylcOptionParser as COP, -) -from cylc.flow.terminal import cli_function - -if TYPE_CHECKING: - from optparse import Values - -MUTATION = ''' -mutation ( - $wFlows: [WorkflowID]!, - $tasks: [NamespaceIDGlob]!, - $outputs: [String], - $flowNum: Int, -) { - setOutputs ( - workflows: $wFlows, - tasks: $tasks, - outputs: $outputs, - flowNum: $flowNum, - ) { - result - } -} -''' - - -def get_option_parser() -> COP: - parser = COP( - __doc__, - comms=True, - multitask=True, - multiworkflow=True, - argdoc=[FULL_ID_MULTI_ARG_DOC], - ) - - parser.add_option( - "-o", "--output", metavar="OUTPUT", - help="Set OUTPUT (default \"succeeded\") completed.", - action="append", default=None, dest="outputs") - - parser.add_option( - "-f", "--flow", metavar="FLOW", - help="Number of the flow to attribute the outputs.", - action="store", default=None, dest="flow_num") - - return parser - - -async def run(options: 'Values', workflow_id: str, *tokens_list) -> None: - pclient = get_client(workflow_id, timeout=options.comms_timeout) - - mutation_kwargs = { - 'request_string': MUTATION, - 'variables': { - 'wFlows': [workflow_id], - 'tasks': [ - tokens.relative_id_with_selectors - for tokens in tokens_list - ], - 'outputs': options.outputs, - 'flowNum': options.flow_num - } - } - - await pclient.async_request('graphql', mutation_kwargs) - - -@cli_function(get_option_parser) -def main(parser: COP, options: 'Values', *ids) -> None: - call_multi( - partial(run, options), - *ids, - ) diff --git a/cylc/flow/scripts/trigger.py b/cylc/flow/scripts/trigger.py index 3e4a3da96f7..9d27fc38bd1 100755 --- a/cylc/flow/scripts/trigger.py +++ b/cylc/flow/scripts/trigger.py @@ -44,7 +44,6 @@ from functools import partial from typing import TYPE_CHECKING -from cylc.flow.exceptions import InputError from cylc.flow.network.client_factory import get_client from cylc.flow.network.multi import call_multi from cylc.flow.option_parsers import ( @@ -52,20 +51,16 @@ CylcOptionParser as COP, ) from cylc.flow.terminal import cli_function -from cylc.flow.flow_mgr import FLOW_NONE, FLOW_NEW, FLOW_ALL +from cylc.flow.flow_mgr import ( + add_flow_opts, + validate_flow_opts +) + if TYPE_CHECKING: from optparse import Values -ERR_OPT_FLOW_VAL = "Flow values must be integer, 'all', 'new', or 'none'" -ERR_OPT_FLOW_INT = "Multiple flow options must all be integer valued" -ERR_OPT_FLOW_META = "Metadata is only for new flows" -ERR_OPT_FLOW_WAIT = ( - f"--wait is not compatible with --flow={FLOW_NEW} or --flow={FLOW_NONE}" -) - - MUTATION = ''' mutation ( $wFlows: [WorkflowID]!, @@ -95,49 +90,10 @@ def get_option_parser() -> COP: multiworkflow=True, argdoc=[FULL_ID_MULTI_ARG_DOC], ) - - parser.add_option( - "--flow", action="append", dest="flow", metavar="FLOW", - help=f"Assign the triggered task to all active flows ({FLOW_ALL});" - f" no flow ({FLOW_NONE}); a new flow ({FLOW_NEW});" - f" or a specific flow (e.g. 2). The default is {FLOW_ALL}." - " Reuse the option to assign multiple specific flows." - ) - - parser.add_option( - "--meta", metavar="DESCRIPTION", action="store", - dest="flow_descr", default=None, - help=f"description of triggered flow (with --flow={FLOW_NEW})." - ) - - parser.add_option( - "--wait", action="store_true", default=False, dest="flow_wait", - help="Wait for merge with current active flows before flowing on." - ) - + add_flow_opts(parser) return parser -def _validate(options): - """Check validity of flow-related options.""" - for val in options.flow: - val = val.strip() - if val in [FLOW_NONE, FLOW_NEW, FLOW_ALL]: - if len(options.flow) != 1: - raise InputError(ERR_OPT_FLOW_INT) - else: - try: - int(val) - except ValueError: - raise InputError(ERR_OPT_FLOW_VAL.format(val)) - - if options.flow_descr and options.flow != [FLOW_NEW]: - raise InputError(ERR_OPT_FLOW_META) - - if options.flow_wait and options.flow[0] in [FLOW_NEW, FLOW_NONE]: - raise InputError(ERR_OPT_FLOW_WAIT) - - async def run(options: 'Values', workflow_id: str, *tokens_list): pclient = get_client(workflow_id, timeout=options.comms_timeout) @@ -154,18 +110,13 @@ async def run(options: 'Values', workflow_id: str, *tokens_list): 'flowDescr': options.flow_descr, } } - await pclient.async_request('graphql', mutation_kwargs) @cli_function(get_option_parser) def main(parser: COP, options: 'Values', *ids: str): """CLI for "cylc trigger".""" - - if options.flow is None: - options.flow = [FLOW_ALL] # default to all active flows - _validate(options) - + validate_flow_opts(options) call_multi( partial(run, options), *ids, diff --git a/cylc/flow/scripts/set_verbosity.py b/cylc/flow/scripts/verbosity.py similarity index 90% rename from cylc/flow/scripts/set_verbosity.py rename to cylc/flow/scripts/verbosity.py index aa40d01345c..0479625c951 100755 --- a/cylc/flow/scripts/set_verbosity.py +++ b/cylc/flow/scripts/verbosity.py @@ -16,13 +16,12 @@ # You should have received a copy of the GNU General Public License # along with this program. If not, see . -"""cylc set-verbosity [OPTIONS] ARGS +"""cylc verbosity [OPTIONS] ARGS -Change the logging severity level of a running workflow. +Set the logging severity level of a running scheduler. -Only messages at or above the chosen severity level will be logged; for -example, if you choose WARNING, only warnings and critical messages will be -logged. +Messages at or above the chosen level are logged. If you choose +WARNING (say) only WARNING and CRITICAL messages will be logged. """ from functools import partial diff --git a/cylc/flow/task_events_mgr.py b/cylc/flow/task_events_mgr.py index 2ff2d7e44d1..ae1dd757f63 100644 --- a/cylc/flow/task_events_mgr.py +++ b/cylc/flow/task_events_mgr.py @@ -76,12 +76,18 @@ TASK_STATUS_SUBMIT_FAILED, TASK_STATUS_RUNNING, TASK_STATUS_FAILED, + TASK_STATUS_EXPIRED, TASK_STATUS_SUCCEEDED, TASK_STATUS_WAITING ) from cylc.flow.task_outputs import ( - TASK_OUTPUT_SUBMITTED, TASK_OUTPUT_STARTED, TASK_OUTPUT_SUCCEEDED, - TASK_OUTPUT_FAILED, TASK_OUTPUT_SUBMIT_FAILED) + TASK_OUTPUT_EXPIRED, + TASK_OUTPUT_SUBMITTED, + TASK_OUTPUT_STARTED, + TASK_OUTPUT_SUCCEEDED, + TASK_OUTPUT_FAILED, + TASK_OUTPUT_SUBMIT_FAILED +) from cylc.flow.wallclock import ( get_current_time_string, get_seconds_as_interval_string as intvl_as_str @@ -183,11 +189,11 @@ def log_task_job_activity(ctx, workflow, point, name, submit_num=None): try: with open(os.path.expandvars(job_activity_log), "ab") as handle: handle.write((ctx_str + '\n').encode()) - except IOError as exc: - # This happens when there is no job directory, e.g. if job host - # selection command causes an submission failure, there will be no job - # directory. In this case, just send the information to the log. - LOG.exception(exc) + except IOError: + # This happens when there is no job directory. E.g., if a job host + # selection command causes a submission failure, or if a waiting task + # expires before a job log directory is otherwise needed. + # (Don't log the exception content, it looks like a bug). LOG.info(ctx_str) if ctx.cmd and ctx.ret_code: LOG.error(ctx_str) @@ -404,6 +410,7 @@ class TaskEventsManager(): EVENT_RETRY = "retry" EVENT_STARTED = TASK_OUTPUT_STARTED EVENT_SUBMITTED = TASK_OUTPUT_SUBMITTED + EVENT_EXPIRED = TASK_OUTPUT_EXPIRED EVENT_SUBMIT_FAILED = "submission failed" EVENT_SUBMIT_RETRY = "submission retry" EVENT_SUCCEEDED = TASK_OUTPUT_SUCCEEDED @@ -606,6 +613,7 @@ def process_message( event_time: Optional[str] = None, flag: str = FLAG_INTERNAL, submit_num: Optional[int] = None, + forced: bool = False ) -> Optional[bool]: """Parse a task message and update task state. @@ -643,13 +651,14 @@ def process_message( submit_num: The submit number of the task relevant for the message. If not specified, use latest submit number. + forced: + If this message is due to manual completion or not (cylc set) Return: None: in normal circumstances. True: if polling is required to confirm a reversal of status. """ - # Log messages if event_time is None: event_time = get_current_time_string() @@ -663,7 +672,8 @@ def process_message( self.reset_inactivity_timer_func() if not self._process_message_check( - itask, severity, message, event_time, flag, submit_num): + itask, severity, message, event_time, flag, submit_num, forced + ): return None # always update the workflow state summary for latest message @@ -679,39 +689,48 @@ def process_message( # Satisfy my output, if possible, and spawn children. # (first remove signal: failed/EXIT -> failed) + # Complete the corresponding task output, if there is one. msg0 = message.split('/')[0] - completed_trigger = itask.state.outputs.set_msg_trg_completion( - message=msg0, is_completed=True) - self.data_store_mgr.delta_task_output(itask, msg0) - - # Check the `started` event has not been missed e.g. due to - # polling delay - if (message not in [self.EVENT_SUBMITTED, self.EVENT_SUBMIT_FAILED, - self.EVENT_STARTED] - and not itask.state.outputs.is_completed(TASK_OUTPUT_STARTED)): - self.setup_event_handlers( - itask, self.EVENT_STARTED, f'job {self.EVENT_STARTED}') - self.spawn_func(itask, TASK_OUTPUT_STARTED) + if message.startswith(ABORT_MESSAGE_PREFIX): + msg0 = TASK_OUTPUT_FAILED + + completed_output = None + if msg0 not in [TASK_OUTPUT_SUBMIT_FAILED, TASK_OUTPUT_FAILED]: + completed_output = itask.state.outputs.set_msg_trg_completion( + message=msg0, is_completed=True) + if completed_output: + self.data_store_mgr.delta_task_output(itask, msg0) + + for implied in ( + itask.state.outputs.get_incomplete_implied(msg0) + ): + # Set submitted and/or started first, if skipped. + # (whether by forced set, or missed message). + LOG.warning( + f"[{itask}] setting missed output: {implied}") + self.process_message( + itask, INFO, implied, event_time, + self.FLAG_INTERNAL, submit_num + ) + if message == self.EVENT_STARTED: if ( flag == self.FLAG_RECEIVED and itask.state.is_gt(TASK_STATUS_RUNNING) ): + # Already running. return True - if itask.state.status == TASK_STATUS_PREPARING: - # The started message must have arrived before the submitted - # one, so assume that a successful submission occurred and act - # accordingly. Note the submitted message is internal, whereas - # the started message comes in on the network. - self._process_message_submitted(itask, event_time) - self.spawn_func(itask, TASK_OUTPUT_SUBMITTED) - - self._process_message_started(itask, event_time) - self.spawn_func(itask, TASK_OUTPUT_STARTED) + self._process_message_started(itask, event_time, forced) + self.spawn_children(itask, TASK_OUTPUT_STARTED) elif message == self.EVENT_SUCCEEDED: - self._process_message_succeeded(itask, event_time) - self.spawn_func(itask, TASK_OUTPUT_SUCCEEDED) + self._process_message_succeeded(itask, event_time, forced) + self.spawn_children(itask, TASK_OUTPUT_SUCCEEDED) + + elif message == self.EVENT_EXPIRED: + self._process_message_expired(itask, event_time, forced) + self.spawn_children(itask, TASK_OUTPUT_EXPIRED) + elif message == self.EVENT_FAILED: if ( flag == self.FLAG_RECEIVED @@ -719,8 +738,10 @@ def process_message( ): return True if self._process_message_failed( - itask, event_time, self.JOB_FAILED): - self.spawn_func(itask, TASK_OUTPUT_FAILED) + itask, event_time, self.JOB_FAILED, forced + ): + self.spawn_children(itask, TASK_OUTPUT_FAILED) + elif message == self.EVENT_SUBMIT_FAILED: if ( flag == self.FLAG_RECEIVED @@ -728,11 +749,10 @@ def process_message( ): return True if self._process_message_submit_failed( - itask, - event_time, - submit_num + itask, event_time, submit_num, forced ): - self.spawn_func(itask, TASK_OUTPUT_SUBMIT_FAILED) + self.spawn_children(itask, TASK_OUTPUT_SUBMIT_FAILED) + elif message == self.EVENT_SUBMITTED: if ( flag == self.FLAG_RECEIVED @@ -746,8 +766,8 @@ def process_message( # If not in the preparing state we already assumed and handled # job submission under the started event above... # (sim mode does not have the job prep state) - self._process_message_submitted(itask, event_time) - self.spawn_func(itask, TASK_OUTPUT_SUBMITTED) + self._process_message_submitted(itask, event_time, forced) + self.spawn_children(itask, TASK_OUTPUT_SUBMITTED) # ... but either way update the job ID in the job proxy (it only # comes in via the submission message). @@ -770,8 +790,10 @@ def process_message( self.workflow_db_mgr.put_update_task_jobs( itask, {"run_signal": signal}) if self._process_message_failed( - itask, event_time, self.JOB_FAILED): - self.spawn_func(itask, TASK_OUTPUT_FAILED) + itask, event_time, self.JOB_FAILED, forced + ): + self.spawn_children(itask, TASK_OUTPUT_FAILED) + elif message.startswith(ABORT_MESSAGE_PREFIX): # Task aborted with message if ( @@ -783,8 +805,11 @@ def process_message( self._db_events_insert(itask, "aborted", message) self.workflow_db_mgr.put_update_task_jobs( itask, {"run_signal": aborted_with}) - if self._process_message_failed(itask, event_time, aborted_with): - self.spawn_func(itask, TASK_OUTPUT_FAILED) + if self._process_message_failed( + itask, event_time, aborted_with, forced + ): + self.spawn_children(itask, TASK_OUTPUT_FAILED) + elif message.startswith(VACATION_MESSAGE_PREFIX): # Task job pre-empted into a vacation state self._db_events_insert(itask, "vacated", message) @@ -793,8 +818,8 @@ def process_message( itask.try_timers[TimerFlags.SUBMISSION_RETRY].num = 0 itask.job_vacated = True # Believe this and change state without polling (could poll?). - if itask.state_reset(TASK_STATUS_SUBMITTED): - itask.state_reset(is_queued=False) + if itask.state_reset(TASK_STATUS_SUBMITTED, forced=forced): + itask.state_reset(is_queued=False, forced=forced) self.data_store_mgr.delta_task_state(itask) self.data_store_mgr.delta_task_queued(itask) self._reset_job_timers(itask) @@ -803,11 +828,13 @@ def process_message( # system, we should probably aim to remove support for job vacation # instead. Otherwise, we should have: # self.setup_event_handlers(itask, 'vacated', message) - elif completed_trigger: - # Message of an as-yet unreported custom task output. + + elif completed_output: + # Message of a custom task output. # No state change. - self.setup_event_handlers(itask, completed_trigger, message) - self.spawn_func(itask, msg0) + self.setup_event_handlers(itask, completed_output, message) + self.spawn_children(itask, msg0) + else: # Unhandled messages. These include: # * general non-output/progress messages @@ -817,9 +844,11 @@ def process_message( LOG.debug(f"[{itask}] unhandled: {message}") self._db_events_insert( itask, (f"message {lseverity}"), message) + if lseverity in self.NON_UNIQUE_EVENTS: itask.non_unique_events.update({lseverity: 1}) self.setup_event_handlers(itask, lseverity, message) + return None def _process_message_check( @@ -830,13 +859,17 @@ def _process_message_check( event_time: str, flag: str, submit_num: int, + forced: bool = False ) -> bool: """Helper for `.process_message`. See `.process_message` for argument list Check whether to process/skip message. - Return True if `.process_message` should contine, False otherwise. + Return True if `.process_message` should continue, False otherwise. """ + if itask.transient or forced: + return True + if self.timestamp: timestamp = f" at {event_time}" else: @@ -1253,7 +1286,7 @@ def _retry_task(self, itask, wallclock_time, submit_retry=False): if itask.state_reset(TASK_STATUS_WAITING): self.data_store_mgr.delta_task_state(itask) - def _process_message_failed(self, itask, event_time, message): + def _process_message_failed(self, itask, event_time, message, forced): """Helper for process_message, handle a failed message. Return True if no retries (hence go to the failed state). @@ -1270,14 +1303,20 @@ def _process_message_failed(self, itask, event_time, message): "time_run_exit": event_time, }) if ( - TimerFlags.EXECUTION_RETRY not in itask.try_timers + forced + or TimerFlags.EXECUTION_RETRY not in itask.try_timers or itask.try_timers[TimerFlags.EXECUTION_RETRY].next() is None ): # No retry lined up: definitive failure. - if itask.state_reset(TASK_STATUS_FAILED): + no_retries = True + if itask.state_reset(TASK_STATUS_FAILED, forced=forced): self.setup_event_handlers(itask, self.EVENT_FAILED, message) self.data_store_mgr.delta_task_state(itask) - no_retries = True + itask.state.outputs.set_msg_trg_completion( + message=TASK_OUTPUT_FAILED, is_completed=True) + self.data_store_mgr.delta_task_output( + itask, TASK_OUTPUT_FAILED) + self.data_store_mgr.delta_task_state(itask) else: # There is an execution retry lined up. timer = itask.try_timers[TimerFlags.EXECUTION_RETRY] @@ -1289,7 +1328,7 @@ def _process_message_failed(self, itask, event_time, message): self._reset_job_timers(itask) return no_retries - def _process_message_started(self, itask, event_time): + def _process_message_started(self, itask, event_time, forced): """Helper for process_message, handle a started message.""" if itask.job_vacated: itask.job_vacated = False @@ -1300,7 +1339,7 @@ def _process_message_started(self, itask, event_time): itask.set_summary_time('started', event_time) self.workflow_db_mgr.put_update_task_jobs(itask, { "time_run": itask.summary['started_time_string']}) - if itask.state_reset(TASK_STATUS_RUNNING): + if itask.state_reset(TASK_STATUS_RUNNING, forced=forced): self.setup_event_handlers( itask, self.EVENT_STARTED, f'job {self.EVENT_STARTED}') self.data_store_mgr.delta_task_state(itask) @@ -1310,8 +1349,23 @@ def _process_message_started(self, itask, event_time): if TimerFlags.SUBMISSION_RETRY in itask.try_timers: itask.try_timers[TimerFlags.SUBMISSION_RETRY].num = 0 - def _process_message_succeeded(self, itask, event_time): - """Helper for process_message, handle a succeeded message.""" + def _process_message_expired(self, itask, event_time, forced): + """Helper for process_message, handle task expiry.""" + if not itask.state_reset(TASK_STATUS_EXPIRED, forced=forced): + return + self.data_store_mgr.delta_task_state(itask) + self.data_store_mgr.delta_task_queued(itask) + self.setup_event_handlers( + itask, + self.EVENT_EXPIRED, + "Task expired: will not submit job." + ) + + def _process_message_succeeded(self, itask, event_time, forced): + """Helper for process_message, handle a succeeded message. + + Ignore forced. + """ job_tokens = itask.tokens.duplicate(job=str(itask.submit_num)) self.data_store_mgr.delta_job_time(job_tokens, 'finished', event_time) @@ -1326,13 +1380,15 @@ def _process_message_succeeded(self, itask, event_time): itask.tdef.elapsed_times.append( itask.summary['finished_time'] - itask.summary['started_time']) - if itask.state_reset(TASK_STATUS_SUCCEEDED): + if itask.state_reset(TASK_STATUS_SUCCEEDED, forced=forced): self.setup_event_handlers( itask, self.EVENT_SUCCEEDED, f"job {self.EVENT_SUCCEEDED}") self.data_store_mgr.delta_task_state(itask) self._reset_job_timers(itask) - def _process_message_submit_failed(self, itask, event_time, submit_num): + def _process_message_submit_failed( + self, itask, event_time, submit_num, forced + ): """Helper for process_message, handle a submit-failed message. Return True if no retries (hence go to the submit-failed state). @@ -1347,16 +1403,21 @@ def _process_message_submit_failed(self, itask, event_time, submit_num): }) itask.summary['submit_method_id'] = None if ( - TimerFlags.SUBMISSION_RETRY not in itask.try_timers + forced + or TimerFlags.SUBMISSION_RETRY not in itask.try_timers or itask.try_timers[TimerFlags.SUBMISSION_RETRY].next() is None ): # No submission retry lined up: definitive failure. # See github #476. no_retries = True - if itask.state_reset(TASK_STATUS_SUBMIT_FAILED): + if itask.state_reset(TASK_STATUS_SUBMIT_FAILED, forced=forced): self.setup_event_handlers( itask, self.EVENT_SUBMIT_FAILED, f'job {self.EVENT_SUBMIT_FAILED}') + itask.state.outputs.set_msg_trg_completion( + message=TASK_OUTPUT_SUBMIT_FAILED, is_completed=True) + self.data_store_mgr.delta_task_output( + itask, TASK_OUTPUT_SUBMIT_FAILED) self.data_store_mgr.delta_task_state(itask) else: # There is a submission retry lined up. @@ -1374,13 +1435,12 @@ def _process_message_submit_failed(self, itask, event_time, submit_num): job_tokens, TASK_STATUS_SUBMIT_FAILED ) - self._reset_job_timers(itask) return no_retries def _process_message_submitted( - self, itask: 'TaskProxy', event_time: str + self, itask: 'TaskProxy', event_time: str, forced: bool ) -> None: """Helper for process_message, handle a submit-succeeded message.""" with suppress(KeyError): @@ -1396,10 +1456,11 @@ def _process_message_submitted( if itask.tdef.run_mode == 'simulation': # Simulate job started as well. itask.set_summary_time('started', event_time) - if itask.state_reset(TASK_STATUS_RUNNING): + if itask.state_reset(TASK_STATUS_RUNNING, forced=forced): self.data_store_mgr.delta_task_state(itask) itask.state.outputs.set_completion(TASK_OUTPUT_STARTED, True) self.data_store_mgr.delta_task_output(itask, TASK_OUTPUT_STARTED) + else: # Unset started and finished times in case of resubmission. itask.set_summary_time('started') @@ -1408,8 +1469,8 @@ def _process_message_submitted( # The job started message can (rarely) come in before the # submit command returns - in which case do not go back to # 'submitted'. - if itask.state_reset(TASK_STATUS_SUBMITTED): - itask.state_reset(is_queued=False) + if itask.state_reset(TASK_STATUS_SUBMITTED, forced=forced): + itask.state_reset(is_queued=False, forced=forced) self.setup_event_handlers( itask, self.EVENT_SUBMITTED, @@ -1458,7 +1519,10 @@ def _insert_task_job( # itask.jobs appends for automatic retries (which reuse the same task # proxy) but a retriggered task that was not already in the pool will # not see previous submissions (so can't use itask.jobs[submit_num-1]). - job_conf = itask.jobs[-1] + if itask.tdef.run_mode == "simulation": + job_conf = {"submit_num": 0} + else: + job_conf = itask.jobs[-1] # insert job into data store self.data_store_mgr.insert_job( @@ -1713,6 +1777,10 @@ def _get_handler_template_variables( def _reset_job_timers(self, itask): """Set up poll timer and timeout for task.""" + + if itask.transient: + return + if not itask.state(*TASK_STATUSES_ACTIVE): # Reset, task not active itask.timeout = None @@ -1883,3 +1951,9 @@ def reset_bad_hosts(self): f'{self.bad_hosts}' ) self.bad_hosts.clear() + + def spawn_children(self, itask, output): + # update DB task outputs + self.workflow_db_mgr.put_update_task_outputs(itask) + # spawn child-tasks + self.spawn_func(itask, output) diff --git a/cylc/flow/task_job_mgr.py b/cylc/flow/task_job_mgr.py index 6651488b4b9..49590245e38 100644 --- a/cylc/flow/task_job_mgr.py +++ b/cylc/flow/task_job_mgr.py @@ -25,6 +25,7 @@ """ from contextlib import suppress +from functools import partial import json import os from logging import ( @@ -85,7 +86,8 @@ TASK_OUTPUT_FAILED, TASK_OUTPUT_STARTED, TASK_OUTPUT_SUBMITTED, - TASK_OUTPUT_SUCCEEDED + TASK_OUTPUT_SUCCEEDED, + TASK_OUTPUT_EXPIRED ) from cylc.flow.task_remote_mgr import ( REMOTE_FILE_INSTALL_DONE, @@ -174,12 +176,25 @@ def check_task_jobs(self, workflow, task_pool): if poll_tasks: self.poll_task_jobs(workflow, poll_tasks) - def kill_task_jobs(self, workflow, itasks): + def kill_task_jobs(self, itasks, expire=False): """Kill jobs of active tasks, and hold the tasks. - If items is specified, kill active tasks matching given IDs. + if expire: expire tasks (in the callback) after killing them. """ + ok = True + if expire: + # Check these tasks are allowed to expire. + output = TASK_OUTPUT_EXPIRED + for itask in itasks: + msg = itask.state.outputs.get_msg(output) + info = f'set: output {itask.identity}:{output}' + if msg is None: + LOG.warning(f"{info} not found") + ok = False + if not ok: + return + to_kill_tasks = [] for itask in itasks: if itask.state(*TASK_STATUSES_ACTIVE): @@ -188,9 +203,10 @@ def kill_task_jobs(self, workflow, itasks): to_kill_tasks.append(itask) else: LOG.warning(f"[{itask}] not killable") + callback = partial(self._kill_task_jobs_callback, expire=expire) self._run_job_cmd( - self.JOBS_KILL, workflow, to_kill_tasks, - self._kill_task_jobs_callback, + self.JOBS_KILL, self.workflow, to_kill_tasks, + callback, self._kill_task_jobs_callback_255 ) @@ -235,6 +251,8 @@ def prep_submit_task_jobs(self, workflow, itasks, check_syntax=True): # state transition message reflects the correct submit_num itask.submit_num += 1 itask.state_reset(TASK_STATUS_PREPARING) + # Avoid clock-expiring an active task: + itask.expire_time = None self.data_store_mgr.delta_task_state(itask) prep_task = self._prep_submit_task_job( workflow, itask, check_syntax=check_syntax) @@ -638,8 +656,12 @@ def _job_cmd_out_callback(workflow, itask, cmd_ctx, line): LOG.warning("%s: write failed\n%s" % (job_activity_log, exc)) LOG.warning(f"[{itask}] {host}{line}") - def _kill_task_jobs_callback(self, ctx, workflow, itasks): - """Callback when kill tasks command exits.""" + def _kill_task_jobs_callback(self, ctx, workflow, itasks, expire=False): + """Callback when kill tasks command exits. + + If expire: expire the killed tasks. + + """ self._manip_task_jobs_callback( ctx, workflow, @@ -648,6 +670,12 @@ def _kill_task_jobs_callback(self, ctx, workflow, itasks): {self.job_runner_mgr.OUT_PREFIX_COMMAND: self._job_cmd_out_callback} ) + if expire: + for itask in itasks: + self.task_events_mgr.process_message( + itask, INFO, TASK_OUTPUT_EXPIRED, forced=True) + info = f'set: output {itask.identity}:{TASK_OUTPUT_EXPIRED}' + LOG.info(f"{info} completed") def _kill_task_jobs_callback_255(self, ctx, workflow, itasks): """Callback when kill tasks command exits.""" @@ -668,7 +696,7 @@ def _kill_task_job_callback_255(self, workflow, itask, cmd_ctx, line): itask.platform, bad_hosts=self.task_remote_mgr.bad_hosts ) - self.kill_task_jobs(workflow, [itask]) + self.kill_task_jobs([itask]) def _kill_task_job_callback(self, workflow, itask, cmd_ctx, line): """Helper for _kill_task_jobs_callback, on one task job.""" diff --git a/cylc/flow/task_outputs.py b/cylc/flow/task_outputs.py index 4da21f62722..597a4145d76 100644 --- a/cylc/flow/task_outputs.py +++ b/cylc/flow/task_outputs.py @@ -15,6 +15,7 @@ # along with this program. If not, see . """Task output message manager and constants.""" +from typing import List # Standard task output strings, used for triggering. TASK_OUTPUT_EXPIRED = "expired" @@ -31,7 +32,8 @@ TASK_OUTPUT_SUBMIT_FAILED, TASK_OUTPUT_STARTED, TASK_OUTPUT_SUCCEEDED, - TASK_OUTPUT_FAILED) + TASK_OUTPUT_FAILED +) TASK_OUTPUTS = ( TASK_OUTPUT_EXPIRED, @@ -43,6 +45,14 @@ TASK_OUTPUT_FINISHED, ) +STANDARD_OUTPUTS = ( # omits expired: requires graph trigger + TASK_OUTPUT_SUBMITTED, + TASK_OUTPUT_SUBMIT_FAILED, + TASK_OUTPUT_STARTED, + TASK_OUTPUT_SUCCEEDED, + TASK_OUTPUT_FAILED +) + _TRIGGER = 0 _MESSAGE = 1 _IS_COMPLETED = 2 @@ -69,7 +79,7 @@ class TaskOutputs: def __init__(self, tdef): self._by_message = {} self._by_trigger = {} - self._required = set() + self._required = {} # trigger: message # Add outputs from task def. for trigger, (message, required) in tdef.outputs.items(): @@ -93,7 +103,7 @@ def _add(self, message, trigger, is_completed=False, required=False): self._by_message[message] = [trigger, message, is_completed] self._by_trigger[trigger] = self._by_message[message] if required: - self._required.add(trigger) + self._required[trigger] = message def set_completed_by_msg(self, message): """For flow trigger --wait: set completed outputs from the DB.""" @@ -114,9 +124,21 @@ def exists(self, message=None, trigger=None): return False def get_all(self): - """Return an iterator for all outputs.""" + """Return an iterator for all output messages.""" return sorted(self._by_message.values(), key=self.msg_sort_key) + def get_msg(self, out): + """Translate a message or label into message, or None if not valid.""" + if out in self._by_message: + # It's already a valid message. + return out + elif out in self._by_trigger: + # It's a valid trigger label, return the message. + return (self._by_trigger[out])[1] + else: + # Not a valid message or trigger label. + return None + def get_completed(self): """Return all completed output messages.""" ret = [] @@ -140,6 +162,16 @@ def has_custom_triggers(self): """Return True if it has any custom triggers.""" return any(key not in SORT_ORDERS for key in self._by_trigger) + def _get_custom_triggers(self, required: bool = False) -> List[str]: + """Return list of all, or required, custom trigger messages.""" + custom = [ + out[1] for trg, out in self._by_trigger.items() + if trg not in SORT_ORDERS + ] + if required: + custom = [out for out in custom if out in self._required.values()] + return custom + def get_not_completed(self): """Return all not-completed output messages.""" ret = [] @@ -251,9 +283,33 @@ def _get_item(self, message, trigger): else: return self._by_message[message] + def get_incomplete_implied(self, output: str) -> List[str]: + """Return an ordered list of incomplete implied outputs. + + Use to determined implied outputs to complete automatically. + + Implied outputs are necessarily earlier outputs. + + - started implies submitted + - succeeded and failed imply started + - custom outputs and expired do not imply other outputs + + """ + implied: List[str] = [] + + if output in [TASK_OUTPUT_SUCCEEDED, TASK_OUTPUT_FAILED]: + # Finished, so it must have submitted and started. + implied = [TASK_OUTPUT_SUBMITTED, TASK_OUTPUT_STARTED] + + elif output == TASK_OUTPUT_STARTED: + # It must have submitted. + implied = [TASK_OUTPUT_SUBMITTED] + + return [out for out in implied if not self.is_completed(out)] + @staticmethod def is_valid_std_name(name): - """Check name is a valid standard output name.""" + """Check name is a valid standard output (including 'expired').""" return name in SORT_ORDERS @staticmethod diff --git a/cylc/flow/task_pool.py b/cylc/flow/task_pool.py index a13ba34d3cb..b0b69cac83e 100644 --- a/cylc/flow/task_pool.py +++ b/cylc/flow/task_pool.py @@ -19,7 +19,6 @@ from contextlib import suppress from collections import Counter import json -from time import time from typing import ( Dict, Iterable, @@ -59,11 +58,7 @@ TASK_STATUS_SUBMITTED, TASK_STATUS_RUNNING, TASK_STATUS_SUCCEEDED, - TASK_STATUS_FAILED, - TASK_OUTPUT_EXPIRED, - TASK_OUTPUT_FAILED, - TASK_OUTPUT_SUCCEEDED, - TASK_OUTPUT_SUBMIT_FAILED, + TASK_STATUS_FAILED ) from cylc.flow.util import ( serialise, @@ -71,9 +66,19 @@ ) from cylc.flow.wallclock import get_current_time_string from cylc.flow.platforms import get_platform +from cylc.flow.task_outputs import ( + TASK_OUTPUT_EXPIRED, + TASK_OUTPUT_FAILED, + TASK_OUTPUT_SUBMIT_FAILED, +) from cylc.flow.task_queues.independent import IndepQueueManager -from cylc.flow.flow_mgr import FLOW_ALL, FLOW_NONE, FLOW_NEW +from cylc.flow.flow_mgr import ( + stringify_flow_nums, + FLOW_ALL, + FLOW_NONE, + FLOW_NEW +) if TYPE_CHECKING: from cylc.flow.config import WorkflowConfig @@ -81,9 +86,11 @@ from cylc.flow.data_store_mgr import DataStoreMgr from cylc.flow.taskdef import TaskDef from cylc.flow.task_events_mgr import TaskEventsManager + from cylc.flow.task_job_mgr import TaskJobManager from cylc.flow.workflow_db_mgr import WorkflowDatabaseManager from cylc.flow.flow_mgr import FlowMgr, FlowNums + Pool = Dict['PointBase', Dict[str, TaskProxy]] @@ -101,29 +108,28 @@ def __init__( workflow_db_mgr: 'WorkflowDatabaseManager', task_events_mgr: 'TaskEventsManager', data_store_mgr: 'DataStoreMgr', - flow_mgr: 'FlowMgr' + flow_mgr: 'FlowMgr', + task_job_mgr: 'TaskJobManager' ) -> None: self.tokens = tokens - self.config: 'WorkflowConfig' = config + self.config = config self.stop_point = config.stop_point or config.final_point - self.workflow_db_mgr: 'WorkflowDatabaseManager' = workflow_db_mgr - self.task_events_mgr: 'TaskEventsManager' = task_events_mgr - # TODO this is ugly: + self.workflow_db_mgr = workflow_db_mgr + self.task_events_mgr = task_events_mgr self.task_events_mgr.spawn_func = self.spawn_on_output - self.data_store_mgr: 'DataStoreMgr' = data_store_mgr - self.flow_mgr: 'FlowMgr' = flow_mgr + self.data_store_mgr = data_store_mgr + self.flow_mgr = flow_mgr + self.task_job_mgr = task_job_mgr self.max_future_offset: Optional['IntervalBase'] = None self._prev_runahead_base_point: Optional['PointBase'] = None self._prev_runahead_sequence_points: Optional[Set['PointBase']] = None self.runahead_limit_point: Optional['PointBase'] = None - self.main_pool: Pool = {} - self.hidden_pool: Pool = {} - self._main_pool_list: List[TaskProxy] = [] - self._hidden_pool_list: List[TaskProxy] = [] - self.main_pool_changed = False - self.hidden_pool_changed = False + # Tasks in the active window of the workflow. + self.active_tasks: Pool = {} + self._active_tasks_list: List[TaskProxy] = [] + self.active_tasks_changed = False self.tasks_removed = False self.hold_point: Optional['PointBase'] = None @@ -167,20 +173,17 @@ def stop_task_done(self): def _swap_out(self, itask): """Swap old task for new, during reload.""" - if itask.identity in self.hidden_pool.get(itask.point, set()): - self.hidden_pool[itask.point][itask.identity] = itask - self.hidden_pool_changed = True - elif itask.identity in self.main_pool.get(itask.point, set()): - self.main_pool[itask.point][itask.identity] = itask - self.main_pool_changed = True + if itask.identity in self.active_tasks.get(itask.point, set()): + self.active_tasks[itask.point][itask.identity] = itask + self.active_tasks_changed = True def load_from_point(self): """Load the task pool for the workflow start point. Add every parentless task out to the runahead limit. """ - flow_num = self.flow_mgr.get_new_flow( - f"original flow from {self.config.start_point}") + flow_num = self.flow_mgr.get_flow_num( + meta=f"original flow from {self.config.start_point}") self.compute_runahead() for name in self.task_name_list: tdef = self.config.get_taskdef(name) @@ -209,34 +212,13 @@ def db_add_new_flow_rows(self, itask: TaskProxy) -> None: self.workflow_db_mgr.put_insert_task_outputs(itask) def add_to_pool(self, itask) -> None: - """Add a task to the hidden (if not satisfied) or main task pool. - - If the task already exists in the hidden pool and is satisfied, move it - to the main pool. - """ - if itask.is_task_prereqs_not_done() and not itask.is_manual_submit: - # Add to hidden pool if not satisfied. - self.hidden_pool.setdefault(itask.point, {}) - self.hidden_pool[itask.point][itask.identity] = itask - self.hidden_pool_changed = True - LOG.debug(f"[{itask}] added to hidden task pool") - else: - # Add to main pool. - # First remove from hidden pool if necessary. - try: - del self.hidden_pool[itask.point][itask.identity] - except KeyError: - pass - else: - self.hidden_pool_changed = True - if not self.hidden_pool[itask.point]: - del self.hidden_pool[itask.point] - self.main_pool.setdefault(itask.point, {}) - self.main_pool[itask.point][itask.identity] = itask - self.main_pool_changed = True - LOG.debug(f"[{itask}] added to main task pool") + """Add a task to the pool.""" + self.active_tasks.setdefault(itask.point, {}) + self.active_tasks[itask.point][itask.identity] = itask + self.active_tasks_changed = True + LOG.info(f"[{itask}] added to active task pool") - self.create_data_store_elements(itask) + self.create_data_store_elements(itask) if itask.tdef.max_future_prereq_offset is not None: # (Must do this once added to the pool). @@ -265,8 +247,8 @@ def release_runahead_tasks(self): Return True if any tasks are released, else False. Call when RH limit changes. """ - if not self.main_pool or not self.runahead_limit_point: - # (At start-up main pool might not exist yet) + if not self.active_tasks or not self.runahead_limit_point: + # (At start-up task pool might not exist yet) return False released = False @@ -275,7 +257,7 @@ def release_runahead_tasks(self): # tasks can cause the task pool to change size during iteration. release_me = [ itask - for point, itask_id_map in self.main_pool.items() + for point, itask_id_map in self.active_tasks.items() for itask in itask_id_map.values() if point <= self.runahead_limit_point if itask.state.is_runahead @@ -283,11 +265,12 @@ def release_runahead_tasks(self): for itask in release_me: self.rh_release_and_queue(itask) - self.spawn_to_rh_limit( - itask.tdef, - itask.tdef.next_point(itask.point), - itask.flow_nums - ) + if itask.flow_nums: + self.spawn_to_rh_limit( + itask.tdef, + itask.tdef.next_point(itask.point), + itask.flow_nums + ) released = True return released @@ -302,6 +285,10 @@ def compute_runahead(self, force=False) -> bool: * The max future offset might have changed. * The runahead limit config or task pool might have changed (reload). + This is a collective task pool computation. Call it once at the end + of a group operation such as removal of multiple tasks (not after + every individual task operation). + Start from earliest point with unfinished tasks. Partially satisfied and incomplete tasks count too because they still need to run. @@ -322,7 +309,7 @@ def compute_runahead(self, force=False) -> bool: base_point: Optional['PointBase'] = None # First get the runahead base point. - if not self.main_pool: + if not self.active_tasks: # Find the earliest sequence point beyond the workflow start point. base_point = min( point @@ -428,7 +415,7 @@ def compute_runahead(self, force=False) -> bool: def update_flow_mgr(self): flow_nums_seen = set() - for itask in self.get_all_tasks(): + for itask in self.get_tasks(): flow_nums_seen.update(itask.flow_nums) self.flow_mgr.load_from_db(flow_nums_seen) @@ -626,14 +613,14 @@ def load_db_task_action_timers(self, row_idx, row) -> None: return LOG.info("+ %s/%s %s" % (cycle, name, ctx_key)) if ctx_key == "poll_timer": - itask = self._get_main_task_by_id(id_) + itask = self._get_task_by_id(id_) if itask is None: LOG.warning("%(id)s: task not found, skip" % {"id": id_}) return itask.poll_timer = TaskActionTimer( ctx, delays, num, delay, timeout) elif ctx_key[0] == "try_timers": - itask = self._get_main_task_by_id(id_) + itask = self._get_task_by_id(id_) if itask is None: LOG.warning("%(id)s: task not found, skip" % {"id": id_}) return @@ -696,20 +683,25 @@ def rh_release_and_queue(self, itask) -> None: # (otherwise waiting on xtriggers etc.) self.queue_task(itask) - def _get_spawned_or_merged_task( - self, point: 'PointBase', name: str, flow_nums: 'FlowNums' + def get_or_spawn_task( + self, + point: 'PointBase', + name: str, + flow_nums: 'FlowNums', + flow_wait: bool = False ) -> Optional[TaskProxy]: - """Return new or existing task point/name with merged flow_nums""" - taskid = Tokens(cycle=str(point), task=name).relative_id - ntask = ( - self._get_hidden_task_by_id(taskid) - or self._get_main_task_by_id(taskid) + """Return new or existing task point/name with merged flow_nums. + + It does not add a spawned task proxy to the pool. + """ + ntask = self._get_task_by_id( + Tokens(cycle=str(point), task=name).relative_id ) if ntask is None: # ntask does not exist: spawn it in the flow. - ntask = self.spawn_task(name, point, flow_nums) + ntask = self.spawn_task(name, point, flow_nums, flow_wait) else: - # ntask already exists (n=0 or incomplete): merge flows. + # ntask already exists (n=0): merge flows. self.merge_flows(ntask, flow_nums) return ntask # may be None @@ -723,7 +715,7 @@ def spawn_to_rh_limit(self, tdef, point, flow_nums) -> None: self.compute_runahead() while point is not None and (point <= self.runahead_limit_point): if tdef.is_parentless(point): - ntask = self._get_spawned_or_merged_task( + ntask = self.get_or_spawn_task( point, tdef.name, flow_nums ) if ntask is not None: @@ -731,42 +723,43 @@ def spawn_to_rh_limit(self, tdef, point, flow_nums) -> None: self.rh_release_and_queue(ntask) point = tdef.next_point(point) - # Once more (for the rh-limited task: don't rh release it!) - if point is not None and tdef.is_parentless(point): - ntask = self._get_spawned_or_merged_task( + # Once more for the runahead-limited task (don't release it). + self.spawn_if_parentless(tdef, point, flow_nums) + + def spawn_if_parentless(self, tdef, point, flow_nums): + """Spawn a task if parentless, regardless of runahead limit.""" + if flow_nums and point is not None and tdef.is_parentless(point): + ntask = self.get_or_spawn_task( point, tdef.name, flow_nums ) if ntask is not None: self.add_to_pool(ntask) - 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})" + def remove(self, itask, reason=None): + """Remove a task from the pool.""" - try: - del self.hidden_pool[itask.point][itask.identity] - except KeyError: - pass - else: - # e.g. for suicide of partially satisfied task - self.hidden_pool_changed = True - if not self.hidden_pool[itask.point]: - del self.hidden_pool[itask.point] - LOG.debug(f"[{itask}] {msg}") - self.task_queue_mgr.remove_task(itask) - return + if itask.state.is_runahead and itask.flow_nums: + # If removing a parentless runahead-limited task + # auto-spawn its next instance first. + self.spawn_if_parentless( + itask.tdef, + itask.tdef.next_point(itask.point), + itask.flow_nums + ) + if reason is None: + msg = "completed" + else: + msg = f"removed ({reason})" try: - del self.main_pool[itask.point][itask.identity] + del self.active_tasks[itask.point][itask.identity] except KeyError: pass else: - self.main_pool_changed = True - if not self.main_pool[itask.point]: - del self.main_pool[itask.point] + self.tasks_removed = True + self.active_tasks_changed = True + if not self.active_tasks[itask.point]: + del self.active_tasks[itask.point] self.task_queue_mgr.remove_task(itask) if itask.tdef.max_future_prereq_offset is not None: self.set_max_future_offset() @@ -777,66 +770,38 @@ def remove(self, itask, reason=""): # Event-driven final update of task_states table. # TODO: same for datastore (still updated by scheduler loop) self.workflow_db_mgr.put_update_task_state(itask) - LOG.debug(f"[{itask}] {msg}") + LOG.info(f"[{itask}] {msg}") del itask - def get_all_tasks(self) -> List[TaskProxy]: - """Return a list of all task proxies.""" - return self.get_hidden_tasks() + self.get_tasks() - def get_tasks(self) -> List[TaskProxy]: - """Return a list of task proxies in the main pool.""" + """Return a list of task proxies in the task pool.""" # Cached list only for use internally in this method. - if self.main_pool_changed: - self.main_pool_changed = False - self._main_pool_list = [] - for _, itask_id_map in self.main_pool.items(): + if self.active_tasks_changed: + self.active_tasks_changed = False + self._active_tasks_list = [] + for _, itask_id_map in self.active_tasks.items(): for __, itask in itask_id_map.items(): - self._main_pool_list.append(itask) - return self._main_pool_list - - def get_hidden_tasks(self) -> List[TaskProxy]: - """Return a list of task proxies in the hidden pool.""" - # Cached list only for use internally in this method. - if self.hidden_pool_changed: - self.hidden_pool_changed = False - self._hidden_pool_list = [] - for itask_id_maps in self.hidden_pool.values(): - self._hidden_pool_list.extend(list(itask_id_maps.values())) - return self._hidden_pool_list + self._active_tasks_list.append(itask) + return self._active_tasks_list def get_tasks_by_point(self) -> 'Dict[PointBase, List[TaskProxy]]': """Return a map of task proxies by cycle point.""" point_itasks = {} - for point, itask_id_map in self.main_pool.items(): + for point, itask_id_map in self.active_tasks.items(): point_itasks[point] = list(itask_id_map.values()) - for point, itask_id_map in self.hidden_pool.items(): - if point not in point_itasks: - point_itasks[point] = list(itask_id_map.values()) - else: - point_itasks[point] += list(itask_id_map.values()) - return point_itasks def get_task(self, point, name) -> Optional[TaskProxy]: """Retrieve a task from the pool.""" rel_id = f'{point}/{name}' - for pool in (self.main_pool, self.hidden_pool): - tasks = pool.get(point) - if tasks and rel_id in tasks: - return tasks[rel_id] - return None - - def _get_hidden_task_by_id(self, id_: str) -> Optional[TaskProxy]: - """Return runahead pool task by ID if it exists, or None.""" - for itask_ids in list(self.hidden_pool.values()): - with suppress(KeyError): - return itask_ids[id_] + tasks = self.active_tasks.get(point) + if tasks and rel_id in tasks: + return tasks[rel_id] return None - def _get_main_task_by_id(self, id_: str) -> Optional[TaskProxy]: - """Return main pool task by ID if it exists, or None.""" - for itask_ids in list(self.main_pool.values()): + def _get_task_by_id(self, id_: str) -> Optional[TaskProxy]: + """Return pool task by ID if it exists, or None.""" + for itask_ids in list(self.active_tasks.values()): with suppress(KeyError): return itask_ids[id_] return None @@ -887,8 +852,8 @@ def release_queued_tasks(self): for itask in released: itask.state_reset(is_queued=False) - itask.waiting_on_job_prep = True self.data_store_mgr.delta_task_queued(itask) + itask.waiting_on_job_prep = True if cylc.flow.flags.cylc7_back_compat: # Cylc 7 Back Compat: spawn downstream to cause Cylc 7 style @@ -903,7 +868,7 @@ def release_queued_tasks(self): def get_min_point(self): """Return the minimum cycle point currently in the pool.""" - cycles = list(self.main_pool) + cycles = list(self.active_tasks) minc = None if cycles: minc = min(cycles) @@ -952,7 +917,7 @@ def reload_taskdefs(self, config: 'WorkflowConfig') -> None: self.config.adopt_orphans(orphans) LOG.info("Reloading task definitions.") - tasks = self.get_all_tasks() + tasks = self.get_tasks() # Log tasks orphaned by a reload but not currently in the task pool. for name in orphans: if name not in (itask.tdef.name for itask in tasks): @@ -1006,9 +971,12 @@ def reload_taskdefs(self, config: 'WorkflowConfig') -> None: self.config.runtime['descendants'] ) + if self.compute_runahead(): + self.release_runahead_tasks() + # Now queue all tasks that are ready to run for itask in self.get_tasks(): - # Recreate data store elements from main pool. + # Recreate data store elements from task pool. self.create_data_store_elements(itask) if itask.state.is_queued: # Already queued @@ -1035,7 +1003,7 @@ def set_stop_point(self, stop_point: 'PointBase') -> bool: ): self.runahead_limit_point = stop_point # Now handle existing waiting tasks (e.g. xtriggered). - for itask in self.get_all_tasks(): + for itask in self.get_tasks(): if ( itask.point > stop_point and itask.state(TASK_STATUS_WAITING) @@ -1064,8 +1032,7 @@ def can_stop(self, stop_mode): and itask.state(*TASK_STATUSES_ACTIVE) and not itask.state.kill_failed ) - # we don't need to check for preparing tasks because they will be - # reset to waiting on restart + # preparing tasks get reset to waiting on restart for itask in self.get_tasks() ) @@ -1125,7 +1092,7 @@ def log_incomplete_tasks(self) -> bool: return False def log_unsatisfied_prereqs(self) -> bool: - """Log unsatisfied prerequisites in the hidden pool. + """Log unsatisfied prerequisites in the pool. Return True if any, ignoring: - prerequisites beyond the stop point @@ -1133,7 +1100,7 @@ def log_unsatisfied_prereqs(self) -> bool: (can be caused by future triggers) """ unsat: Dict[str, List[str]] = {} - for itask in self.get_hidden_tasks(): + for itask in self.get_tasks(): task_point = itask.point if self.stop_point and task_point > self.stop_point: continue @@ -1170,6 +1137,8 @@ def is_stalled(self) -> bool: ) or ( itask.state(TASK_STATUS_WAITING) and not itask.state.is_runahead + # (avoid waiting pre-spawned absolute-triggered tasks:) + and not itask.is_task_prereqs_not_done() ) for itask in self.get_tasks() ): return False @@ -1195,7 +1164,7 @@ def release_held_active_task(self, itask: TaskProxy) -> None: def set_hold_point(self, point: 'PointBase') -> None: """Set the point after which all tasks must be held.""" self.hold_point = point - for itask in self.get_all_tasks(): + for itask in self.get_tasks(): if itask.point > point: self.hold_active_task(itask) self.workflow_db_mgr.put_workflow_hold_cycle_point(point) @@ -1239,7 +1208,7 @@ def release_held_tasks(self, items: Iterable[str]) -> int: def release_hold_point(self) -> None: """Unset the workflow hold point and release all held active tasks.""" self.hold_point = None - for itask in self.get_all_tasks(): + for itask in self.get_tasks(): self.release_held_active_task(itask) self.tasks_to_hold.clear() self.workflow_db_mgr.put_tasks_to_hold(self.tasks_to_hold) @@ -1253,26 +1222,29 @@ def check_abort_on_task_fails(self): return self.abort_task_failed def spawn_on_output(self, itask, output, forced=False): - """Spawn and update itask's children, remove itask if finished. + """Spawn child-tasks of given output, into the pool. + + Remove the parent task from the pool if complete. - Also set a the abort-on-task-failed flag if necessary. + Called by task event manager on receiving output messages, and after + forced setting of task outputs (in this case the parent task could + be transient, i.e. not in the pool). - If not flowing on: - - update existing children but don't spawn new ones - - unless forced (manual command): spawn but with no flow number + Also set the abort-on-task-failed flag if necessary. + + If not flowing on, update existing children but don't spawn new ones + (unless manually forced to spawn with no flow number). If an absolute output is completed update the store of completed abs outputs, and update the prerequisites of every instance of the child - in the pool. (And in self.spawn() use the store of completed abs - outputs to satisfy any tasks with abs prerequisites). + in the pool. (The self.spawn method uses the store of completed abs + outputs to satisfy any tasks with absolute prerequisites). Args: - tasks: List of identifiers or task globs. - outputs: List of outputs to spawn on. - forced: If True this is a manual spawn command. + output: output to spawn on. + forced: True if called from manual set task command """ - self.workflow_db_mgr.put_update_task_outputs(itask) if ( output == TASK_OUTPUT_FAILED and self.expected_failed_tasks is not None @@ -1287,6 +1259,12 @@ def spawn_on_output(self, itask, output, forced=False): suicide = [] for c_name, c_point, is_abs in children: + + if itask.flow_wait: + LOG.warning( + f"[{itask}] not spawning on {output}: flow wait requested") + continue + if is_abs: self.abs_outputs_done.add( (str(itask.point), itask.tdef.name, output)) @@ -1298,17 +1276,13 @@ def spawn_on_output(self, itask, output, forced=False): cycle=str(c_point), task=c_name, ).relative_id - c_task = ( - self._get_hidden_task_by_id(c_taskid) - or self._get_main_task_by_id(c_taskid) - ) + c_task = self._get_task_by_id(c_taskid) if c_task is not None and c_task != itask: # (Avoid self-suicide: A => !A) self.merge_flows(c_task, itask.flow_nums) elif ( c_task is None and (itask.flow_nums or forced) - and not itask.flow_wait ): # If child is not in the pool already, and parent belongs to a # flow (so it can spawn children), and parent is not waiting @@ -1327,11 +1301,8 @@ def spawn_on_output(self, itask, output, forced=False): else: tasks = [c_task] for t in tasks: - t.state.satisfy_me({ - (str(itask.point), itask.tdef.name, output) - }) + t.satisfy_me([f"{itask.identity}:{output}"]) self.data_store_mgr.delta_task_prerequisite(t) - # Add it to the hidden pool or move it to the main pool. self.add_to_pool(t) if t.point <= self.runahead_limit_point: @@ -1354,17 +1325,12 @@ def spawn_on_output(self, itask, output, forced=False): msg += " suiciding while active" self.remove(c_task, msg) - if not forced and output in [ - # final task statuses - TASK_OUTPUT_SUCCEEDED, - TASK_OUTPUT_EXPIRED, - TASK_OUTPUT_FAILED, - TASK_OUTPUT_SUBMIT_FAILED, - ]: - self.remove_if_complete(itask) + self.remove_if_complete(itask) - def remove_if_complete(self, itask): - """Remove finished itask if required outputs are complete. + def remove_if_complete(self, itask: TaskProxy) -> bool: + """Remove a finished task if required outputs are complete. + + Return True if removed else False. Cylc 8: - if complete: @@ -1378,32 +1344,43 @@ def remove_if_complete(self, itask): else (failed): - retain and recompute runahead (C7 failed tasks don't count toward runahead limit) + """ - ret = False + if not itask.state(*TASK_STATUSES_FINAL): + return False + + if itask.identity == self.stop_task_id: + self.stop_task_finished = True + if cylc.flow.flags.cylc7_back_compat: + ret = False if not itask.state(TASK_STATUS_FAILED, TASK_OUTPUT_SUBMIT_FAILED): - self.remove(itask, 'finished') + self.remove(itask) ret = True + # Recompute runhead either way; failed tasks don't count in C7. if self.compute_runahead(): self.release_runahead_tasks() - else: - incomplete = itask.state.outputs.get_incomplete() - if incomplete: - # Retain as incomplete. - LOG.warning( - f"[{itask}] did not complete required outputs:" - f" {incomplete}" - ) - else: - # Remove as completed. - self.remove(itask, 'finished') - ret = True - if itask.identity == self.stop_task_id: - self.stop_task_finished = True - if self.compute_runahead(): - self.release_runahead_tasks() + return ret + + if itask.state(TASK_STATUS_EXPIRED): + self.remove(itask, "expired") + if self.compute_runahead(): + self.release_runahead_tasks() + return True + + incomplete = itask.state.outputs.get_incomplete() + if incomplete: + # Retain as an incomplete task. + LOG.warning( + f"[{itask}] did not complete required outputs:" + f" {incomplete}" + ) + return False - return ret + self.remove(itask) + if self.compute_runahead(): + self.release_runahead_tasks() + return True def spawn_on_all_outputs( self, itask: TaskProxy, completed_only: bool = False @@ -1438,10 +1415,7 @@ def spawn_on_all_outputs( cycle=str(c_point), task=c_name, ).relative_id - c_task = ( - self._get_hidden_task_by_id(c_taskid) - or self._get_main_task_by_id(c_taskid) - ) + c_task = self._get_task_by_id(c_taskid) if c_task is not None: # already spawned continue @@ -1451,9 +1425,7 @@ def spawn_on_all_outputs( # not spawnable continue if completed_only: - c_task.state.satisfy_me({ - (str(itask.point), itask.tdef.name, output) - }) + c_task.satisfy_me([f"{itask.identity}:{output}"]) self.data_store_mgr.delta_task_prerequisite(c_task) self.add_to_pool(c_task) if ( @@ -1462,12 +1434,13 @@ def spawn_on_all_outputs( ): self.rh_release_and_queue(c_task) - def can_spawn(self, name: str, point: 'PointBase') -> bool: - """Return True if the task with the given name & point is within - various workflow limits.""" + def can_be_spawned(self, name: str, point: 'PointBase') -> bool: + """Return True if a point/name is within graph bounds.""" + if name not in self.config.taskdefs: LOG.debug('No task definition %s', name) return False + # Don't spawn outside of graph limits. # TODO: is it possible for initial_point to not be defined?? # (see also the similar check + log message in scheduler.py) @@ -1477,176 +1450,398 @@ def can_spawn(self, name: str, point: 'PointBase') -> bool: LOG.debug( 'Not spawning %s/%s: before initial cycle point', point, name) return False - elif self.config.final_point and point > self.config.final_point: + + if self.config.final_point and point > self.config.final_point: # Only happens on manual trigger beyond FCP LOG.debug( 'Not spawning %s/%s: beyond final cycle point', point, name) return False + + # Is it on-sequence and within recurrence bounds. + if not self.config.get_taskdef(name).is_valid_point(point): + LOG.warning( + self.ERR_PREFIX_TASK_NOT_ON_SEQUENCE.format( + name, point + ) + ) + return False + return True + def _get_task_history(self, name, point, flow_nums): + """Get details of previous submits for this task. + + """ + info = self.workflow_db_mgr.pri_dao.select_prev_instances( + name, str(point) + ) + try: + submit_num = max(s[0] for s in info) + except ValueError: + # never spawned before in any flow + submit_num = 0 + + prev_completed = False # did not complete in the flow + prev_flow_wait = False # did not wait in the flow + # orig_fnums = set() + + for snum, f_wait, old_fnums, is_complete in info: + if set.intersection(flow_nums, old_fnums): + # matching flows + prev_completed = is_complete + prev_flow_wait = f_wait + # orig_fnums = old_fnums + if not prev_completed: + # There may be multiple entries with flow overlap due + # to merges (they'll have have same snum and f_wait); + # keep going to find the complete one, if any . + continue + LOG.warning( + f"{point}/{name} already completed for flow" + f" {stringify_flow_nums(flow_nums, full=True)} via" + f" {point}/{name}/{snum:02d}" + f"{stringify_flow_nums(old_fnums, full=True)}" + ) + break + + return submit_num, prev_completed, prev_flow_wait + + def _load_historical_outputs(self, itask): + """Load a task's historical outputs from the DB.""" + info = self.workflow_db_mgr.pri_dao.select_task_outputs( + itask.tdef.name, str(itask.point)) + if not info: + # task never ran before + self.db_add_new_flow_rows(itask) + else: + for outputs_str, fnums in info.items(): + if itask.flow_nums.intersection(fnums): + for msg in json.loads(outputs_str): + itask.state.outputs.set_completed_by_msg(msg) + def spawn_task( self, name: str, point: 'PointBase', flow_nums: Set[int], force: bool = False, - is_manual_submit: bool = False, flow_wait: bool = False, ) -> Optional[TaskProxy]: - """Spawn point/name. Return the spawned task, or None. + """Spawn a task if not already completed for this flow, or if forced. + + The creates the task proxy but does not add it to the pool. + + If completed previously with flow wait, just try to spawn children. + + # (TODO - reuse "cylc set" machinery for this?) - Force arg used in manual triggering. """ - if not self.can_spawn(name, point): + if not self.can_be_spawned(name, point): return None - # Get submit number by flow_nums {flow_nums: submit_num, ...} - snums = self.workflow_db_mgr.pri_dao.select_submit_nums( - name, str(point) + submit_num, prev_completed, prev_flow_wait = ( + self._get_task_history(name, point, flow_nums) ) - try: - submit_num = max(s for s in snums.keys()) - except ValueError: - # Task never spawned in any flow. - submit_num = 0 - flow_wait_done = False - for f_wait, old_fnums in snums.values(): - # Flow_nums of previous instances. - if ( - not force and - set.intersection(flow_nums, old_fnums) - ): - if f_wait: - flow_wait_done = f_wait - break - # To avoid "conditional reflow" with (e.g.) "foo | bar => baz". - LOG.warning( - f"Task {point}/{name} already spawned in {flow_nums}" - ) - return None - - # Spawn if on-sequence and within recurrence bounds. - taskdef = self.config.get_taskdef(name) - if not taskdef.is_valid_point(point): - LOG.warning( - self.ERR_PREFIX_TASK_NOT_ON_SEQUENCE.format( - taskdef.name, point - ) - ) + # If already completed and spawned on outputs, quit unless forced. + if ( + prev_completed and not prev_flow_wait + and not force + ): return None - itask = TaskProxy( - self.tokens, - taskdef, + # A transient task is only used for spawning on outputs. + if force: + transient = False + else: + # transient if completed, else run again. + transient = prev_completed + + itask = self._get_task_proxy( point, + self.config.get_taskdef(name), flow_nums, submit_num=submit_num, - is_manual_submit=is_manual_submit, flow_wait=flow_wait, + transient=transient ) - if (name, point) in self.tasks_to_hold: - LOG.info(f"[{itask}] holding (as requested earlier)") - self.hold_active_task(itask) - elif self.hold_point and itask.point > self.hold_point: - # Hold if beyond the workflow hold point - LOG.info( - f"[{itask}] holding (beyond workflow " - f"hold point: {self.hold_point})" - ) - self.hold_active_task(itask) + if not itask: + return None + + if not transient: + if (name, point) in self.tasks_to_hold: + LOG.info(f"[{itask}] holding (as requested earlier)") + self.hold_active_task(itask) + elif self.hold_point and itask.point > self.hold_point: + # Hold if beyond the workflow hold point + LOG.info( + f"[{itask}] holding (beyond workflow " + f"hold point: {self.hold_point})" + ) + self.hold_active_task(itask) - if self.stop_point and itask.point <= self.stop_point: - future_trigger_overrun = False + # Don't add to pool if it depends on a task beyond the stop point. + # """ + # foo + # foo[+P1] & bar => baz + # """ + # Here, in the final cycle bar wants to spawn baz, but that would stall + # the workflow because baz also depends on foo after the final point. + if ( + not transient + and self.stop_point + and itask.point <= self.stop_point + ): for pct in itask.state.prerequisites_get_target_points(): if pct > self.stop_point: - future_trigger_overrun = True - break - if future_trigger_overrun: - LOG.warning( - f"[{itask}] won't run: depends on a task beyond " - f"the stop point ({self.stop_point})" - ) + LOG.warning( + f"[{itask}] not spawned: a prerequisite is beyond" + f" the workflow stop point ({self.stop_point})" + ) + return None # Satisfy any absolute triggers. if ( - itask.tdef.has_abs_triggers and - itask.state.prerequisites_are_not_all_satisfied() + not transient + and itask.tdef.has_abs_triggers + and itask.state.prerequisites_are_not_all_satisfied() ): - itask.state.satisfy_me(self.abs_outputs_done) + itask.satisfy_me( + [f"{a[0]}/{a[1]}:{a[2]}" for a in self.abs_outputs_done] + ) + + if prev_flow_wait: + self._spawn_after_flow_wait(itask) + return None - if flow_wait_done: - for outputs_str, fnums in ( - self.workflow_db_mgr.pri_dao.select_task_outputs( - itask.tdef.name, str(itask.point)) - ).items(): + self.db_add_new_flow_rows(itask) + return itask + + def _spawn_after_flow_wait(self, itask: TaskProxy) -> None: + LOG.info( + f"spawning children of {itask.identity} after flow wait" + ) + self.spawn_on_all_outputs(itask, completed_only=True) + # update flow wait status in the DB + itask.flow_wait = False + # itask.flow_nums = orig_fnums + self.workflow_db_mgr.put_update_task_flow_wait(itask) + return None + + def _get_task_proxy( + self, + point: 'PointBase', + taskdef: 'TaskDef', + flow_nums: 'FlowNums', + flow_wait: bool = False, + transient: bool = False, + is_manual_submit: bool = False, + submit_num: int = 0 + ) -> Optional['TaskProxy']: + """Spawn a task proxy and update its outputs from the DB. """ + + if not self.can_be_spawned(taskdef.name, point): + return None + + itask = TaskProxy( + self.tokens, + taskdef, + point, + flow_nums, + flow_wait=flow_wait, + submit_num=submit_num, + transient=transient, + is_manual_submit=is_manual_submit + ) + + if itask is not None: + # Update it with outputs that were already completed. + info = self.workflow_db_mgr.pri_dao.select_task_outputs( + itask.tdef.name, str(itask.point)) + if not info: + self.db_add_new_flow_rows(itask) + spawn_kids = False + for outputs_str, fnums in info.items(): if flow_nums.intersection(fnums): + if itask.flow_wait: + spawn_kids = True for msg in json.loads(outputs_str): itask.state.outputs.set_completed_by_msg(msg) - break - LOG.info(f"{itask} spawning on outputs after flow wait") - self.spawn_on_all_outputs(itask, completed_only=True) - return None - LOG.debug(f"[{itask}] spawned") - self.db_add_new_flow_rows(itask) + if spawn_kids: + self._spawn_after_flow_wait(itask) + return itask - def force_spawn_children( + def set( # noqa: A003 self, items: Iterable[str], - outputs: Optional[List[str]] = None, - flow_num: Optional[int] = None + outputs: Optional[List[str]], + prereqs: Optional[List[str]], + flow: List[str], + flow_wait: bool = False, + flow_descr: Optional[str] = None ): - """Spawn downstream children of given outputs, on user command. + """Set prerequisites or outputs of target tasks. + + Default: set all required outputs. + + Set prerequisites: + - spawn the task (if not spawned) + - update its prerequisites - User-facing command name: set_outputs. Creates a transient parent just - for the purpose of spawning children. + Set outputs: + - update task outputs in the DB + - (implied outputs are handled by the event manager) + - spawn children of the outputs (if not spawned) + - update the child prerequisites + + Uses a transient task proxy to spawn children. (Even if parent was + previously spawned in this flow its children might not have been). + + Task matching restrictions (for now): + - globs (cycle and name) only match in the pool + - future tasks must be specified individually + - family names are not expanded to members Args: - items: Identifiers for matching task definitions, each with the - form "point/name". - outputs: List of outputs to spawn on - flow_num: Flow number to attribute the outputs + items: task ID match patterns + prereqs: prerequisites to set + outputs: outputs to set and spawn children of + flow: flow numbers for spawned or merged tasks + flow_wait: wait for flows to catch up before continuing + flow_descr: description of new flow + + """ + flow_nums = self._get_flow_nums(flow, flow_descr) + if flow_nums is None: + # Illegal flow command opts + return + + # Get matching pool tasks and future task definitions. + itasks, future_tasks, unmatched = self.filter_task_proxies( + items, + future=True, + warn=False, + ) + + for itask in itasks: + self.merge_flows(itask, flow_nums) + if prereqs: + self._set_prereqs_itask( + itask, prereqs, flow_nums, flow_wait) + else: + self._set_outputs_itask(itask, outputs) + + for name, point in future_tasks: + tdef = self.config.get_taskdef(name) + if prereqs: + self._set_prereqs_tdef( + point, tdef, prereqs, flow_nums, flow_wait) + else: + trans = self._get_task_proxy( + point, tdef, flow_nums, flow_wait, transient=True) + if trans is not None: + self._set_outputs_itask(trans, outputs) + + if self.compute_runahead(): + self.release_runahead_tasks() + + def kill_expire_job(self, itask): + if self.config.run_mode('simulation'): + itask.state_reset(TASK_STATUS_EXPIRED) + self.data_store_mgr.delta_task_state(itask) + else: + self.task_job_mgr.kill_task_jobs([itask], expire=True) + + def _set_outputs_itask( + self, + itask: 'TaskProxy', + outputs: Optional[Iterable[str]], + ) -> None: + """Set requested outputs on a task proxy and spawn children.""" + + # Default to required outputs. + outputs = outputs or itask.tdef.get_required_outputs() + + changed = False + for output in outputs: + if ( + output == TASK_OUTPUT_EXPIRED + and itask.state(*TASK_STATUSES_ACTIVE) + ): + LOG.warning(f"killing expired active task {itask}") + # To expire an active task, kill the job first. + self.kill_expire_job(itask) + continue + + # convert trigger label to output message + msg = itask.state.outputs.get_msg(output) + info = f'set: output {itask.identity}:{output}' + if msg is None: + LOG.warning(f"{info} not found") + continue + if itask.state.outputs.is_completed(msg): + LOG.info(f"{info} completed already") + continue + changed = True + self.task_events_mgr.process_message( + itask, logging.INFO, msg, forced=True) + LOG.info(f"{info} completed") + + if changed and itask.transient: + self.workflow_db_mgr.put_update_task_state(itask) + self.workflow_db_mgr.put_update_task_outputs(itask) + + def _set_prereqs_itask( + self, + itask: 'TaskProxy', + prereqs: List[str], + flow_nums: Set[int], + flow_wait: bool + ) -> None: + """Set prerequisites on a task proxy. + + Prerequisite format: "cycle/task:message" or "all". """ - outputs = outputs or [TASK_OUTPUT_SUCCEEDED] - if flow_num is None: - flow_nums = None + if prereqs == ["all"]: + itask.state.set_all_satisfied() else: - flow_nums = {flow_num} + itask.satisfy_me(prereqs) + if ( + self.runahead_limit_point is not None + and itask.point <= self.runahead_limit_point + ): + self.rh_release_and_queue(itask) + self.data_store_mgr.delta_task_prerequisite(itask) - n_warnings, task_items = self.match_taskdefs(items) - for (_, point), taskdef in sorted(task_items.items()): - # This the parent task: - itask = TaskProxy( - self.tokens, - taskdef, - point, - flow_nums=flow_nums, - ) - # Spawn children of selected outputs. - for trig, out, _ in itask.state.outputs.get_all(): - if trig in outputs: - LOG.info(f"[{itask}] Forced spawning on {out}") - self.spawn_on_output(itask, out, forced=True) + def _set_prereqs_tdef(self, point, taskdef, prereqs, flow_nums, flow_wait): + """Spawn a future task and set prerequisites on it. + + """ + itask = self.spawn_task(taskdef.name, point, flow_nums, flow_wait) + if itask is None: + return + self.add_to_pool(itask) + self._set_prereqs_itask(itask, prereqs, flow_nums, flow_wait) def _get_active_flow_nums(self) -> Set[int]: - """Return all active, or most recent previous, flow numbers. + """Return active flow numbers. + + If there are no active flows (e.g. on restarting a completed workflow) + return the most recent active flows. - If there are any active flows, return all active flow numbers. - Otherwise (e.g. on restarting a completed workflow) return - the flow numbers of the most recent previous active task. """ fnums = set() - for itask in self.get_all_tasks(): + for itask in self.get_tasks(): fnums.update(itask.flow_nums) if not fnums: fnums = self.workflow_db_mgr.pri_dao.select_latest_flow_nums() return fnums def remove_tasks(self, items): - """Remove tasks from the pool.""" + """Remove tasks from the pool (forced by command).""" itasks, _, bad_items = self.filter_task_proxies(items) for itask in itasks: self.remove(itask, 'request') @@ -1654,127 +1849,141 @@ def remove_tasks(self, items): self.release_runahead_tasks() return len(bad_items) - def force_trigger_tasks( - self, items: Iterable[str], - flow: List[str], - flow_wait: bool = False, - flow_descr: Optional[str] = None - ) -> int: - """Manual task triggering. - - Don't get a new flow number for existing n=0 tasks (e.g. incomplete - tasks). These can carry on in the original flow if retriggered. - - Queue the task if not queued, otherwise release it to run. - - """ + def _get_flow_nums( + self, + flow: List[str], + meta: Optional[str] = None, + ) -> Optional[Set[int]]: + """Get correct flow numbers given user command options.""" if set(flow).intersection({FLOW_ALL, FLOW_NEW, FLOW_NONE}): if len(flow) != 1: LOG.warning( f'The "flow" values {FLOW_ALL}, {FLOW_NEW} & {FLOW_NONE}' ' cannot be used in combination with integer flow numbers.' ) - return 0 + return None if flow[0] == FLOW_ALL: flow_nums = self._get_active_flow_nums() elif flow[0] == FLOW_NEW: - flow_nums = {self.flow_mgr.get_new_flow(flow_descr)} + flow_nums = {self.flow_mgr.get_flow_num(meta=meta)} elif flow[0] == FLOW_NONE: flow_nums = set() else: try: - flow_nums = {int(n) for n in flow} + flow_nums = { + self.flow_mgr.get_flow_num( + flow_num=int(n), meta=meta + ) + for n in flow + } except ValueError: LOG.warning( - f"Trigger ignored, illegal flow values {flow}" + f"Ignoring command: illegal flow values {flow}" ) - return 0 + return None + return flow_nums + + def _force_trigger(self, itask): + """Assumes task is in the pool""" + # TODO is this flag still needed, and consistent with "cylc set"? + itask.is_manual_submit = True + itask.reset_try_timers() + if itask.state_reset(TASK_STATUS_WAITING): + # (could also be unhandled failed) + self.data_store_mgr.delta_task_state(itask) + # (No need to set prerequisites satisfied here). + if itask.state.is_runahead: + # Release from runahead, and queue it. + self.rh_release_and_queue(itask) + self.spawn_to_rh_limit( + itask.tdef, + itask.tdef.next_point(itask.point), + itask.flow_nums + ) + else: + # De-queue it to run now. + self.task_queue_mgr.force_release_task(itask) - # n_warnings, task_items = self.match_taskdefs(items) - itasks, future_tasks, unmatched = self.filter_task_proxies( - items, - future=True, - warn=False, + def force_trigger_tasks( + self, items: Iterable[str], + flow: List[str], + flow_wait: bool = False, + flow_descr: Optional[str] = None + ): + """Force a task to trigger (user command). + + Always run the task, even if a previous run was flow-waited. + + If the task did not run before in the flow: + - run it, and spawn on outputs unless flow-wait is set. + (but load the previous outputs from the DB) + + Else if the task ran before in the flow: + - load previous outputs + If the previous run was not flow-wait + - run it, and try to spawn on outputs + Else if the previous run was flow-wait: + - just try to spawn, unless flow-wait is set. + + ("try to spawn": unless the output already spawned in the flow) + + """ + # TODO CHECK FLOW MERGE BASED ON CLI FLOWS + + # Get flow numbers for the tasks to be triggered. + flow_nums = self._get_flow_nums(flow, flow_descr) + if flow_nums is None: + return + + # Get matching tasks proxies, and matching future task IDs. + existing_tasks, future_ids, unmatched = self.filter_task_proxies( + items, future=True, warn=False, ) - # Spawn future tasks. - for name, point in future_tasks: - # (Flow values already validated by the trigger client). - itask = self.spawn_task( - name, + # Trigger existing tasks. + for itask in existing_tasks: + if itask.state(TASK_STATUS_PREPARING, *TASK_STATUSES_ACTIVE): + LOG.warning(f"[{itask}] ignoring trigger - already active") + continue + self._force_trigger(itask) + + # Spawn and trigger future tasks. + for name, point in future_ids: + + if not self.can_be_spawned(name, point): + continue + + submit_num, _, prev_fwait = self._get_task_history( + name, point, flow_nums) + + itask = TaskProxy( + self.tokens, + self.config.get_taskdef(name), point, flow_nums, - force=True, - is_manual_submit=True, - flow_wait=flow_wait + flow_wait=flow_wait, + submit_num=submit_num, ) if itask is None: continue - itasks.append(itask) - - # Trigger matched tasks if not already active. - for itask in itasks: - if itask.state(TASK_STATUS_PREPARING, *TASK_STATUSES_ACTIVE): - LOG.warning(f"[{itask}] ignoring trigger - already active") - continue - itask.is_manual_submit = True - itask.reset_try_timers() - # (If None, spawner reports cycle bounds errors). - if itask.state_reset(TASK_STATUS_WAITING): - # (could also be unhandled failed) - self.data_store_mgr.delta_task_state(itask) - # (No need to set prerequisites satisfied here). - self.add_to_pool(itask) # move from hidden if necessary. - if itask.state.is_runahead: - # Release from runahead, and queue it. - self.rh_release_and_queue(itask) - self.spawn_to_rh_limit( - itask.tdef, - itask.tdef.next_point(itask.point), - itask.flow_nums - ) - else: - # De-queue it to run now. - self.task_queue_mgr.force_release_task(itask) - return len(unmatched) + self.db_add_new_flow_rows(itask) - def set_expired_tasks(self): - res = False - for itask in self.get_tasks(): - if self._set_expired_task(itask): - res = True - return res + if prev_fwait: + # update completed outputs from the DB + self._load_historical_outputs(itask) - def _set_expired_task(self, itask): - """Check if task has expired. Set state and event handler if so. + # run it (or run it again for incomplete flow-wait) + self.add_to_pool(itask) + self._force_trigger(itask) - Return True if task has expired. - """ - if ( - not itask.state( - TASK_STATUS_WAITING, - is_held=False - ) - or itask.tdef.expiration_offset is None - ): - return False - if itask.expire_time is None: - itask.expire_time = ( - itask.get_point_as_seconds() + - itask.get_offset_as_seconds(itask.tdef.expiration_offset)) - if time() > itask.expire_time: - msg = 'Task expired (skipping job).' - LOG.warning(f"[{itask}] {msg}") - self.task_events_mgr.setup_event_handlers(itask, "expired", msg) - # TODO succeeded and expired states are useless due to immediate - # removal under all circumstances (unhandled failed is still used). - if itask.state_reset(TASK_STATUS_EXPIRED, is_held=False): - self.data_store_mgr.delta_task_state(itask) - self.data_store_mgr.delta_task_held(itask) - self.remove(itask, 'expired') - return True - return False + def clock_expire_tasks(self): + """Expire any tasks past their clock-expiry time.""" + for itask in self.get_tasks(): + if not itask.clock_expire(): + continue + self.task_events_mgr.process_message( + itask, logging.WARNING, TASK_OUTPUT_EXPIRED) def task_succeeded(self, id_): """Return True if task with id_ is in the succeeded state.""" @@ -1792,7 +2001,7 @@ def stop_flow(self, flow_num): Remove the flow number from every task in the pool, and remove any task with no remaining flow numbers if it is not already active. """ - for itask in self.get_all_tasks(): + for itask in self.get_tasks(): try: itask.flow_nums.remove(flow_num) except KeyError: @@ -1803,25 +2012,20 @@ def stop_flow(self, flow_num): *TASK_STATUSES_ACTIVE, TASK_STATUS_PREPARING) and not itask.flow_nums ): + # Don't spawn successor if the task is parentless. self.remove(itask, "flow stopped") + if self.compute_runahead(): + self.release_runahead_tasks() + def log_task_pool(self, log_lvl=logging.DEBUG): - """Log content of task and prerequisite pools in debug mode.""" - for pool, name in [ - (self.get_tasks(), "Main"), - (self.get_hidden_tasks(), "Hidden") - ]: - if pool: - LOG.log( - log_lvl, - f"{name} pool:\n" - + "\n".join( - f"* {itask} status={itask.state.status}" - f" runahead={itask.state.is_runahead}" - f" queued={itask.state.is_queued}" - for itask in pool - ) - ) + """Log content of task pool, for debugging.""" + LOG.log( + log_lvl, + "\n".join( + f"* {itask}" for itask in self.get_tasks() + ) + ) def filter_task_proxies( self, @@ -1850,7 +2054,7 @@ def filter_task_proxies( """ matched, unmatched = filter_ids( - [self.main_pool, self.hidden_pool], + self.active_tasks, ids, warn=warn, ) diff --git a/cylc/flow/task_proxy.py b/cylc/flow/task_proxy.py index 0e7fe1d8868..8fe2aa74b40 100644 --- a/cylc/flow/task_proxy.py +++ b/cylc/flow/task_proxy.py @@ -19,12 +19,14 @@ from collections import Counter from copy import copy from fnmatch import fnmatchcase +from time import time from typing import ( Any, Callable, Counter as TypingCounter, Dict, List, + Iterable, Optional, Set, TYPE_CHECKING, @@ -34,9 +36,15 @@ from metomi.isodatetime.timezone import get_local_time_zone from cylc.flow import LOG +from cylc.flow.flow_mgr import stringify_flow_nums +from cylc.flow.id import Tokens from cylc.flow.platforms import get_platform from cylc.flow.task_action_timer import TimerFlags -from cylc.flow.task_state import TaskState, TASK_STATUS_WAITING +from cylc.flow.task_state import ( + TaskState, + TASK_STATUS_WAITING, + TASK_STATUS_EXPIRED +) from cylc.flow.taskdef import generate_graph_children from cylc.flow.wallclock import get_unix_time_from_time_string as str2time from cylc.flow.cycling.iso8601 import ( @@ -46,7 +54,6 @@ ) if TYPE_CHECKING: - from cylc.flow.id import Tokens from cylc.flow.cycling import PointBase from cylc.flow.task_action_timer import TaskActionTimer from cylc.flow.taskdef import TaskDef @@ -145,6 +152,9 @@ class TaskProxy: .waiting_on_job_prep: True whilst task is awaiting job prep, reset to False once the preparation has completed. + .transient: + This is a transient proxy - not to be added to the task pool, but + used e.g. to spawn children, or to get task-specific infomation. Args: tdef: The definition object of this task. @@ -186,6 +196,7 @@ class TaskProxy: 'tokens', 'try_timers', 'waiting_on_job_prep', + 'transient' ] def __init__( @@ -201,6 +212,7 @@ def __init__( is_manual_submit: bool = False, flow_wait: bool = False, data_mode: bool = False, + transient: bool = False ) -> None: self.tdef = tdef @@ -246,6 +258,8 @@ def __init__( else: self.platform = get_platform() + self.transient = transient + self.job_vacated = False self.poll_timer: Optional['TaskActionTimer'] = None self.timeout: Optional[float] = None @@ -266,16 +280,28 @@ def __init__( else: self.graph_children = generate_graph_children(tdef, self.point) + if self.tdef.expiration_offset is not None: + self.expire_time = ( + self.get_point_as_seconds() + + self.get_offset_as_seconds( + self.tdef.expiration_offset + ) + ) + def __repr__(self) -> str: return f"<{self.__class__.__name__} '{self.tokens}'>" def __str__(self) -> str: - """Stringify with tokens, state, submit_num, and flow_nums.""" + """Stringify with tokens, state, submit_num, and flow_nums. + + Format: "//{}:status". + """ + id_ = self.identity + if self.state.status != TASK_STATUS_WAITING: + id_ += f"/{self.submit_num:02d}" + return ( - f"{self.identity} " - f"{self.state} " - f"job:{self.submit_num:02d}" - f" flows:{','.join(str(i) for i in self.flow_nums) or 'none'}" + f"{id_}{stringify_flow_nums(self.flow_nums)}:{self.state}" ) def copy_to_reload_successor(self, reload_successor, check_output): @@ -478,12 +504,47 @@ def merge_flows(self, flow_nums: Set) -> None: def state_reset( self, status=None, is_held=None, is_queued=None, is_runahead=None, - silent=False + silent=False, forced=False ) -> bool: """Set new state and log the change. Return whether it changed.""" before = str(self) - if self.state.reset(status, is_held, is_queued, is_runahead): - if not silent: + if status == TASK_STATUS_EXPIRED: + is_queued = False + if self.state.reset( + status, is_held, is_queued, is_runahead, forced + ): + if not silent and not self.transient: LOG.info(f"[{before}] => {self.state}") return True return False + + def satisfy_me(self, outputs: Iterable[str]) -> None: + """Try to satisfy my prerequisites with given outputs. + + The output strings are of the form "cycle/task:message" + Log a warning for outputs that I don't depend on. + + """ + tokens = [Tokens(p, relative=True) for p in outputs] + used = self.state.satisfy_me(tokens) + for output in set(outputs) - used: + LOG.warning( + f"{self.identity} does not depend on {output}" + ) + + def clock_expire(self) -> bool: + """Return True if clock expire time is up, else False.""" + if ( + self.expire_time is None # expiry not configured + or self.state(TASK_STATUS_EXPIRED) # already expired + or time() < self.expire_time # not time yet + ): + return False + return True + + def is_complete(self) -> bool: + """Return True if complete or expired, else False.""" + return ( + self.state(TASK_STATUS_EXPIRED) + or not self.state.outputs.is_incomplete() + ) diff --git a/cylc/flow/task_qualifiers.py b/cylc/flow/task_qualifiers.py index 68558f76975..eae93ec1ff0 100644 --- a/cylc/flow/task_qualifiers.py +++ b/cylc/flow/task_qualifiers.py @@ -44,6 +44,8 @@ QUAL_FAM_SUBMIT_ANY = "submit-any" QUAL_FAM_SUBMIT_FAIL_ALL = "submit-fail-all" QUAL_FAM_SUBMIT_FAIL_ANY = "submit-fail-any" +QUAL_FAM_EXPIRE_ALL = "expire-all" +QUAL_FAM_EXPIRE_ANY = "expire-any" # alternative (shorthand) qualifiers ALT_QUALIFIERS = { diff --git a/cylc/flow/task_state.py b/cylc/flow/task_state.py index 9dea7e9af28..1915dac4062 100644 --- a/cylc/flow/task_state.py +++ b/cylc/flow/task_state.py @@ -1,6 +1,6 @@ # 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 @@ -17,14 +17,14 @@ """Task state related logic.""" -from typing import List +from typing import List, Iterable, Set, TYPE_CHECKING from cylc.flow.prerequisite import Prerequisite -from cylc.flow.task_outputs import ( - TaskOutputs, - TASK_OUTPUT_EXPIRED, TASK_OUTPUT_SUBMITTED, TASK_OUTPUT_SUBMIT_FAILED, - TASK_OUTPUT_STARTED, TASK_OUTPUT_SUCCEEDED, TASK_OUTPUT_FAILED) +from cylc.flow.task_outputs import TaskOutputs from cylc.flow.wallclock import get_current_time_string +if TYPE_CHECKING: + from cylc.flow.id import Tokens + # Task status names and meanings. # Held back from job submission due to un-met prerequisites: @@ -197,8 +197,10 @@ class TaskState: xtriggers as {trigger (str): satisfied (boolean), ...}. ._is_satisfied (boolean): Are prerequisites satisfied? + Reset None to force re-evaluation when a prereq gets satisfied. ._suicide_is_satisfied (boolean): Are prerequisites to trigger suicide satisfied? + Reset None to force re-evaluation when a prereq gets satisfied. """ # Memory optimization - constrain possible attributes to this list. @@ -308,13 +310,23 @@ def __call__( ) ) - def satisfy_me(self, all_task_outputs): - """Attempt to get my prerequisites satisfied.""" - for prereqs in [self.prerequisites, self.suicide_prerequisites]: - for prereq in prereqs: - if prereq.satisfy_me(all_task_outputs): - self._is_satisfied = None - self._suicide_is_satisfied = None + def satisfy_me( + self, + outputs: Iterable['Tokens'] + ) -> Set[str]: + """Try to satisfy my prerequisites with given outputs. + + Return which outputs I actually depend on. + """ + valid: Set[str] = set() + for prereq in (*self.prerequisites, *self.suicide_prerequisites): + yep = prereq.satisfy_me(outputs) + if yep: + valid = valid.union(yep) + continue + self._is_satisfied = None + self._suicide_is_satisfied = None + return valid def xtriggers_all_satisfied(self): """Return True if all xtriggers are satisfied.""" @@ -324,6 +336,12 @@ def external_triggers_all_satisfied(self): """Return True if all external triggers are satisfied.""" return all(self.external_triggers.values()) + def set_all_satisfied(self): + """Set all my prerequisites satisfied.""" + for p in self.prerequisites: + p.set_satisfied() + self._is_satisfied = True + def prerequisites_all_satisfied(self): """Return True if (non-suicide) prerequisites are fully satisfied.""" if self._is_satisfied is None: @@ -380,12 +398,10 @@ def get_resolved_dependencies(self): ) def reset( - self, status=None, is_held=None, is_queued=None, is_runahead=None): - """Change status, and manipulate outputs and prerequisites accordingly. - - Outputs are manipulated on manual state reset to reflect the new task - status. Since spawn-on-demand implementation, state reset is only used - for internal state changes. + self, status=None, is_held=None, is_queued=None, is_runahead=None, + forced=False + ): + """Change status. Args: status (str): @@ -395,7 +411,7 @@ def reset( unchanged. Returns: - returns: whether state change or not (bool) + returns: whether state changed or not (bool) """ current_status = ( @@ -414,6 +430,14 @@ def reset( # no change - do nothing return False + if ( + forced and + requested_status in [TASK_STATUS_SUBMITTED, TASK_STATUS_RUNNING] + ): + # For manual setting of task outputs: return True but don't action + # the state change (there's no real job associated with the task). + return True + # perform the actual state change self.status, self.is_held, self.is_queued, self.is_runahead = ( requested_status @@ -421,32 +445,13 @@ def reset( self.time_updated = get_current_time_string() self.is_updated = True - - if is_held: - # only reset task outputs if not setting task to held - # https://github.com/cylc/cylc-flow/pull/2116 - return True - self.kill_failed = False # Set standard outputs in accordance with task state. + if status is None: # NOTE: status is None if the task is being released status = self.status - if status_leq(status, TASK_STATUS_SUBMITTED): - self.outputs.set_all_incomplete() - self.outputs.set_completion( - TASK_OUTPUT_EXPIRED, status == TASK_STATUS_EXPIRED) - self.outputs.set_completion( - TASK_OUTPUT_SUBMITTED, status_geq(status, TASK_STATUS_SUBMITTED)) - self.outputs.set_completion( - TASK_OUTPUT_STARTED, status_geq(status, TASK_STATUS_RUNNING)) - self.outputs.set_completion( - TASK_OUTPUT_SUBMIT_FAILED, status == TASK_STATUS_SUBMIT_FAILED) - self.outputs.set_completion( - TASK_OUTPUT_SUCCEEDED, status == TASK_STATUS_SUCCEEDED) - self.outputs.set_completion( - TASK_OUTPUT_FAILED, status == TASK_STATUS_FAILED) return True diff --git a/cylc/flow/taskdef.py b/cylc/flow/taskdef.py index a4ae681e2d9..afa5eaff7ca 100644 --- a/cylc/flow/taskdef.py +++ b/cylc/flow/taskdef.py @@ -22,13 +22,13 @@ import cylc.flow.flags from cylc.flow.exceptions import TaskDefError from cylc.flow.task_id import TaskID -from cylc.flow.task_state import ( +from cylc.flow.task_outputs import ( TASK_OUTPUT_SUBMITTED, TASK_OUTPUT_SUBMIT_FAILED, TASK_OUTPUT_SUCCEEDED, - TASK_OUTPUT_FAILED + TASK_OUTPUT_FAILED, + STANDARD_OUTPUTS ) -from cylc.flow.task_outputs import SORT_ORDERS if TYPE_CHECKING: from cylc.flow.cycling import PointBase @@ -181,7 +181,7 @@ def add_output(self, output, message): def _add_std_outputs(self): """Add the standard outputs.""" # optional/required is None until defined by the graph - for output in SORT_ORDERS: + for output in STANDARD_OUTPUTS: self.outputs[output] = (output, None) def set_required_output(self, output, required): @@ -190,6 +190,14 @@ def set_required_output(self, output, required): message, _ = self.outputs[output] self.outputs[output] = (message, required) + def get_required_outputs(self): + """Return list of required outputs.""" + res = [] + for out, (_msg, req) in self.outputs.items(): + if req: + res.append(out) + return res + def tweak_outputs(self): """Output consistency checking and tweaking.""" diff --git a/cylc/flow/tui/data.py b/cylc/flow/tui/data.py index 04c3d7220b1..90344e2462b 100644 --- a/cylc/flow/tui/data.py +++ b/cylc/flow/tui/data.py @@ -110,6 +110,7 @@ 'kill', 'trigger', 'poll', + 'set', ], 'job': [ 'kill', diff --git a/cylc/flow/workflow_db_mgr.py b/cylc/flow/workflow_db_mgr.py index c370992f7b0..4b2ade047da 100644 --- a/cylc/flow/workflow_db_mgr.py +++ b/cylc/flow/workflow_db_mgr.py @@ -423,13 +423,35 @@ def put_update_task_state(self, itask): "time_updated": itask.state.time_updated, "status": itask.state.status, "flow_wait": itask.flow_wait, - "is_manual_submit": itask.is_manual_submit + "is_manual_submit": itask.is_manual_submit, + "is_complete": itask.is_complete() + } + where_args = { + "cycle": str(itask.point), + "name": itask.tdef.name, + "flow_nums": serialise(itask.flow_nums), + } + # Note tasks_states table rows are for latest submit_num only + # (not one row per submit). + self.db_updates_map.setdefault(self.TABLE_TASK_STATES, []) + self.db_updates_map[self.TABLE_TASK_STATES].append( + (set_args, where_args)) + + def put_update_task_flow_wait(self, itask): + """Update flow_wait status of a task, in the task_states table. + + NOTE the task_states table is normally updated along with the task pool + table. This method is only needed as a final update for a non-pool task + that just spawned its children after a flow wait. + """ + set_args = { + "time_updated": itask.state.time_updated, + "flow_wait": itask.flow_wait, } where_args = { "cycle": str(itask.point), "name": itask.tdef.name, "flow_nums": serialise(itask.flow_nums), - "submit_num": itask.submit_num, } self.db_updates_map.setdefault(self.TABLE_TASK_STATES, []) self.db_updates_map[self.TABLE_TASK_STATES].append( @@ -453,7 +475,7 @@ def put_task_pool(self, pool: 'TaskPool') -> None: # This should already be done by self.put_task_event_timers above: # self.db_deletes_map[self.TABLE_TASK_ACTION_TIMERS].append({}) self.db_deletes_map[self.TABLE_TASK_TIMEOUT_TIMERS].append({}) - for itask in pool.get_all_tasks(): + for itask in pool.get_tasks(): for prereq in itask.state.prerequisites: for (p_cycle, p_name, p_output), satisfied_state in ( prereq.satisfied.items() @@ -507,7 +529,9 @@ def put_task_pool(self, pool: 'TaskPool') -> None: "time_updated": itask.state.time_updated, "submit_num": itask.submit_num, "try_num": itask.get_try_num(), - "status": itask.state.status + "status": itask.state.status, + "is_manual_submit": itask.is_manual_submit, + "is_complete": itask.is_complete() } where_args = { "cycle": str(itask.point), @@ -736,6 +760,25 @@ def upgrade_pre_810(pri_dao: CylcWorkflowDAO) -> None: ) conn.commit() + @classmethod + def upgrade_pre_830(cls, pri_dao: CylcWorkflowDAO) -> None: + """Upgrade on restart from a pre-8.3.0 database. + + Add "is_complete" column to the task states table. + """ + conn = pri_dao.connect() + c_name = "is_complete" + LOG.info( + f"DB upgrade (pre-8.3.0): " + f"add {c_name} column to {cls.TABLE_TASK_STATES}" + ) + conn.execute( + rf"ALTER TABLE {cls.TABLE_TASK_STATES} " + rf"ADD COLUMN {c_name} INTEGER " + r"DEFAULT 0 NOT NULL" + ) + conn.commit() + @classmethod def upgrade(cls, db_file: Union['Path', str]) -> None: """Upgrade this database to this Cylc version. @@ -746,6 +789,8 @@ def upgrade(cls, db_file: Union['Path', str]) -> None: cls.upgrade_pre_803(pri_dao) if last_run_ver < parse_version("8.1.0.dev"): cls.upgrade_pre_810(pri_dao) + if last_run_ver < parse_version("8.3.0.dev"): + cls.upgrade_pre_830(pri_dao) @classmethod def check_db_compatibility(cls, db_file: Union['Path', str]) -> Version: diff --git a/setup.cfg b/setup.cfg index 7db9eb8bf4e..a8ee30cf79d 100644 --- a/setup.cfg +++ b/setup.cfg @@ -189,11 +189,11 @@ cylc.command = remove = cylc.flow.scripts.remove:main report-timings = cylc.flow.scripts.report_timings:main [report-timings] scan = cylc.flow.scripts.scan:cli - set-verbosity = cylc.flow.scripts.set_verbosity:main show = cylc.flow.scripts.show:main - set-outputs = cylc.flow.scripts.set_outputs:main + set = cylc.flow.scripts.set:main stop = cylc.flow.scripts.stop:main subscribe = cylc.flow.scripts.subscribe:main + verbosity = cylc.flow.scripts.verbosity:main workflow-state = cylc.flow.scripts.workflow_state:main tui = cylc.flow.scripts.tui:main trigger = cylc.flow.scripts.trigger:main diff --git a/tests/conftest.py b/tests/conftest.py index 8ed3e210fc6..4207a10f165 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -112,7 +112,7 @@ def _log_filter( if (name is None or name == log_name) and (level is None or level == log_level) and (contains is None or contains in log_message) - and (regex is None or re.match(regex, log_message)) + and (regex is None or re.search(regex, log_message)) and (exact_match is None or exact_match == log_message) ] return _log_filter diff --git a/tests/flakyfunctional/cylc-poll/03-poll-all/flow.cylc b/tests/flakyfunctional/cylc-poll/03-poll-all/flow.cylc index 5e21a978fea..709e3bb16e6 100644 --- a/tests/flakyfunctional/cylc-poll/03-poll-all/flow.cylc +++ b/tests/flakyfunctional/cylc-poll/03-poll-all/flow.cylc @@ -41,7 +41,7 @@ cylc poll "${CYLC_WORKFLOW_ID}//*" cylc__job__poll_grep_workflow_log \ - "${CYLC_TASK_CYCLE_POINT}/submit_hold preparing .* => submitted" + "${CYLC_TASK_CYCLE_POINT}/submit_hold/01:preparing.* => submitted" st_file="${CYLC_WORKFLOW_RUN_DIR}/log/job/${CYLC_TASK_CYCLE_POINT}/submit_hold/NN/job.status" pkill -g "$(awk -F= '$1 == "CYLC_JOB_ID" {print $2}' "${st_file}")" diff --git a/tests/flakyfunctional/cylc-poll/16-execution-time-limit.t b/tests/flakyfunctional/cylc-poll/16-execution-time-limit.t index 830a4f71aff..c437867251d 100755 --- a/tests/flakyfunctional/cylc-poll/16-execution-time-limit.t +++ b/tests/flakyfunctional/cylc-poll/16-execution-time-limit.t @@ -58,14 +58,14 @@ __PYTHON__ LOG="${WORKFLOW_RUN_DIR}/log/scheduler/log" # Test logging of the "next job poll" message when task starts. TEST_NAME="${TEST_NAME_BASE}-log-entry" -LINE="$(grep '\[1/foo\ .* execution timeout=None, polling intervals=' "${LOG}")" +LINE="$(grep '\[1/foo.* execution timeout=None, polling intervals=' "${LOG}")" run_ok "${TEST_NAME}" grep -q 'health: execution timeout=None, polling intervals=' <<< "${LINE}" # Determine poll times. PREDICTED_POLL_TIME=$(time_offset \ "$(cut -d ' ' -f 1 <<< "${LINE}")" \ "PT10S") # PT5S time limit + PT5S polling interval ACTUAL_POLL_TIME=$(sed -n \ - 's|\(.*\) DEBUG - \[1/foo running .* (polled)failed .*|\1|p' "${LOG}") + 's|\(.*\) DEBUG - \[1/foo.* (polled)failed .*|\1|p' "${LOG}") # Test execution timeout polling. # Main loop is roughly 1 second, but integer rounding may give an apparent 2 diff --git a/tests/flakyfunctional/database/00-simple/schema.out b/tests/flakyfunctional/database/00-simple/schema.out index 8ed66a1f2db..c9d1a610017 100644 --- a/tests/flakyfunctional/database/00-simple/schema.out +++ b/tests/flakyfunctional/database/00-simple/schema.out @@ -10,7 +10,7 @@ CREATE TABLE task_late_flags(cycle TEXT, name TEXT, value INTEGER, PRIMARY KEY(c CREATE TABLE task_outputs(cycle TEXT, name TEXT, flow_nums TEXT, outputs TEXT, PRIMARY KEY(cycle, name, flow_nums)); CREATE TABLE task_pool(cycle TEXT, name TEXT, flow_nums TEXT, status TEXT, is_held INTEGER, PRIMARY KEY(cycle, name, flow_nums)); CREATE TABLE task_prerequisites(cycle TEXT, name TEXT, flow_nums TEXT, prereq_name TEXT, prereq_cycle TEXT, prereq_output TEXT, satisfied TEXT, PRIMARY KEY(cycle, name, flow_nums, prereq_name, prereq_cycle, prereq_output)); -CREATE TABLE task_states(name TEXT, cycle TEXT, flow_nums TEXT, time_created TEXT, time_updated TEXT, submit_num INTEGER, status TEXT, flow_wait INTEGER, is_manual_submit INTEGER, PRIMARY KEY(name, cycle, flow_nums)); +CREATE TABLE task_states(name TEXT, cycle TEXT, flow_nums TEXT, time_created TEXT, time_updated TEXT, submit_num INTEGER, status TEXT, flow_wait INTEGER, is_manual_submit INTEGER, is_complete INTEGER, PRIMARY KEY(name, cycle, flow_nums)); CREATE TABLE task_timeout_timers(cycle TEXT, name TEXT, timeout REAL, PRIMARY KEY(cycle, name)); CREATE TABLE tasks_to_hold(name TEXT, cycle TEXT); CREATE TABLE workflow_flows(flow_num INTEGER, start_time TEXT, description TEXT, PRIMARY KEY(flow_num)); diff --git a/tests/flakyfunctional/hold-release/14-hold-kill/flow.cylc b/tests/flakyfunctional/hold-release/14-hold-kill/flow.cylc index 7b82bd3e15c..9a2c51a88fd 100644 --- a/tests/flakyfunctional/hold-release/14-hold-kill/flow.cylc +++ b/tests/flakyfunctional/hold-release/14-hold-kill/flow.cylc @@ -9,7 +9,7 @@ echo '# killing "sleeper"' cylc kill "${CYLC_WORKFLOW_ID}//1/sleeper" cylc__job__poll_grep_workflow_log -E \ - '1/sleeper\ waiting\(held\) .* job killed' + '1/sleeper:waiting\(held\).* job killed' sleep 10 # sleep, should still be held after 10 seconds cylc dump -s -t "${CYLC_WORKFLOW_ID}" >'cylc-dump.out' diff --git a/tests/flakyfunctional/hold-release/15-hold-after/flow.cylc b/tests/flakyfunctional/hold-release/15-hold-after/flow.cylc index 71cb8d14b25..fe9e2067a34 100644 --- a/tests/flakyfunctional/hold-release/15-hold-after/flow.cylc +++ b/tests/flakyfunctional/hold-release/15-hold-after/flow.cylc @@ -23,7 +23,7 @@ script = cylc hold --after '20140101T12' "${CYLC_WORKFLOW_ID}" [[stopper]] script = """ - cylc__job__poll_grep_workflow_log -E '20140101T1200Z/bar .* \(received\)succeeded' + cylc__job__poll_grep_workflow_log -E '20140101T1200Z/bar/01.* \(received\)succeeded' cylc stop "${CYLC_WORKFLOW_ID}" """ [[[job]]] diff --git a/tests/functional/broadcast/07-timeout/flow.cylc b/tests/functional/broadcast/07-timeout/flow.cylc index 46bbc749fe9..055f09e729b 100644 --- a/tests/functional/broadcast/07-timeout/flow.cylc +++ b/tests/functional/broadcast/07-timeout/flow.cylc @@ -18,7 +18,7 @@ [[timeout]] script = """ cylc__job__poll_grep_workflow_log -E \ - "${CYLC_TASK_ID} .* execution timeout after PT1S" + "${CYLC_TASK_ID}.* execution timeout after PT1S" """ [[[events]]] execution timeout = PT1M diff --git a/tests/functional/cli/02-now.t b/tests/functional/cli/02-now.t deleted file mode 100755 index f6bed459275..00000000000 --- a/tests/functional/cli/02-now.t +++ /dev/null @@ -1,62 +0,0 @@ -#!/usr/bin/env bash -# 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 . -#------------------------------------------------------------------------------- -# Test "cylc play --icp=now WORKFLOW". -# And "cylc play --icp=next(...) WORKFLOW" and "cylc play --icp=previous(...) WORKFLOW" -# And restart. - -. "$(dirname "$0")/test_header" -set_test_number 10 -init_workflow "${TEST_NAME_BASE}" <<'__FLOW_CONFIG__' -[scheduler] - [[events]] - abort on stall timeout = true - stall timeout = PT0S - abort on inactivity timeout = true - inactivity timeout = PT1M -[scheduling] - [[graph]] - R1 = foo -[runtime] - [[foo]] - script = cylc__job__wait_cylc_message_started; cylc stop --now --now "${CYLC_WORKFLOW_ID}" -__FLOW_CONFIG__ - -run_ok "${TEST_NAME_BASE}-validate" cylc validate --icp='now' "${WORKFLOW_NAME}" - -for ICP in 'now' 'next(T00)' 'previous(T00)'; do - workflow_run_ok "${TEST_NAME_BASE}-run-icp-${ICP}" \ - cylc play --debug --no-detach --icp="${ICP}" "${WORKFLOW_NAME}" - - workflow_run_ok "${TEST_NAME_BASE}-restart-icp-${ICP}" \ - cylc play --debug --no-detach "${WORKFLOW_NAME}" - - sqlite3 "${WORKFLOW_RUN_DIR}/log/db" 'SELECT * FROM task_pool' >'task_pool.out' - cmp_ok 'task_pool.out' <'/dev/null' - # TODO - is this test still useful? - # Consider checking the task succeeded in the task_states table, instead. - # pre-SoD: - # sqlite3 "${WORKFLOW_RUN_DIR}/log/db" 'SELECT * FROM task_pool' >'task_pool.out' - # cmp_ok 'task_pool.out' <<__OUT__ - # ${MY_CYCLE}|foo|1|succeeded|0 - # __OUT__ - - delete_db -done - -purge -exit diff --git a/tests/functional/cli/03-set-verbosity.t b/tests/functional/cli/03-verbosity.t similarity index 89% rename from tests/functional/cli/03-set-verbosity.t rename to tests/functional/cli/03-verbosity.t index 7938f91cab7..b3103fd67d3 100755 --- a/tests/functional/cli/03-set-verbosity.t +++ b/tests/functional/cli/03-verbosity.t @@ -15,13 +15,13 @@ # You should have received a copy of the GNU General Public License # along with this program. If not, see . #------------------------------------------------------------------------------- -# Test "cylc set-verbosity" +# Test "cylc verbosity" . "$(dirname "$0")/test_header" set_test_number 4 # Test illegal log level TEST_NAME="${TEST_NAME_BASE}-bad" -run_fail "$TEST_NAME" cylc set-verbosity duck quack +run_fail "$TEST_NAME" cylc verbosity duck quack grep_ok 'InputError: Illegal logging level, duck' "${TEST_NAME}.stderr" # Test good log level @@ -42,8 +42,8 @@ init_workflow "${TEST_NAME_BASE}" << '__FLOW__' echo "CYLC_DEBUG: $CYLC_DEBUG" [[ "$CYLC_DEBUG" != 'true' ]] - cylc set-verbosity DEBUG "$CYLC_WORKFLOW_ID" - cylc__job__poll_grep_workflow_log 'Command actioned: set_verbosity' + cylc verbosity DEBUG "$CYLC_WORKFLOW_ID" + cylc__job__poll_grep_workflow_log 'Command "set_verbosity" actioned' """ [[getter]] script = """ diff --git a/tests/functional/clock-expire/00-basic/flow.cylc b/tests/functional/clock-expire/00-basic/flow.cylc index afeb7df356d..3605ca885e8 100644 --- a/tests/functional/clock-expire/00-basic/flow.cylc +++ b/tests/functional/clock-expire/00-basic/flow.cylc @@ -23,7 +23,7 @@ Skip a daily post-processing workflow if the 'copy' task has expired.""" # behind "now + 1 day". This makes the first two 'copy' tasks expire. [[graph]] P1D = """ - model[-P1D] => model => copy => proc + model[-P1D] => model => copy? => proc copy:expired? => !proc """ [runtime] diff --git a/tests/functional/cylc-cat-log/05-remote-tail.t b/tests/functional/cylc-cat-log/05-remote-tail.t index f3a787b5675..e75a87df862 100755 --- a/tests/functional/cylc-cat-log/05-remote-tail.t +++ b/tests/functional/cylc-cat-log/05-remote-tail.t @@ -41,7 +41,7 @@ $SCP "${PWD}/bin/my-tailer.sh" \ # Run detached. workflow_run_ok "${TEST_NAME_BASE}-run" cylc play "${WORKFLOW_NAME}" #------------------------------------------------------------------------------- -poll_grep_workflow_log -E '1/foo preparing .* => submitted' +poll_grep_workflow_log -E '1/foo/01:preparing.* => submitted' # cylc cat-log -m 't' tail-follows a file, so needs to be killed. # Send interrupt signal to tail command after 15 seconds. TEST_NAME="${TEST_NAME_BASE}-cat-log" diff --git a/tests/functional/cylc-kill/03-simulation.t b/tests/functional/cylc-kill/03-simulation.t index f9152a84a1f..dadfeede123 100755 --- a/tests/functional/cylc-kill/03-simulation.t +++ b/tests/functional/cylc-kill/03-simulation.t @@ -28,7 +28,7 @@ run_ok "${TEST_NAME_BASE}-validate" cylc validate "${WORKFLOW_NAME}" cylc play --debug -m simulation "${WORKFLOW_NAME}" >/dev/null 2>&1 # wait for simulated job start -poll_grep_workflow_log "1/foo .* running" -E +poll_grep_workflow_log "1/foo.* running" -E # kill it run_ok killer cylc kill "${WORKFLOW_NAME}//1/foo" @@ -37,6 +37,6 @@ run_ok killer cylc kill "${WORKFLOW_NAME}//1/foo" poll_grep_workflow_log "INFO - DONE" # check the sim job was kiled -grep_workflow_log_ok killed "1/foo .* failed" -E +grep_workflow_log_ok killed "1/foo.* failed" -E purge diff --git a/tests/functional/cylc-message/02-multi.t b/tests/functional/cylc-message/02-multi.t index eab3148cde4..17a0e3b4189 100755 --- a/tests/functional/cylc-message/02-multi.t +++ b/tests/functional/cylc-message/02-multi.t @@ -57,19 +57,19 @@ sed -i 's/\(^.*\) at .*$/\1/;' 'sed.out' # Note: the continuation bit gets printed twice, because the message gets a # warning as being unhandled. cmp_ok 'sed.out' <<__LOG__ -DEBUG - [1/foo submitted job:01 flows:1] (received)started -WARNING - [1/foo running job:01 flows:1] (received)Warn this -INFO - [1/foo running job:01 flows:1] (received)Greeting -WARNING - [1/foo running job:01 flows:1] (received)Warn that -DEBUG - [1/foo running job:01 flows:1] (received)Remove stuffs such as +DEBUG - [1/foo/01:submitted] (received)started +WARNING - [1/foo/01:running] (received)Warn this +INFO - [1/foo/01:running] (received)Greeting +WARNING - [1/foo/01:running] (received)Warn that +DEBUG - [1/foo/01:running] (received)Remove stuffs such as ${LOG_INDENT}badness ${LOG_INDENT}slowness ${LOG_INDENT}and other incorrectness. ${LOG_INDENT}badness ${LOG_INDENT}slowness ${LOG_INDENT}and other incorrectness. -INFO - [1/foo running job:01 flows:1] (received)whatever -DEBUG - [1/foo running job:01 flows:1] (received)succeeded +INFO - [1/foo/01:running] (received)whatever +DEBUG - [1/foo/01:running] (received)succeeded __LOG__ purge diff --git a/tests/functional/cylc-poll/05-poll-multi-messages/flow.cylc b/tests/functional/cylc-poll/05-poll-multi-messages/flow.cylc index 9adf11d9107..18fd4112e51 100644 --- a/tests/functional/cylc-poll/05-poll-multi-messages/flow.cylc +++ b/tests/functional/cylc-poll/05-poll-multi-messages/flow.cylc @@ -20,8 +20,8 @@ echo "CYLC_MESSAGE=$(date +%FT%H:%M:%SZ)|INFO|hello1" echo "CYLC_MESSAGE=$(date +%FT%H:%M:%SZ)|INFO|hello2" } >>"${CYLC_TASK_LOG_ROOT}.status" - cylc__job__poll_grep_workflow_log -E '1/speaker1 running .* \(polled\)hello1' - cylc__job__poll_grep_workflow_log -E '1/speaker1 running .* \(polled\)hello2' + cylc__job__poll_grep_workflow_log -E '1/speaker1/01:running.* \(polled\)hello1' + cylc__job__poll_grep_workflow_log -E '1/speaker1/01:running.* \(polled\)hello2' """ [[[outputs]]] hello1 = "hello1" @@ -34,7 +34,7 @@ # get sent back to the workflow echo "CYLC_MESSAGE=$(date +%FT%H:%M:%SZ)|INFO|greet" \ >>"${CYLC_TASK_LOG_ROOT}.status" - cylc__job__poll_grep_workflow_log -E '1/speaker2 running .* \(polled\)greet' + cylc__job__poll_grep_workflow_log -E '1/speaker2/01:running.* \(polled\)greet' """ [[[outputs]]] greet = "greet" diff --git a/tests/functional/cylc-poll/13-comm-method.t b/tests/functional/cylc-poll/13-comm-method.t index 91da26bd4cd..282f7106671 100755 --- a/tests/functional/cylc-poll/13-comm-method.t +++ b/tests/functional/cylc-poll/13-comm-method.t @@ -39,7 +39,7 @@ PRE_MSG='health:' POST_MSG='.*, polling intervals=10\*PT6S...' for INDEX in 1 2; do for STAGE in 'submission' 'execution'; do - grep_ok "1/t${INDEX} .* ${PRE_MSG} ${STAGE}${POST_MSG}" "${LOG_FILE}" -E + grep_ok "1/t${INDEX}.* ${PRE_MSG} ${STAGE}${POST_MSG}" "${LOG_FILE}" -E done done #------------------------------------------------------------------------------- diff --git a/tests/functional/cylc-poll/14-intervals.t b/tests/functional/cylc-poll/14-intervals.t index 9d9b308f4c3..f7807b27d4d 100755 --- a/tests/functional/cylc-poll/14-intervals.t +++ b/tests/functional/cylc-poll/14-intervals.t @@ -41,7 +41,7 @@ for INDEX in 1 2; do POLL_INT='2\*PT1S,10\*PT6S,' fi POST_MSG=".*, polling intervals=${POLL_INT}..." - grep_ok "1/t${INDEX} .* ${PRE_MSG} ${STAGE}${POST_MSG}" "${LOG_FILE}" -E + grep_ok "1/t${INDEX}.*${PRE_MSG} ${STAGE}${POST_MSG}" "${LOG_FILE}" -E done done #------------------------------------------------------------------------------- diff --git a/tests/functional/cylc-poll/15-job-st-file-no-batch.t b/tests/functional/cylc-poll/15-job-st-file-no-batch.t index 4331cab1b63..017d0c39f1b 100755 --- a/tests/functional/cylc-poll/15-job-st-file-no-batch.t +++ b/tests/functional/cylc-poll/15-job-st-file-no-batch.t @@ -28,7 +28,7 @@ LOG="${WORKFLOW_RUN_DIR}/log/scheduler/log" run_ok "${TEST_NAME_BASE}-log-1" \ grep -F '[jobs-poll err] 1/t1/01/job.status: incomplete job runner info' "${LOG}" run_ok "${TEST_NAME_BASE}-log-2" \ - grep -E '1/t1 running .*\(polled\)failed' "${LOG}" + grep -E '1/t1/01:running.*\(polled\)failed' "${LOG}" purge exit diff --git a/tests/functional/cylc-remove/00-simple/flow.cylc b/tests/functional/cylc-remove/00-simple/flow.cylc index 15c0b09dc64..84c740ad421 100644 --- a/tests/functional/cylc-remove/00-simple/flow.cylc +++ b/tests/functional/cylc-remove/00-simple/flow.cylc @@ -15,7 +15,7 @@ script = false [[cleaner]] script = """ -cylc__job__poll_grep_workflow_log -E '1/b\ running .* \(received\)failed' +cylc__job__poll_grep_workflow_log -E '1/b/01:running.* \(received\)failed' # Remove the unhandled failed task cylc remove "$CYLC_WORKFLOW_ID//1/b" # Remove waiting 1/c diff --git a/tests/functional/cylc-remove/02-cycling/flow.cylc b/tests/functional/cylc-remove/02-cycling/flow.cylc index a2a236eb004..3b6c1051493 100644 --- a/tests/functional/cylc-remove/02-cycling/flow.cylc +++ b/tests/functional/cylc-remove/02-cycling/flow.cylc @@ -17,8 +17,8 @@ [runtime] [[remover]] script = """ - cylc__job__poll_grep_workflow_log -E '2020/bar running .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '2021/baz running .* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '2020/bar/01:running.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '2021/baz/01:running.* \(received\)failed' # Remove the two unhandled failed tasks. cylc remove "$CYLC_WORKFLOW_ID//*/ba*:failed" # Remove the two unsatisfied waiting tasks. diff --git a/tests/functional/cylc-set/00-set-succeeded.t b/tests/functional/cylc-set/00-set-succeeded.t new file mode 100644 index 00000000000..f76eeb93d11 --- /dev/null +++ b/tests/functional/cylc-set/00-set-succeeded.t @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 1 - Carry on as if a failed task had succeeded +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#1-carry-on-as-if-a-failed-task-had-succeeded + +. "$(dirname "$0")/test_header" +set_test_number 6 + +install_and_validate +reftest_run + +for TASK in foo bar +do + sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT status FROM task_states WHERE name is \"$TASK\"" > "${TASK}.1" + + cmp_ok ${TASK}.1 - << __OUT__ +succeeded +__OUT__ + + sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT outputs FROM task_outputs WHERE name is \"$TASK\"" > "${TASK}.2" + + # Json string list of outputs from the db may not be ordered correctly. + # E.g., '["submitted", "started", "succeeded", "failed"]'. + python3 - << __END__ > "${TASK}.3" +import json +with open("${TASK}.2", 'r') as f: + print( + ','.join( + sorted( + json.load(f) + ) + ) + ) +__END__ + + cmp_ok "${TASK}.3" - << __OUT__ +failed,started,submitted,succeeded +__OUT__ + +done +purge diff --git a/tests/functional/cylc-set/00-set-succeeded/flow.cylc b/tests/functional/cylc-set/00-set-succeeded/flow.cylc new file mode 100644 index 00000000000..2385af8b427 --- /dev/null +++ b/tests/functional/cylc-set/00-set-succeeded/flow.cylc @@ -0,0 +1,41 @@ +# 1. foo and bar fail incomplete. +# 2. setter sets foo and bar to succeeded. +# 3. foo and bar are completed, post runs, scheduler shuts down. + +[scheduler] + [[events]] + inactivity timeout = PT30S + abort on inactivity timeout = True + expected task failures = 1/foo, 1/bar + +[task parameters] + m = 1..2 + +[scheduling] + [[graph]] + R1 = """ + foo & bar => post + setter + """ +[runtime] + [[post]] + [[foo, bar]] + script = false + [[setter]] + script = """ + # wait for foo and bar to fail. + for TASK in foo bar + do + cylc workflow-state \ + --max-polls=10 \ + --interval=1 \ + --task=$TASK \ + --point=${CYLC_TASK_CYCLE_POINT} \ + --status=failed \ + $CYLC_WORKFLOW_ID + done + # set foo succeeded (via --output) + cylc set -o succeeded $CYLC_WORKFLOW_ID//$CYLC_TASK_CYCLE_POINT/foo + # set bar succeeded (via default) + cylc set $CYLC_WORKFLOW_ID//$CYLC_TASK_CYCLE_POINT/bar + """ diff --git a/tests/functional/cylc-set/00-set-succeeded/reference.log b/tests/functional/cylc-set/00-set-succeeded/reference.log new file mode 100644 index 00000000000..26468845a5c --- /dev/null +++ b/tests/functional/cylc-set/00-set-succeeded/reference.log @@ -0,0 +1,5 @@ +1/setter -triggered off [] in flow 1 +1/foo -triggered off [] in flow 1 +1/bar -triggered off [] in flow 1 +1/post_m1 -triggered off ['1/bar', '1/foo'] in flow 1 +1/post_m2 -triggered off ['1/bar', '1/foo'] in flow 1 diff --git a/tests/functional/cylc-set/01-off-flow-pre.t b/tests/functional/cylc-set/01-off-flow-pre.t new file mode 100644 index 00000000000..818feedd445 --- /dev/null +++ b/tests/functional/cylc-set/01-off-flow-pre.t @@ -0,0 +1,37 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- +# +# "cylc set" proposal examples: 2 - Set off-flow prerequisites to prevent a new flow from stalling. +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#2-set-off-flow-prerequisites-to-prep-for-a-new-flow + +. "$(dirname "$0")/test_header" +set_test_number 8 + +install_and_validate +reftest_run + +grep_workflow_log_ok "${TEST_NAME_BASE}-ab" "1/a does not depend on 1/b_cold:succeeded" +grep_workflow_log_ok "${TEST_NAME_BASE}-ac" "1/a does not depend on 1/c_cold:succeeded" + +grep_workflow_log_ok "${TEST_NAME_BASE}-ba" "1/b does not depend on 1/a_cold:succeeded" +grep_workflow_log_ok "${TEST_NAME_BASE}-bc" "1/b does not depend on 1/c_cold:succeeded" + +grep_workflow_log_ok "${TEST_NAME_BASE}-ca" "1/c does not depend on 1/a_cold:succeeded" +grep_workflow_log_ok "${TEST_NAME_BASE}-cb" "1/c does not depend on 1/b_cold:succeeded" + +purge diff --git a/tests/functional/cylc-set/01-off-flow-pre/flow.cylc b/tests/functional/cylc-set/01-off-flow-pre/flow.cylc new file mode 100644 index 00000000000..5c2ae5b97c4 --- /dev/null +++ b/tests/functional/cylc-set/01-off-flow-pre/flow.cylc @@ -0,0 +1,35 @@ +# start a new flow after setting off-flow prerequites to avoid stall. + +[scheduler] + [[events]] + stall timeout = PT0S + abort on stall timeout = True + inactivity timeout = PT30S + abort on inactivity timeout = True + +[scheduling] + [[graph]] + R1 = """ + # the tasks we want the flow to run + a => b => c => reflow + # the off-flow prerequisites + a_cold => a + b_cold => b + c_cold => c + """ +[runtime] + [[a, b, c]] + [[a_cold, b_cold, c_cold]] + [[reflow]] + script = """ + if (( CYLC_TASK_SUBMIT_NUMBER == 1 )); then + # set off-flow prerequisites (and trigger 1/a) + cylc set --flow=new \ + --pre=1/a_cold:succeeded \ + --pre=1/b_cold:succeeded \ + --pre=1/c_cold:succeeded \ + ${CYLC_WORKFLOW_ID}//1/a \ + ${CYLC_WORKFLOW_ID}//1/b \ + ${CYLC_WORKFLOW_ID}//1/c + fi + """ diff --git a/tests/functional/cylc-set/01-off-flow-pre/reference.log b/tests/functional/cylc-set/01-off-flow-pre/reference.log new file mode 100644 index 00000000000..07c980ca981 --- /dev/null +++ b/tests/functional/cylc-set/01-off-flow-pre/reference.log @@ -0,0 +1,11 @@ +1/c_cold -triggered off [] in flow 1 +1/a_cold -triggered off [] in flow 1 +1/b_cold -triggered off [] in flow 1 +1/a -triggered off ['1/a_cold'] in flow 1 +1/b -triggered off ['1/a', '1/b_cold'] in flow 1 +1/c -triggered off ['1/b', '1/c_cold'] in flow 1 +1/reflow -triggered off ['1/c'] in flow 1 +1/a -triggered off ['1/a_cold'] in flow 2 +1/b -triggered off ['1/a', '1/b_cold'] in flow 2 +1/c -triggered off ['1/b', '1/c_cold'] in flow 2 +1/reflow -triggered off ['1/c'] in flow 2 diff --git a/tests/functional/cylc-set/02-off-flow-out.t b/tests/functional/cylc-set/02-off-flow-out.t new file mode 100644 index 00000000000..db3364b65c2 --- /dev/null +++ b/tests/functional/cylc-set/02-off-flow-out.t @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 2 - Set off-flow outputs to prevent a new flow from stalling. +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#2-set-off-flow-prerequisites-to-prep-for-a-new-flow + +. "$(dirname "$0")/test_header" +set_test_number 11 + +install_and_validate +reftest_run + +# Check that we set: +# - all the required outputs of a_cold +# - the requested and implied outputs of b_cold and c_cold + +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a1" '1/a_cold.* setting missed output: submitted' +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a2" '1/a_cold.* setting missed output: started' +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a3" 'output 1/a_cold:succeeded completed' + +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a1" '1/b_cold.* setting missed output: submitted' +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a2" '1/b_cold.* setting missed output: started' +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-b3" 'output 1/b_cold:succeeded completed' + +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a1" '1/c_cold.* setting missed output: submitted' +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-a2" '1/c_cold.* setting missed output: started' +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-c3" 'output 1/c_cold:succeeded completed' + +purge diff --git a/tests/functional/cylc-set/02-off-flow-out/flow.cylc b/tests/functional/cylc-set/02-off-flow-out/flow.cylc new file mode 100644 index 00000000000..f8f2001f077 --- /dev/null +++ b/tests/functional/cylc-set/02-off-flow-out/flow.cylc @@ -0,0 +1,32 @@ +# start a new flow after setting off-flow outputs to avoid stall. + +[scheduler] + [[events]] + stall timeout = PT0S + abort on stall timeout = True + inactivity timeout = PT30S + abort on inactivity timeout = True + +[scheduling] + [[graph]] + R1 = """ + # the tasks we want the flow to run + a => b => c => reflow + # the off-flow prerequisites + a_cold => a + b_cold => b + c_cold => c + """ +[runtime] + [[a, b, c]] + [[a_cold, b_cold, c_cold]] + [[reflow]] + script = """ + if (( CYLC_TASK_SUBMIT_NUMBER == 1 )); then + # set off-flow outputs of x_cold + cylc set --flow=new \ + ${CYLC_WORKFLOW_ID}//1/a_cold \ + ${CYLC_WORKFLOW_ID}//1/b_cold \ + ${CYLC_WORKFLOW_ID}//1/c_cold + fi + """ diff --git a/tests/functional/cylc-set/02-off-flow-out/reference.log b/tests/functional/cylc-set/02-off-flow-out/reference.log new file mode 100644 index 00000000000..07c980ca981 --- /dev/null +++ b/tests/functional/cylc-set/02-off-flow-out/reference.log @@ -0,0 +1,11 @@ +1/c_cold -triggered off [] in flow 1 +1/a_cold -triggered off [] in flow 1 +1/b_cold -triggered off [] in flow 1 +1/a -triggered off ['1/a_cold'] in flow 1 +1/b -triggered off ['1/a', '1/b_cold'] in flow 1 +1/c -triggered off ['1/b', '1/c_cold'] in flow 1 +1/reflow -triggered off ['1/c'] in flow 1 +1/a -triggered off ['1/a_cold'] in flow 2 +1/b -triggered off ['1/a', '1/b_cold'] in flow 2 +1/c -triggered off ['1/b', '1/c_cold'] in flow 2 +1/reflow -triggered off ['1/c'] in flow 2 diff --git a/tests/functional/cylc-set/03-set-failed.t b/tests/functional/cylc-set/03-set-failed.t new file mode 100644 index 00000000000..07ba070aa02 --- /dev/null +++ b/tests/functional/cylc-set/03-set-failed.t @@ -0,0 +1,65 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 4 -check that we can set a dead orphaned job to failed. +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#4-set-jobs-to-failed-when-a-job-platform-is-known-to-be-down + +. "$(dirname "$0")/test_header" +set_test_number 4 + +install_and_validate + +run_ok play-it cylc play --debug "${WORKFLOW_NAME}" + +poll_grep_workflow_log -E "1/foo.* \(internal\)submitted" + +cylc set -o failed "${WORKFLOW_NAME}//1/foo" + +poll_grep_workflow_log -E "1/foo.* => failed" +poll_grep_workflow_log -E "1/foo.* did not complete required outputs" + +cylc stop --now --now --interval=2 --max-polls=5 "${WORKFLOW_NAME}" + +# Check the log for: +# - set completion message +# - implied outputs reported as already completed + +grep_workflow_log_ok "${TEST_NAME_BASE}-grep-3" 'set: output 1/foo:failed completed' + +# Check the DB records all the outputs. +sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT outputs FROM task_outputs WHERE name is \"foo\"" > db-foo.1 + +# Json string list of outputs from the db may not be ordered correctly. +python3 - << __END__ > db-foo.2 +import json +with open("db-foo.1", 'r') as f: + print( + ','.join( + sorted( + json.load(f) + ) + ) + ) +__END__ + +cmp_ok "db-foo.2" - << __OUT__ +failed,started,submitted +__OUT__ + +purge diff --git a/tests/functional/cylc-set/03-set-failed/flow.cylc b/tests/functional/cylc-set/03-set-failed/flow.cylc new file mode 100644 index 00000000000..9d7514ccb83 --- /dev/null +++ b/tests/functional/cylc-set/03-set-failed/flow.cylc @@ -0,0 +1,18 @@ +# A single task that dies silently, requiring set to failed + +[scheduler] + [[events]] + inactivity timeout = PT20S + abort on inactivity timeout = True + +[scheduling] + [[graph]] + R1 = "foo" + +[runtime] + [[foo]] + init-script = cylc__job__disable_fail_signals + script = """ + cylc__job__wait_cylc_message_started + exit 1 + """ diff --git a/tests/functional/cylc-set/04-switch.t b/tests/functional/cylc-set/04-switch.t new file mode 100644 index 00000000000..4a48d0f5d01 --- /dev/null +++ b/tests/functional/cylc-set/04-switch.t @@ -0,0 +1,65 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 5 - Set and complete a future switch task with the "--wait" flag +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#5-set-switch-tasks-at-an-optional-branch-point-to-direct-the-future-flow + +. "$(dirname "$0")/test_header" +set_test_number 5 + +install_and_validate +reftest_run + +# The branch-point task foo should be recorded as succeeded. + +sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT status FROM task_states WHERE name is \"foo\"" > db-foo.2 + +cmp_ok "db-foo.2" - << __OUT__ +succeeded +__OUT__ + +# the outputs of foo should be recorded as: +# a, succeeded +# and the implied outputs (of succeeded) as well: +# submitted, started + +sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT outputs FROM task_outputs WHERE name is \"foo\"" > db-foo.1 + +# Json string list of outputs from the db may not be ordered correctly. +python3 - << __END__ > db-foo.2 +import json +with open("db-foo.1", 'r') as f: + print( + ','.join( + sorted( + json.load(f) + ) + ) + ) +__END__ + +cmp_ok "db-foo.2" - << __OUT__ +a,started,submitted,succeeded +__OUT__ + +# Check the flow-wait worked +grep_workflow_log_ok check-wait "spawning children of 1/foo.* after flow wait" -E + +purge diff --git a/tests/functional/cylc-set/04-switch/flow.cylc b/tests/functional/cylc-set/04-switch/flow.cylc new file mode 100644 index 00000000000..8a0ded59ce0 --- /dev/null +++ b/tests/functional/cylc-set/04-switch/flow.cylc @@ -0,0 +1,31 @@ +# Set outputs of future task to direct the flow at an optional branch point. + +[scheduler] + [[events]] + inactivity timeout = PT20S + abort on inactivity timeout = True + stall timeout = PT0S + abort on stall timeout = True + +[scheduling] + [[graph]] + R1 = """ + switcher => foo + foo:a? => a + foo:b? => b + """ +[runtime] + [[switcher]] + script = """ + cylc set --output=a,succeeded --wait ${CYLC_WORKFLOW_ID}//1/foo + # wait for command actioned, to avoid race condition + cylc__job__poll_grep_workflow_log "actioned" + """ + [[foo]] + script = "cylc message b" # always go b-way if I run + [[[outputs]]] + a = a + b = b + [[a]] + [[b]] + script = false diff --git a/tests/functional/cylc-set/04-switch/reference.log b/tests/functional/cylc-set/04-switch/reference.log new file mode 100644 index 00000000000..e4fe80d6929 --- /dev/null +++ b/tests/functional/cylc-set/04-switch/reference.log @@ -0,0 +1,2 @@ +1/switcher -triggered off [] in flow 1 +1/a -triggered off ['1/foo'] in flow 1 diff --git a/tests/functional/cylc-set/05-expire.t b/tests/functional/cylc-set/05-expire.t new file mode 100644 index 00000000000..dae60be234e --- /dev/null +++ b/tests/functional/cylc-set/05-expire.t @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 6 - check that forced task expiry works +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#6-expire-a-task + +. "$(dirname "$0")/test_header" +set_test_number 4 + +install_and_validate +reftest_run + +sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT status FROM task_states WHERE name is \"bar\"" > db-bar.1 + +cmp_ok "db-bar.1" - << __OUT__ +expired +__OUT__ + +sqlite3 ~/cylc-run/"${WORKFLOW_NAME}"/log/db \ + "SELECT outputs FROM task_outputs WHERE name is \"bar\"" > db-bar.2 + +cmp_ok "db-bar.2" - << __OUT__ +["expired"] +__OUT__ + +purge diff --git a/tests/functional/cylc-set/05-expire/flow.cylc b/tests/functional/cylc-set/05-expire/flow.cylc new file mode 100644 index 00000000000..57d94dbb99e --- /dev/null +++ b/tests/functional/cylc-set/05-expire/flow.cylc @@ -0,0 +1,24 @@ +# Expire a future task, so it won't run. + +[scheduler] + [[events]] + inactivity timeout = PT20S + abort on inactivity timeout = True + stall timeout = PT0S + abort on stall timeout = True + +[scheduling] + [[graph]] + R1 = """ + # bar and baz should not run if bar expires + expirer => foo => bar? => baz + bar:expired? + """ +[runtime] + [[expirer]] + script = """ + cylc set --output=expired ${CYLC_WORKFLOW_ID}//1/bar + """ + [[foo]] + [[bar, baz]] + script = false diff --git a/tests/functional/cylc-set/05-expire/reference.log b/tests/functional/cylc-set/05-expire/reference.log new file mode 100644 index 00000000000..0966b1b6f90 --- /dev/null +++ b/tests/functional/cylc-set/05-expire/reference.log @@ -0,0 +1,2 @@ +1/expirer -triggered off [] in flow 1 +1/foo -triggered off ['1/expirer'] in flow 1 diff --git a/tests/functional/cylc-set/06-parentless.t b/tests/functional/cylc-set/06-parentless.t new file mode 100644 index 00000000000..9ccf16e30e0 --- /dev/null +++ b/tests/functional/cylc-set/06-parentless.t @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 7 - Check spawning a parentless task without ignoring xtriggers. +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#7-spawning-parentless-tasks + +. "$(dirname "$0")/test_header" +set_test_number 3 + +install_and_validate +REFTEST_OPTS="--start-task=1800/a" reftest_run + +grep_workflow_log_ok "${TEST_NAME_BASE}-clock" "xtrigger satisfied: wall_clock" + +purge diff --git a/tests/functional/cylc-set/06-parentless/flow.cylc b/tests/functional/cylc-set/06-parentless/flow.cylc new file mode 100644 index 00000000000..5078b84e484 --- /dev/null +++ b/tests/functional/cylc-set/06-parentless/flow.cylc @@ -0,0 +1,22 @@ +# Start this with --start-task=1800/a. +# It should stall because x => b is off-flow. +# The stall handler should unstall it by spawning x. +# The log should show a clock-trigger check before x runs. + +[scheduler] + [[events]] + inactivity timeout = PT30S + abort on inactivity timeout = True + stall timeout = PT10S + abort on stall timeout = True + stall handlers = "cylc set --pre=all %(workflow)s//1800/x" + +[scheduling] + initial cycle point = 1800 + [[graph]] + R1 = """ + a => b + @wall_clock => x => b + """ +[runtime] + [[a, b, x]] diff --git a/tests/functional/cylc-set/06-parentless/reference.log b/tests/functional/cylc-set/06-parentless/reference.log new file mode 100644 index 00000000000..f977d1f086b --- /dev/null +++ b/tests/functional/cylc-set/06-parentless/reference.log @@ -0,0 +1,4 @@ +Start task: ['1800/a'] +18000101T0000Z/a -triggered off [] in flow 1 +18000101T0000Z/x -triggered off [] in flow 1 +18000101T0000Z/b -triggered off ['18000101T0000Z/a', '18000101T0000Z/x'] in flow 1 diff --git a/tests/functional/cylc-set/08-switch2.t b/tests/functional/cylc-set/08-switch2.t new file mode 100644 index 00000000000..3f947668501 --- /dev/null +++ b/tests/functional/cylc-set/08-switch2.t @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- + +# "cylc set" proposal examples: 5 - Set and complete a future switch task. +# https://cylc.github.io/cylc-admin/proposal-cylc-set.html#5-set-switch-tasks-at-an-optional-branch-point-to-direct-the-future-flow + +. "$(dirname "$0")/test_header" +set_test_number 2 + +reftest + +purge diff --git a/tests/functional/cylc-set/08-switch2/flow.cylc b/tests/functional/cylc-set/08-switch2/flow.cylc new file mode 100644 index 00000000000..0a221f1227e --- /dev/null +++ b/tests/functional/cylc-set/08-switch2/flow.cylc @@ -0,0 +1,41 @@ + +# Complete a parentless switch task that already exists in the pool but is +# beyond the runahead limit. Cylc should auto-spawn its next instance to +# avoid premature shutdown when it is removed as complete. +# (We only spawn the first runahead-limited instance of parentless tasks). + +[scheduler] + allow implicit tasks = True + +[scheduling] + initial cycle point = 1 + final cycle point = 4 + cycling mode = integer + runahead limit = P0 + [[graph]] + P1 = """ + a:x? => x + a:y? => y + x | y => z + """ +[runtime] + [[a]] + script = """ + cylc__job__wait_cylc_message_started + cylc message -- x # always go x-path + """ + [[[outputs]]] + x = x + y = y + [[z]] + script = """ + if (( CYLC_TASK_CYCLE_POINT == 1 )); then + # mark 2/a as succeeded with output y + # (task will be skipped) + cylc set "${CYLC_WORKFLOW_ID}//2/a" --out=y,succeeded + elif (( CYLC_TASK_CYCLE_POINT == 2 )); then + # mark 2/a as having generated output y + # (task will re-run and generate output x in the prociess) + cylc set "${CYLC_WORKFLOW_ID}//3/a" --out=y + fi + """ diff --git a/tests/functional/cylc-set/08-switch2/reference.log b/tests/functional/cylc-set/08-switch2/reference.log new file mode 100644 index 00000000000..a41fff43a9e --- /dev/null +++ b/tests/functional/cylc-set/08-switch2/reference.log @@ -0,0 +1,17 @@ +# 1/a runs naturally and generates the output "x" +1/a -triggered off [] in flow 1 +1/x -triggered off ['1/a'] in flow 1 +1/z -triggered off ['1/x'] in flow 1 +# 1/a is artificially completed with the output "y" +2/y -triggered off ['2/a'] in flow 1 +2/z -triggered off ['2/y'] in flow 1 +# 1/a has the output "y" is artificially set but is not completed +# (so 1/a will re-run and generate the output "x" naturally) +3/a -triggered off [] in flow 1 +3/x -triggered off ['3/a'] in flow 1 +3/y -triggered off ['3/a'] in flow 1 +3/z -triggered off ['3/y'] in flow 1 +# 1/a runs naturally and generates the output "x" +4/a -triggered off [] in flow 1 +4/x -triggered off ['4/a'] in flow 1 +4/z -triggered off ['4/x'] in flow 1 diff --git a/tests/functional/cylc-set/test_header b/tests/functional/cylc-set/test_header new file mode 120000 index 00000000000..90bd5a36f92 --- /dev/null +++ b/tests/functional/cylc-set/test_header @@ -0,0 +1 @@ +../lib/bash/test_header \ No newline at end of file diff --git a/tests/functional/cylc-trigger/02-filter-failed/flow.cylc b/tests/functional/cylc-trigger/02-filter-failed/flow.cylc index e5a3065782e..7416cf5790d 100644 --- a/tests/functional/cylc-trigger/02-filter-failed/flow.cylc +++ b/tests/functional/cylc-trigger/02-filter-failed/flow.cylc @@ -18,9 +18,9 @@ [[fixer]] script = """ cylc__job__wait_cylc_message_started - cylc__job__poll_grep_workflow_log -E '1/fixable1 running .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '1/fixable2 running .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '1/fixable3 running .* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable1/01:running.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable2/01:running.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable3/01:running.* \(received\)failed' cylc trigger "${CYLC_WORKFLOW_ID}//1/fixable*" """ [[Z]] diff --git a/tests/functional/cylc-trigger/04-filter-names/flow.cylc b/tests/functional/cylc-trigger/04-filter-names/flow.cylc index 5997dcaa201..31839c1b77f 100644 --- a/tests/functional/cylc-trigger/04-filter-names/flow.cylc +++ b/tests/functional/cylc-trigger/04-filter-names/flow.cylc @@ -22,11 +22,11 @@ [[fixer]] script = """ cylc__job__wait_cylc_message_started - cylc__job__poll_grep_workflow_log -E '1/fixable-1a .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '1/fixable-1b .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '1/fixable-2a .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '1/fixable-2b .* \(received\)failed' - cylc__job__poll_grep_workflow_log -E '1/fixable-3 .* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable-1a/01.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable-1b/01.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable-2a/01.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable-2b/01.* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/fixable-3/01.* \(received\)failed' cylc trigger "${CYLC_WORKFLOW_ID}//" \ '//1/FIXABLE-1' '//1/fixable-2*' '//1/fixable-3' """ diff --git a/tests/functional/cylc-trigger/06-already-active/flow.cylc b/tests/functional/cylc-trigger/06-already-active/flow.cylc index b939aa2d290..c7d99f6a6a8 100644 --- a/tests/functional/cylc-trigger/06-already-active/flow.cylc +++ b/tests/functional/cylc-trigger/06-already-active/flow.cylc @@ -9,14 +9,14 @@ [runtime] [[triggerer]] script = """ - cylc__job__poll_grep_workflow_log "1/triggeree .* running" -E + cylc__job__poll_grep_workflow_log "1/triggeree/01:running" cylc trigger "$CYLC_WORKFLOW_ID//1/triggeree" cylc__job__poll_grep_workflow_log \ - "1/triggeree .* ignoring trigger - already active" -E + "1/triggeree.* ignoring trigger - already active" -E """ [[triggeree]] script = """ cylc__job__poll_grep_workflow_log \ - "1/triggeree .* ignoring trigger - already active" -E + "1/triggeree.* ignoring trigger - already active" -E """ diff --git a/tests/functional/events/23-workflow-stalled-handler/flow.cylc b/tests/functional/events/23-workflow-stalled-handler/flow.cylc index 5981611b409..4d3673daafa 100644 --- a/tests/functional/events/23-workflow-stalled-handler/flow.cylc +++ b/tests/functional/events/23-workflow-stalled-handler/flow.cylc @@ -1,6 +1,6 @@ [scheduler] [[events]] - stall handlers = "cylc set-outputs --flow=1 %(workflow)s//1/bar" + stall handlers = "cylc set %(workflow)s//1/bar" stall timeout = PT0S abort on stall timeout = False expected task failures = 1/bar diff --git a/tests/functional/events/38-task-event-handler-custom.t b/tests/functional/events/38-task-event-handler-custom.t index 92d1ca9722e..d1f825e1c16 100755 --- a/tests/functional/events/38-task-event-handler-custom.t +++ b/tests/functional/events/38-task-event-handler-custom.t @@ -28,7 +28,7 @@ WORKFLOW_LOG="${WORKFLOW_RUN_DIR}/log/scheduler/log" grep_ok \ "\[(('event-handler-00', 'custom-1'), 1) out\] !!CUSTOM!! 1/foo fugu Data ready for barring" \ "${FOO_ACTIVITY_LOG}" -grep_ok "1/foo .*Data ready for barring" "${WORKFLOW_LOG}" -E -grep_ok "1/foo .*Data ready for bazzing" "${WORKFLOW_LOG}" -E -grep_ok "1/foo .*Aren't the hydrangeas nice" "${WORKFLOW_LOG}" -E +grep_ok "1/foo.*Data ready for barring" "${WORKFLOW_LOG}" -E +grep_ok "1/foo.*Data ready for bazzing" "${WORKFLOW_LOG}" -E +grep_ok "1/foo.*Aren't the hydrangeas nice" "${WORKFLOW_LOG}" -E purge diff --git a/tests/functional/execution-time-limit/04-polling-intervals.t b/tests/functional/execution-time-limit/04-polling-intervals.t index 4b213c70546..e1df403f155 100644 --- a/tests/functional/execution-time-limit/04-polling-intervals.t +++ b/tests/functional/execution-time-limit/04-polling-intervals.t @@ -47,13 +47,13 @@ poll_grep_workflow_log "INFO - DONE" # NOTE: execution timeout polling is delayed by PT1M to let things settle # PT10M = (3*PT3S + PT9M30S) - PT1M -grep_workflow_log_ok grep-limit10M "\[1/limit10M running job:01 flows:1\] health: execution timeout=None, polling intervals=3\*PT30S,PT9M30S,PT2M,PT7M,..." +grep_workflow_log_ok grep-limit10M "\[1/limit10M/01:running\] health: execution timeout=None, polling intervals=3\*PT30S,PT9M30S,PT2M,PT7M,..." # PT60M = (3*PT3S + PT10M + PT49M30S) - PT1M -grep_workflow_log_ok grep-limit1H "\[1/limit1H running job:01 flows:1\] health: execution timeout=None, polling intervals=3\*PT30S,PT10M,PT49M30S,PT2M,PT7M,..." +grep_workflow_log_ok grep-limit1H "\[1/limit1H/01:running\] health: execution timeout=None, polling intervals=3\*PT30S,PT10M,PT49M30S,PT2M,PT7M,..." # PT70S = (2*PT30S + PT1M10S) - PT1M -grep_workflow_log_ok grep-limit70S "\[1/limit70S running job:01 flows:1\] health: execution timeout=None, polling intervals=2\*PT30S,PT1M10S,PT2M,PT7M,..." +grep_workflow_log_ok grep-limit70S "\[1/limit70S/01:running\] health: execution timeout=None, polling intervals=2\*PT30S,PT1M10S,PT2M,PT7M,..." # PT95M = (3*PT3S + PT10M + PT1H + PT24M30S) - PT1M -grep_workflow_log_ok grep-limit95M "\[1/limit95M running job:01 flows:1\] health: execution timeout=None, polling intervals=3\*PT30S,PT10M,PT1H,PT24M30S,PT2M,PT7M,..." -grep_workflow_log_ok grep-no-limit "\[1/nolimit running job:01 flows:1\] health: execution timeout=None, polling intervals=3\*PT30S,PT10M,PT1H,..." +grep_workflow_log_ok grep-limit95M "\[1/limit95M/01:running\] health: execution timeout=None, polling intervals=3\*PT30S,PT10M,PT1H,PT24M30S,PT2M,PT7M,..." +grep_workflow_log_ok grep-no-limit "\[1/nolimit/01:running\] health: execution timeout=None, polling intervals=3\*PT30S,PT10M,PT1H,..." purge diff --git a/tests/functional/ext-trigger/01-no-nudge/flow.cylc b/tests/functional/ext-trigger/01-no-nudge/flow.cylc index f5569601950..1bf744e8ac6 100644 --- a/tests/functional/ext-trigger/01-no-nudge/flow.cylc +++ b/tests/functional/ext-trigger/01-no-nudge/flow.cylc @@ -31,7 +31,7 @@ [[foo]] script = """ cylc kill "$CYLC_WORKFLOW_ID//1/bar" - cylc__job__poll_grep_workflow_log -E '1/bar .* \(internal\)failed' + cylc__job__poll_grep_workflow_log -E '1/bar.* \(internal\)failed' cylc release "$CYLC_WORKFLOW_ID//1/bar" """ [[bar]] diff --git a/tests/functional/flow-triggers/00-new-future/flow.cylc b/tests/functional/flow-triggers/00-new-future/flow.cylc index c67fd0fdd19..94add5a60d0 100644 --- a/tests/functional/flow-triggers/00-new-future/flow.cylc +++ b/tests/functional/flow-triggers/00-new-future/flow.cylc @@ -34,5 +34,5 @@ [[a]] script = """ cylc trigger --flow=new ${CYLC_WORKFLOW_ID}//1/d - cylc__job__poll_grep_workflow_log -E '1/d.*started' + cylc__job__poll_grep_workflow_log -E '1/d/01.*started' """ diff --git a/tests/functional/flow-triggers/01-all-future.t b/tests/functional/flow-triggers/01-all-future.t index 8ecc1448e74..27a80f9892c 100644 --- a/tests/functional/flow-triggers/01-all-future.t +++ b/tests/functional/flow-triggers/01-all-future.t @@ -41,11 +41,12 @@ install_workflow "${TEST_NAME_BASE}" "${TEST_NAME_BASE}" true reftest_run TEST_NAME="${TEST_NAME_BASE}-order-wait" +# Note flow_wait is updated to False once used. QUERY="SELECT name,flow_nums,flow_wait FROM task_states ORDER BY time_created" run_ok "${TEST_NAME}" sqlite3 "${DB}" "$QUERY" cmp_ok "${TEST_NAME}.stdout" <<\__END__ a|[1]|0 -d|[1]|1 +d|[1]|0 b|[1]|0 c|[1]|0 e|[1]|0 diff --git a/tests/functional/flow-triggers/01-all-future/flow.cylc b/tests/functional/flow-triggers/01-all-future/flow.cylc index a47ce127429..d7a9b0284d9 100644 --- a/tests/functional/flow-triggers/01-all-future/flow.cylc +++ b/tests/functional/flow-triggers/01-all-future/flow.cylc @@ -39,7 +39,5 @@ [[a]] script = """ cylc trigger {{OPT}} ${CYLC_WORKFLOW_ID}//1/d - {% if WAIT is not defined %} cylc__job__poll_grep_workflow_log -E '1/d.*succeeded' - {% endif %} """ diff --git a/tests/functional/flow-triggers/03-new-past/flow.cylc b/tests/functional/flow-triggers/03-new-past/flow.cylc index 9c77123d40c..2bc3505022e 100644 --- a/tests/functional/flow-triggers/03-new-past/flow.cylc +++ b/tests/functional/flow-triggers/03-new-past/flow.cylc @@ -37,6 +37,6 @@ script = """ if (( $CYLC_TASK_SUBMIT_NUMBER == 1 )); then cylc trigger --flow=new ${CYLC_WORKFLOW_ID}//1/a - cylc__job__poll_grep_workflow_log -E '1/a submitted job:02 .*started' + cylc__job__poll_grep_workflow_log -E '1/a/02\(2\):submitted.*started' fi """ diff --git a/tests/functional/flow-triggers/04-all-past/flow.cylc b/tests/functional/flow-triggers/04-all-past/flow.cylc index 85721f2b5e5..11b6ac7e5d9 100644 --- a/tests/functional/flow-triggers/04-all-past/flow.cylc +++ b/tests/functional/flow-triggers/04-all-past/flow.cylc @@ -41,6 +41,6 @@ script = """ if (( $CYLC_TASK_SUBMIT_NUMBER == 1 )); then cylc trigger {{OPT}} ${CYLC_WORKFLOW_ID}//1/a - cylc__job__poll_grep_workflow_log -E '1/a running job:02 .*succeeded' + cylc__job__poll_grep_workflow_log -E '1/a/02:running.*succeeded' fi """ diff --git a/tests/functional/flow-triggers/05-none-past/flow.cylc b/tests/functional/flow-triggers/05-none-past/flow.cylc index 6709ebf0d3c..e6f03dc2b7a 100644 --- a/tests/functional/flow-triggers/05-none-past/flow.cylc +++ b/tests/functional/flow-triggers/05-none-past/flow.cylc @@ -34,6 +34,6 @@ script = """ if (( $CYLC_TASK_SUBMIT_NUMBER == 1 )); then cylc trigger --flow=none ${CYLC_WORKFLOW_ID}//1/a - cylc__job__poll_grep_workflow_log -E '1/a submitted job:02 .*started' + cylc__job__poll_grep_workflow_log -E '1/a/02\(none\):submitted.*started' fi """ diff --git a/tests/functional/flow-triggers/06-new-past-switch/flow.cylc b/tests/functional/flow-triggers/06-new-past-switch/flow.cylc index b33855b74ca..1b648319bfd 100644 --- a/tests/functional/flow-triggers/06-new-past-switch/flow.cylc +++ b/tests/functional/flow-triggers/06-new-past-switch/flow.cylc @@ -55,5 +55,5 @@ [[c]] script = """ cylc trigger --flow=new ${CYLC_WORKFLOW_ID}//1/a - cylc__job__poll_grep_workflow_log -E '1/a submitted job:02 .*started' + cylc__job__poll_grep_workflow_log -E '1/a/02\(2\):submitted.*started' """ diff --git a/tests/functional/flow-triggers/07-all-past-switch/flow.cylc b/tests/functional/flow-triggers/07-all-past-switch/flow.cylc index 87ea0445a2d..4965bc77886 100644 --- a/tests/functional/flow-triggers/07-all-past-switch/flow.cylc +++ b/tests/functional/flow-triggers/07-all-past-switch/flow.cylc @@ -64,6 +64,6 @@ script = """ if (( CYLC_TASK_SUBMIT_NUMBER == 1 )); then cylc trigger {{OPT}} ${CYLC_WORKFLOW_ID}//1/a - cylc__job__poll_grep_workflow_log -E '1/a running job:02 .*succeeded' + cylc__job__poll_grep_workflow_log -E '1/a/02:running.*succeeded' fi """ diff --git a/tests/functional/flow-triggers/08-none-past-switch/flow.cylc b/tests/functional/flow-triggers/08-none-past-switch/flow.cylc index 419bf72d67a..2f33482edb1 100644 --- a/tests/functional/flow-triggers/08-none-past-switch/flow.cylc +++ b/tests/functional/flow-triggers/08-none-past-switch/flow.cylc @@ -53,5 +53,5 @@ [[c]] script = """ cylc trigger --flow=none ${CYLC_WORKFLOW_ID}//1/a - cylc__job__poll_grep_workflow_log -E '1/a submitted job:02 .*started' + cylc__job__poll_grep_workflow_log -E '1/a/02\(none\):submitted.*started' """ diff --git a/tests/functional/flow-triggers/09-retrigger/flow.cylc b/tests/functional/flow-triggers/09-retrigger/flow.cylc index a8bdb524870..a0e1341c06e 100644 --- a/tests/functional/flow-triggers/09-retrigger/flow.cylc +++ b/tests/functional/flow-triggers/09-retrigger/flow.cylc @@ -1,7 +1,5 @@ -# Check if a task gets triggered multiple times with --wait, only the outputs -# from the last run in the flow are spawned when the flow catches up. - -# baz should run twice in flow 1, then y should trigger off of the second baz. +# If a task gets triggered twice with --wait, the outputs from both runs should be +# spawned when the flow catches up. [scheduling] [[graph]] @@ -15,9 +13,9 @@ [[foo]] script = """ cylc trigger --wait ${CYLC_WORKFLOW_ID}//1/baz - cylc__job__poll_grep_workflow_log "1/baz running job:01 .* succeeded" + cylc__job__poll_grep_workflow_log "1/baz/01:running.*succeeded" cylc trigger --wait ${CYLC_WORKFLOW_ID}//1/baz - cylc__job__poll_grep_workflow_log "1/baz running job:02 .* succeeded" + cylc__job__poll_grep_workflow_log "1/baz/02:running.*succeeded" """ [[baz]] script = """ diff --git a/tests/functional/flow-triggers/09-retrigger/reference.log b/tests/functional/flow-triggers/09-retrigger/reference.log index 2323320048b..00afaf9c6fe 100644 --- a/tests/functional/flow-triggers/09-retrigger/reference.log +++ b/tests/functional/flow-triggers/09-retrigger/reference.log @@ -4,4 +4,5 @@ Final point: 1 1/baz -triggered off [] in flow 1 1/baz -triggered off [] in flow 1 1/bar -triggered off ['1/foo'] in flow 1 +1/x -triggered off ['1/baz'] in flow 1 1/y -triggered off ['1/baz'] in flow 1 diff --git a/tests/functional/flow-triggers/10-specific-flow/flow.cylc b/tests/functional/flow-triggers/10-specific-flow/flow.cylc index 3d648ddcac4..46ba6dab4c1 100644 --- a/tests/functional/flow-triggers/10-specific-flow/flow.cylc +++ b/tests/functional/flow-triggers/10-specific-flow/flow.cylc @@ -1,7 +1,7 @@ # Check targetting a specific flow works, with trigger --wait. # At start-up, trigger f with --wait for flow 2 -# Then when flow 1 reaches d, trigger a new flow (2) at b. +# Then when flow 1 reaches d, trigger flow 2 at b. # Flow 1 should overrun the manually triggered f. # Flow 2 should skip over it to g. @@ -17,6 +17,6 @@ [[trigger-happy]] script = """ cylc trigger --flow=2 --wait ${CYLC_WORKFLOW_ID}//1/f - cylc__job__poll_grep_workflow_log "1/d submitted job:01 .*started" - cylc trigger --flow=new ${CYLC_WORKFLOW_ID}//1/b + cylc__job__poll_grep_workflow_log "1/d/01:submitted.*running" + cylc trigger --flow=2 ${CYLC_WORKFLOW_ID}//1/b """ diff --git a/tests/functional/flow-triggers/11-wait-merge/flow.cylc b/tests/functional/flow-triggers/11-wait-merge/flow.cylc index 9e398c7bf74..b956b45ddf0 100644 --- a/tests/functional/flow-triggers/11-wait-merge/flow.cylc +++ b/tests/functional/flow-triggers/11-wait-merge/flow.cylc @@ -16,7 +16,7 @@ [[a]] script = """ if ((CYLC_TASK_SUBMIT_NUMBER == 2)); then - cylc__job__poll_grep_workflow_log "1/d .*(received)started" + cylc__job__poll_grep_workflow_log "\[1/d/01(1,2):submitted] (received)started" fi """ [[b]] @@ -24,11 +24,12 @@ if ((CYLC_TASK_SUBMIT_NUMBER == 1)); then cylc trigger --flow=new ${CYLC_WORKFLOW_ID}//1/a cylc trigger --flow=2 --wait ${CYLC_WORKFLOW_ID}//1/c - cylc__job__poll_grep_workflow_log "1/c .*(received)x" + cylc__job__poll_grep_workflow_log "\[1/c/01(2):running] (received)x" fi """ [[c]] script = """ + cylc__job__wait_cylc_message_started cylc message x if ((CYLC_TASK_SUBMIT_NUMBER == 1)); then cylc__job__poll_grep_workflow_log "merged" diff --git a/tests/functional/flow-triggers/12-all-future-multi/flow.cylc b/tests/functional/flow-triggers/12-all-future-multi/flow.cylc index 782690f24a5..e67aff4f20e 100644 --- a/tests/functional/flow-triggers/12-all-future-multi/flow.cylc +++ b/tests/functional/flow-triggers/12-all-future-multi/flow.cylc @@ -45,9 +45,9 @@ )); then # trigger 3/a in a new flow cylc trigger --flow=new ${CYLC_WORKFLOW_ID}//3/a - cylc__job__poll_grep_workflow_log -E '3/a.*started' + cylc__job__poll_grep_workflow_log -E '3/a.*=> running' # trigger 5/a in all flows cylc trigger ${CYLC_WORKFLOW_ID}//5/a - cylc__job__poll_grep_workflow_log -E '5/a.*started' + cylc__job__poll_grep_workflow_log -E '5/a.*=> running' fi """ diff --git a/tests/functional/flow-triggers/13-noflow-nomerge.t b/tests/functional/flow-triggers/13-noflow-nomerge.t index c8b4528a2f9..d380268e449 100644 --- a/tests/functional/flow-triggers/13-noflow-nomerge.t +++ b/tests/functional/flow-triggers/13-noflow-nomerge.t @@ -27,7 +27,7 @@ run_ok "${TEST_NAME_BASE}-run" cylc play "${WORKFLOW_NAME}" poll_grep_workflow_log "Workflow stalled" run_ok "${TEST_NAME_BASE}-trigger" cylc trigger --flow=none "${WORKFLOW_NAME}//1/a" -poll_grep_workflow_log -E "1/a running job:02 flows:none.*=> succeeded" +poll_grep_workflow_log -E "1/a/02\(none\):running.*=> succeeded" cylc stop --now --now --max-polls=5 --interval=2 "$WORKFLOW_NAME" diff --git a/tests/functional/hold-release/00-workflow/flow.cylc b/tests/functional/hold-release/00-workflow/flow.cylc index 4afdbc92980..a0df872ebab 100644 --- a/tests/functional/hold-release/00-workflow/flow.cylc +++ b/tests/functional/hold-release/00-workflow/flow.cylc @@ -23,7 +23,7 @@ script = """ cylc__job__wait_cylc_message_started cylc hold --after=1900 "${CYLC_WORKFLOW_ID}" - cylc__job__poll_grep_workflow_log -F 'INFO - Command actioned: set_hold_point' + cylc__job__poll_grep_workflow_log 'Command "set_hold_point" actioned' cylc release --all "${CYLC_WORKFLOW_ID}" """ [[foo,bar]] diff --git a/tests/functional/hold-release/02-hold-on-spawn.t b/tests/functional/hold-release/02-hold-on-spawn.t index 9efd2e18ee8..dad99d32297 100755 --- a/tests/functional/hold-release/02-hold-on-spawn.t +++ b/tests/functional/hold-release/02-hold-on-spawn.t @@ -34,7 +34,7 @@ workflow_run_ok "${TEST_NAME_BASE}-run" \ cylc release "${WORKFLOW_NAME}//1/foo" # 1/foo should run and spawn 1/bar as waiting and held -poll_grep_workflow_log -E '1/bar .* spawned' +poll_grep_workflow_log -E '1/bar.* added to active task pool' sqlite3 "${WORKFLOW_RUN_DIR}/log/db" \ 'SELECT cycle, name, status, is_held FROM task_pool' > task-pool.out diff --git a/tests/functional/hold-release/05-release.t b/tests/functional/hold-release/05-release.t index 805fe2395cc..ea4cc3d95a5 100755 --- a/tests/functional/hold-release/05-release.t +++ b/tests/functional/hold-release/05-release.t @@ -34,7 +34,7 @@ init_workflow "${TEST_NAME_BASE}" <<'__FLOW_CONFIG__' script = """ cylc__job__wait_cylc_message_started cylc hold --after=0 ${CYLC_WORKFLOW_ID} - cylc__job__poll_grep_workflow_log 'Command actioned: set_hold_point' + cylc__job__poll_grep_workflow_log 'Command "set_hold_point" actioned' cylc release "${CYLC_WORKFLOW_ID}//1/*FF" # inexact fam cylc release "${CYLC_WORKFLOW_ID}//1/TOAST" # exact fam cylc release "${CYLC_WORKFLOW_ID}//1/cat*" # inexact tasks @@ -65,7 +65,7 @@ init_workflow "${TEST_NAME_BASE}" <<'__FLOW_CONFIG__' inherit = STOP script = """ cylc__job__poll_grep_workflow_log -E \ - '1/dog1 succeeded .* task proxy removed \(finished\)' + '1/dog1/01:succeeded.* completed' cylc stop "${CYLC_WORKFLOW_ID}" """ __FLOW_CONFIG__ diff --git a/tests/functional/hold-release/08-hold.t b/tests/functional/hold-release/08-hold.t index 206abc1efdb..27d6020f7aa 100755 --- a/tests/functional/hold-release/08-hold.t +++ b/tests/functional/hold-release/08-hold.t @@ -33,14 +33,14 @@ init_workflow "${TEST_NAME_BASE}" <<'__FLOW_CONFIG__' [[holdrelease]] script = """ cylc__job__wait_cylc_message_started - cylc__job__poll_grep_workflow_log -E '1/foo .* spawned' - cylc__job__poll_grep_workflow_log -E '1/bar .* spawned' - cylc__job__poll_grep_workflow_log -E '1/cheese .* spawned' - cylc__job__poll_grep_workflow_log -E '1/jam .* spawned' - cylc__job__poll_grep_workflow_log -E '1/cat1 .* spawned' - cylc__job__poll_grep_workflow_log -E '1/cat2 .* spawned' - cylc__job__poll_grep_workflow_log -E '1/dog1 .* spawned' - cylc__job__poll_grep_workflow_log -E '1/dog2 .* spawned' + cylc__job__poll_grep_workflow_log -E '1/foo.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/bar.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/cheese.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/jam.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/cat1.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/cat2.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/dog1.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '1/dog2.* added to active task pool' cylc hold "${CYLC_WORKFLOW_ID}//1/*FF" # inexact fam cylc hold "${CYLC_WORKFLOW_ID}//1/TOAST" # exact fam cylc hold "${CYLC_WORKFLOW_ID}//1/cat*" # inexact tasks diff --git a/tests/functional/hold-release/11-retrying/flow.cylc b/tests/functional/hold-release/11-retrying/flow.cylc index 03cd0f6b039..0e08699af09 100644 --- a/tests/functional/hold-release/11-retrying/flow.cylc +++ b/tests/functional/hold-release/11-retrying/flow.cylc @@ -18,26 +18,26 @@ t-retry-able => t-analyse [[t-hold-release]] script = """ cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able running job:01.* \(received\)failed' + '1/t-retry-able/01:running.* \(received\)failed' cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able running job:01.* => waiting' + '1/t-retry-able/01:running.* => waiting' cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able waiting job:01.* retrying in PT15S' + '1/t-retry-able:waiting.* retrying in PT15S' cylc hold "${CYLC_WORKFLOW_ID}//1/t-retry-able" cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able waiting job:01.* => waiting\(held\)' + '1/t-retry-able:waiting.* => waiting\(held\)' cylc release "${CYLC_WORKFLOW_ID}//1/t-retry-able" cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able waiting\(held\) job:01.* => waiting' + '1/t-retry-able:waiting\(held\).* => waiting' cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able waiting job:01.* => waiting\(queued\)' + '1/t-retry-able:waiting.* => waiting\(queued\)' """ [[t-analyse]] script = """ diff --git a/tests/functional/hold-release/18-hold-cycle-globs/flow.cylc b/tests/functional/hold-release/18-hold-cycle-globs/flow.cylc index a74c7a20c3f..aa946b9b418 100644 --- a/tests/functional/hold-release/18-hold-cycle-globs/flow.cylc +++ b/tests/functional/hold-release/18-hold-cycle-globs/flow.cylc @@ -23,9 +23,9 @@ [runtime] [[holder]] script = """ - cylc__job__poll_grep_workflow_log -E '19900101T0000Z/t1 .* spawned' - cylc__job__poll_grep_workflow_log -E '20100101T0000Z/t2 .* spawned' - cylc__job__poll_grep_workflow_log -E '20300101T0000Z/t3 .* spawned' + cylc__job__poll_grep_workflow_log -E '19900101T0000Z/t1.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '20100101T0000Z/t2.* added to active task pool' + cylc__job__poll_grep_workflow_log -E '20300101T0000Z/t3.* added to active task pool' cylc hold "${CYLC_WORKFLOW_ID}//*/t*" """ [[releaser]] diff --git a/tests/functional/hold-release/19-no-reset-prereq-on-waiting/flow.cylc b/tests/functional/hold-release/19-no-reset-prereq-on-waiting/flow.cylc index 9ad270b4e84..edbd80b5770 100644 --- a/tests/functional/hold-release/19-no-reset-prereq-on-waiting/flow.cylc +++ b/tests/functional/hold-release/19-no-reset-prereq-on-waiting/flow.cylc @@ -16,7 +16,7 @@ script = true [[holder]] script = """ - cylc__job__poll_grep_workflow_log -E '1/t1 .* spawned' + cylc__job__poll_grep_workflow_log -E '1/t1.* added to active task pool' cylc hold "${CYLC_WORKFLOW_ID}//1/t1" """ [[releaser]] diff --git a/tests/functional/intelligent-host-selection/02-badhosts.t b/tests/functional/intelligent-host-selection/02-badhosts.t index 0689866d22f..2d12c62f4c9 100644 --- a/tests/functional/intelligent-host-selection/02-badhosts.t +++ b/tests/functional/intelligent-host-selection/02-badhosts.t @@ -68,11 +68,11 @@ LOGFILE="${WORKFLOW_RUN_DIR}/log/scheduler/log" # Check that badhosttask has submit failed, but not good or mixed named_grep_ok "badhost task submit failed" \ - "1/badhosttask .* submit-failed" "${LOGFILE}" + "1/badhosttask.* submit-failed" "${LOGFILE}" named_grep_ok "goodhost suceeded" \ - "1/mixedhosttask .* succeeded" "${LOGFILE}" + "1/mixedhosttask.* succeeded" "${LOGFILE}" named_grep_ok "mixedhost task suceeded" \ - "1/goodhosttask .* succeeded" "${LOGFILE}" + "1/goodhosttask.* succeeded" "${LOGFILE}" # Check that when a task fail badhosts associated with that task's platform # are removed from the badhosts set. diff --git a/tests/functional/intelligent-host-selection/05-from-platform-group.t b/tests/functional/intelligent-host-selection/05-from-platform-group.t index 527cfaeba43..801372922fb 100644 --- a/tests/functional/intelligent-host-selection/05-from-platform-group.t +++ b/tests/functional/intelligent-host-selection/05-from-platform-group.t @@ -83,7 +83,7 @@ log_scan \ "platform: ${CYLC_TEST_PLATFORM} - Could not connect to unreachable_host." \ "platform: ${CYLC_TEST_PLATFORM} - remote init (on ${CYLC_TEST_HOST})" \ "platform: ${CYLC_TEST_PLATFORM} - remote file install (on ${CYLC_TEST_HOST})" \ - "\[1/ugly preparing job:01 flows:1\] => submitted" + "\[1/ugly/01:preparing\] => submitted" purge exit 0 diff --git a/tests/functional/lib/bash/test_header b/tests/functional/lib/bash/test_header index c4a07603126..915821c0889 100644 --- a/tests/functional/lib/bash/test_header +++ b/tests/functional/lib/bash/test_header @@ -90,6 +90,7 @@ # tries grepping for each PATTERN in turn. Tests will only pass if the # PATTERNs appear in FILE in the correct order. Runs one test per # pattern, each prefixed by TEST_NAME. +# set LOG_SCAN_GREP_OPTS in the environment, e.g. "-E" for "grep -E" # make_rnd_workflow # Create a randomly-named workflow source directory. # mock_smtpd_init @@ -579,6 +580,7 @@ log_scan () { local FILE="$2" local REPS=$3 local DELAY=$4 + local OPTS=${LOG_SCAN_GREP_OPTS:-} if ${CYLC_TEST_DEBUG:-false}; then local ERR=2 else @@ -595,7 +597,8 @@ log_scan () { echo -n "scanning for '${pattern:0:30}'" >& $ERR for _ in $(seq 1 "${REPS}"); do echo -n '.' >& $ERR - newposition=$(grep -n "$pattern" "$FILE" | \ + # shellcheck disable=SC2086 + newposition=$(grep -n $OPTS "$pattern" "$FILE" | \ tail -n 1 | cut -d ':' -f 1) if (( newposition > position )); then position=$newposition diff --git a/tests/functional/logging/02-duplicates/flow.cylc b/tests/functional/logging/02-duplicates/flow.cylc index 3d1c1dea3ea..30cdbbe6457 100644 --- a/tests/functional/logging/02-duplicates/flow.cylc +++ b/tests/functional/logging/02-duplicates/flow.cylc @@ -22,7 +22,8 @@ script = false [[bar]] script = """ -cylc set-outputs --flow=1 "${CYLC_WORKFLOW_ID}" "foo.${CYLC_TASK_CYCLE_POINT}" + cylc set --output=succeeded \ + "${CYLC_WORKFLOW_ID}//${CYLC_TASK_CYCLE_POINT}/foo" """ [[restart]] script = """ diff --git a/tests/functional/pause-resume/00-workflow/flow.cylc b/tests/functional/pause-resume/00-workflow/flow.cylc index 57a0a24aed5..a668e76ceba 100644 --- a/tests/functional/pause-resume/00-workflow/flow.cylc +++ b/tests/functional/pause-resume/00-workflow/flow.cylc @@ -19,7 +19,7 @@ script = """ wait cylc pause "${CYLC_WORKFLOW_ID}" - cylc__job__poll_grep_workflow_log -F 'INFO - Command actioned: pause()' + cylc__job__poll_grep_workflow_log 'Command "pause" actioned' cylc play "${CYLC_WORKFLOW_ID}" """ [[foo,bar]] diff --git a/tests/functional/pause-resume/12-pause-then-retry/flow.cylc b/tests/functional/pause-resume/12-pause-then-retry/flow.cylc index c732dc3bdc9..7be27343939 100644 --- a/tests/functional/pause-resume/12-pause-then-retry/flow.cylc +++ b/tests/functional/pause-resume/12-pause-then-retry/flow.cylc @@ -19,7 +19,7 @@ [[t-pause]] script = """ cylc pause "${CYLC_WORKFLOW_ID}" - cylc__job__poll_grep_workflow_log -F 'Command actioned: pause' + cylc__job__poll_grep_workflow_log 'Command "pause" actioned' # Poll t-submit-retry-able, should return submit-fail cylc poll "${CYLC_WORKFLOW_ID}//*/t-submit-retry-able" @@ -27,19 +27,19 @@ rm -f "${CYLC_WORKFLOW_RUN_DIR}/file" cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able running .* => waiting' + '1/t-retry-able/01:running.* => waiting' cylc__job__poll_grep_workflow_log -E \ - '1/t-submit-retry-able submitted .* => waiting' + '1/t-submit-retry-able/01:submitted.* => waiting' # Resume the workflow cylc play "${CYLC_WORKFLOW_ID}" cylc__job__poll_grep_workflow_log -E \ - '1/t-retry-able waiting .* => waiting\(queued\)' + '1/t-retry-able:waiting.* => waiting\(queued\)' cylc__job__poll_grep_workflow_log -E \ - '1/t-submit-retry-able waiting .* => waiting\(queued\)' + '1/t-submit-retry-able:waiting.* => waiting\(queued\)' """ [[t-retry-able]] script = """ diff --git a/tests/functional/reload/11-retrying/flow.cylc b/tests/functional/reload/11-retrying/flow.cylc index d5b278b2798..ed4694a5294 100644 --- a/tests/functional/reload/11-retrying/flow.cylc +++ b/tests/functional/reload/11-retrying/flow.cylc @@ -22,7 +22,7 @@ execution retry delays = PT0S [[reloader]] script = """ - cylc__job__poll_grep_workflow_log -E '1/retrier running\(held\) .* => waiting\(held\)' + cylc__job__poll_grep_workflow_log -E '1/retrier/01:running\(held\).* => waiting\(held\)' cylc reload "${CYLC_WORKFLOW_ID}" cylc reload "${CYLC_WORKFLOW_ID}" cylc__job__poll_grep_workflow_log -F 'Reload completed' diff --git a/tests/functional/reload/14-waiting/flow.cylc b/tests/functional/reload/14-waiting/flow.cylc index f81ac3533b0..01f383862e6 100644 --- a/tests/functional/reload/14-waiting/flow.cylc +++ b/tests/functional/reload/14-waiting/flow.cylc @@ -3,7 +3,7 @@ [scheduling] [[graph]] -# SoD: starter:start addeded to create a partially satisfied waiting task +# SoD: starter:start added to create a partially satisfied waiting task # to be present during the reload. R1 = """ starter:start => waiter @@ -23,8 +23,9 @@ done script = true [[reloader]] script = """ +cylc__job__wait_cylc_message_started cylc reload "${CYLC_WORKFLOW_ID}" -cylc__job__poll_grep_workflow_log -E '1/waiter .* reloaded task definition' +cylc__job__poll_grep_workflow_log -E '1/waiter.* reloaded task definition' rm -f "${CYLC_WORKFLOW_WORK_DIR}/1/sleeping-waiter/file" rm -f "${CYLC_WORKFLOW_WORK_DIR}/1/starter/file" """ diff --git a/tests/functional/reload/17-graphing-change.t b/tests/functional/reload/17-graphing-change.t index 26b7b247464..9df561384ff 100755 --- a/tests/functional/reload/17-graphing-change.t +++ b/tests/functional/reload/17-graphing-change.t @@ -66,8 +66,8 @@ cp "${TEST_SOURCE_DIR}/graphing-change/flow-2.cylc" \ "${RUN_DIR}/${WORKFLOW_NAME}/flow.cylc" # Spawn a couple of task proxies, to get "task definition removed" message. -cylc set-outputs --flow=1 "${WORKFLOW_NAME}//1/foo" -cylc set-outputs --flow=1 "${WORKFLOW_NAME}//1/baz" +cylc set "${WORKFLOW_NAME}//1/foo" +cylc set "${WORKFLOW_NAME}//1/baz" # reload workflow run_ok "${TEST_NAME_BASE}-swap-reload" cylc reload "${WORKFLOW_NAME}" poll grep_workflow_log_n_times 'Reload completed' 3 diff --git a/tests/functional/reload/19-remote-kill/flow.cylc b/tests/functional/reload/19-remote-kill/flow.cylc index 2bd548b15f7..4a9e965a73c 100644 --- a/tests/functional/reload/19-remote-kill/flow.cylc +++ b/tests/functional/reload/19-remote-kill/flow.cylc @@ -17,7 +17,7 @@ cylc reload "${CYLC_WORKFLOW_ID}" cylc__job__poll_grep_workflow_log -F 'Reload completed' cylc kill "${CYLC_WORKFLOW_ID}//1/foo" - cylc__job__poll_grep_workflow_log -E '1/foo failed\(held\) job:01.* job killed' + cylc__job__poll_grep_workflow_log -E '1/foo/01:failed\(held\).* job killed' """ [[[job]]] execution time limit = PT1M diff --git a/tests/functional/reload/23-cycle-point-time-zone.t b/tests/functional/reload/23-cycle-point-time-zone.t index 42edfaf1402..d9bf2166560 100644 --- a/tests/functional/reload/23-cycle-point-time-zone.t +++ b/tests/functional/reload/23-cycle-point-time-zone.t @@ -39,13 +39,13 @@ run_ok "${TEST_NAME_BASE}-validate" cylc validate "${WORKFLOW_NAME}" export TZ=BST-1 workflow_run_ok "${TEST_NAME_BASE}-run" cylc play "${WORKFLOW_NAME}" --pause -poll_workflow_running +poll_grep_workflow_log "Paused on start up" # Simulate DST change export TZ=UTC run_ok "${TEST_NAME_BASE}-reload" cylc reload "${WORKFLOW_NAME}" -poll_workflow_running +poll_grep_workflow_log "Reload completed" cylc stop --now --now "${WORKFLOW_NAME}" diff --git a/tests/functional/reload/25-xtriggers.t b/tests/functional/reload/25-xtriggers.t index 8fd1505fe6d..0269a2e3775 100644 --- a/tests/functional/reload/25-xtriggers.t +++ b/tests/functional/reload/25-xtriggers.t @@ -42,8 +42,7 @@ init_workflow "${TEST_NAME_BASE}" <<'__FLOW_CONFIG__' [[reload]] script = """ # wait for "broken" to fail - cylc__job__poll_grep_workflow_log \ - '1/broken .* (received)failed/ERR' + cylc__job__poll_grep_workflow_log -E '1/broken/01.* \(received\)failed/ERR' # fix "broken" to allow it to pass sed -i 's/false/true/' "${CYLC_WORKFLOW_RUN_DIR}/flow.cylc" # reload the workflow @@ -60,12 +59,20 @@ workflow_run_ok "${TEST_NAME_BASE}-run" cylc play "${WORKFLOW_NAME}" --no-detach # 3. the retry xtrigger for "1/broken" becomes satisfied (after the reload) # (thus proving that the xtrigger survived the reload) # 4. "1/broken" succeeds + +log_scan "${TEST_NAME_BASE}-scan" \ + "$(cylc cat-log -m p "${WORKFLOW_NAME}")" \ + 1 1 \ + '1/broken.* (received)failed/ERR' + +log_scan "${TEST_NAME_BASE}-scan" \ + "$(cylc cat-log -m p "${WORKFLOW_NAME}")" 1 1 \ + 'Command "reload_workflow" actioned' \ + log_scan "${TEST_NAME_BASE}-scan" \ "$(cylc cat-log -m p "${WORKFLOW_NAME}")" \ 1 1 \ - '1/broken .* (received)failed/ERR' \ - 'Command actioned: reload_workflow()' \ 'xtrigger satisfied: _cylc_retry_1/broken' \ - '\[1/broken .* => succeeded' + '1/broken.* => succeeded' purge diff --git a/tests/functional/reload/runahead/flow.cylc b/tests/functional/reload/runahead/flow.cylc index 60d11e6477b..c65b5e11d6d 100644 --- a/tests/functional/reload/runahead/flow.cylc +++ b/tests/functional/reload/runahead/flow.cylc @@ -20,7 +20,7 @@ script = true [[reloader]] script = """ - cylc__job__poll_grep_workflow_log -E "${CYLC_TASK_CYCLE_POINT}/foo running .*\(received\)failed" + cylc__job__poll_grep_workflow_log -E "${CYLC_TASK_CYCLE_POINT}/foo/01:running.*\(received\)failed" perl -pi -e 's/(runahead limit = )P1( # marker)/\1 P3\2/' $CYLC_WORKFLOW_RUN_DIR/flow.cylc cylc reload $CYLC_WORKFLOW_ID """ diff --git a/tests/functional/remote/06-poll.t b/tests/functional/remote/06-poll.t index 4eaaa505251..905516e8499 100644 --- a/tests/functional/remote/06-poll.t +++ b/tests/functional/remote/06-poll.t @@ -52,8 +52,8 @@ log_scan \ "$(cylc cat-log -m p "$WORKFLOW_NAME")" \ 10 \ 1 \ - '\[1/foo submitted .* (polled)foo' \ - '\[1/foo .* (polled)succeeded' + '\[1/foo.* (polled)foo' \ + '\[1/foo.* (polled)succeeded' purge exit diff --git a/tests/functional/remote/09-restart-running-file-install.t b/tests/functional/remote/09-restart-running-file-install.t index deb9cf72b4d..c3249b2f5db 100644 --- a/tests/functional/remote/09-restart-running-file-install.t +++ b/tests/functional/remote/09-restart-running-file-install.t @@ -68,7 +68,7 @@ workflow_run_ok "${TEST_NAME_BASE}-restart" \ cylc play --debug --no-detach "${WORKFLOW_NAME}" LOG="${WORKFLOW_RUN_DIR}/log/scheduler/log" grep_ok "remote file install complete" "${LOG}" -grep_ok "\[1/starter running job:01 flows:1\] (received)succeeded" "${LOG}" +grep_ok "\[1/starter/01:running\] (received)succeeded" "${LOG}" ls "${WORKFLOW_RUN_DIR}/log/remote-install" > 'ls.out' cmp_ok ls.out <<__RLOGS__ 01-start-${CYLC_TEST_INSTALL_TARGET}.log diff --git a/tests/functional/restart/22-hold/flow.cylc b/tests/functional/restart/22-hold/flow.cylc index 4f2b44bdde8..213c8a00acd 100644 --- a/tests/functional/restart/22-hold/flow.cylc +++ b/tests/functional/restart/22-hold/flow.cylc @@ -17,7 +17,7 @@ [[t1]] script = """ if [[ "${CYLC_TASK_CYCLE_POINT}" == '2016' ]]; then - cylc__job__poll_grep_workflow_log -E '2016/t2 .* spawned' + cylc__job__poll_grep_workflow_log -E '2016/t2.* added to active task pool' cylc hold "${CYLC_WORKFLOW_ID}//" //2016/t2 //2017/t2 cylc stop "${CYLC_WORKFLOW_ID}" else diff --git a/tests/functional/restart/50-two-flows/flow.cylc b/tests/functional/restart/50-two-flows/flow.cylc index 8837e8835f4..1181e4e80cb 100644 --- a/tests/functional/restart/50-two-flows/flow.cylc +++ b/tests/functional/restart/50-two-flows/flow.cylc @@ -14,7 +14,7 @@ [[a]] script = """ if ((CYLC_TASK_FLOW_NUMBERS == 2)); then - cylc__job__poll_grep_workflow_log "\[1/c .* succeeded" + cylc__job__poll_grep_workflow_log -E "/c.* succeeded" fi """ [[b, d]] @@ -23,7 +23,7 @@ if ((CYLC_TASK_FLOW_NUMBERS == 1)); then cylc trigger --flow=new --meta="cheese wizard" \ "$CYLC_WORKFLOW_ID//1/a" - cylc__job__poll_grep_workflow_log "\[1/a submitted job:02 flows:2\] => running" + cylc__job__poll_grep_workflow_log -E "\[1/a/02\(2\):submitted\] => running" cylc stop $CYLC_WORKFLOW_ID fi """ diff --git a/tests/functional/restart/58-removed-task.t b/tests/functional/restart/58-removed-task.t index 17dc19f626e..1c3b79efe05 100755 --- a/tests/functional/restart/58-removed-task.t +++ b/tests/functional/restart/58-removed-task.t @@ -39,10 +39,10 @@ workflow_run_ok "${TEST_NAME}" cylc play --no-detach "${WORKFLOW_NAME}" TEST_NAME="${TEST_NAME_BASE}-restart" workflow_run_ok "${TEST_NAME}" cylc play --set="INCL_B_C=False" --no-detach "${WORKFLOW_NAME}" -grep_workflow_log_ok "grep-3" "\[1/a running job:01 flows:1\] (polled)started" -grep_workflow_log_ok "grep-4" "\[1/b failed job:01 flows:1\] (polled)failed" +grep_workflow_log_ok "grep-3" "\[1/a/01:running\] (polled)started" +grep_workflow_log_ok "grep-4" "\[1/b/01:failed\] (polled)failed" # Failed (but not incomplete) task c should not have been polled. -grep_fail "\[1/c failed job:01 flows:1\] (polled)failed" "${WORKFLOW_RUN_DIR}/log/scheduler/log" +grep_fail "\[1/c/01:failed\] (polled)failed" "${WORKFLOW_RUN_DIR}/log/scheduler/log" purge diff --git a/tests/functional/restart/58-removed-task/flow.cylc b/tests/functional/restart/58-removed-task/flow.cylc index 94c5cf27b24..0584d4b54fc 100644 --- a/tests/functional/restart/58-removed-task/flow.cylc +++ b/tests/functional/restart/58-removed-task/flow.cylc @@ -22,11 +22,11 @@ [runtime] [[a]] script = """ - cylc__job__poll_grep_workflow_log "1/b .*failed" - cylc__job__poll_grep_workflow_log "1/c .*failed" + cylc__job__poll_grep_workflow_log "1/b.*failed" + cylc__job__poll_grep_workflow_log "1/c.*failed" cylc stop --now $CYLC_WORKFLOW_ID - cylc__job__poll_grep_workflow_log "1/a .*(polled)started" - cylc__job__poll_grep_workflow_log "1/b .*(polled)failed" + cylc__job__poll_grep_workflow_log "1/a.*(polled)started" + cylc__job__poll_grep_workflow_log "1/b.*(polled)failed" """ [[b, c]] script = "false" diff --git a/tests/functional/restart/58-waiting-manual-triggered.t b/tests/functional/restart/58-waiting-manual-triggered.t index efba9f42b70..455cb289592 100644 --- a/tests/functional/restart/58-waiting-manual-triggered.t +++ b/tests/functional/restart/58-waiting-manual-triggered.t @@ -41,7 +41,7 @@ __EOF__ # It should restart and shut down normally, not stall with 2/foo waiting on 1/foo. workflow_run_ok "${TEST_NAME_BASE}-restart" cylc play --no-detach "${WORKFLOW_NAME}" # Check that 2/foo job 02 did run before shutdown. -grep_workflow_log_ok "${TEST_NAME_BASE}-grep" "\[2\/foo running job:02 flows:1\] => succeeded" +grep_workflow_log_ok "${TEST_NAME_BASE}-grep" "\[2\/foo\/02:running\] => succeeded" purge exit diff --git a/tests/functional/runahead/06-release-update.t b/tests/functional/runahead/06-release-update.t index c4ab28530e3..45fb680c69f 100644 --- a/tests/functional/runahead/06-release-update.t +++ b/tests/functional/runahead/06-release-update.t @@ -27,14 +27,23 @@ CYLC_RUN_PID="$!" poll_workflow_running YYYY="$(date +%Y)" NEXT1=$(( YYYY + 1 )) -poll_grep_workflow_log -E "${NEXT1}/bar .* spawned" +poll_grep_workflow_log -E "${NEXT1}/bar.* added to active task pool" # sleep a little to allow the datastore to update (`cylc dump` sees the # datastore) TODO can we avoid this flaky sleep somehow? sleep 10 + # (gratuitous use of --flows for test coverage) cylc dump --flows -t "${WORKFLOW_NAME}" | awk '{print $1 $2 $3 $7}' >'log' + +# The scheduler task pool should contain: +# NEXT1/foo - waiting on clock trigger +# NEXT1/bar - waiting, partially satisfied +# The n=1 data store should also contain: +# YYYY/bar - succeeded + cmp_ok 'log' - <<__END__ +bar,$NEXT1,waiting,[1] foo,$NEXT1,waiting,[1] __END__ diff --git a/tests/functional/runahead/default-future/flow.cylc b/tests/functional/runahead/default-future/flow.cylc index ee083a1dc91..c78522d7a8c 100644 --- a/tests/functional/runahead/default-future/flow.cylc +++ b/tests/functional/runahead/default-future/flow.cylc @@ -27,7 +27,7 @@ [[spawner]] script = """ # spawn wibble - cylc set-outputs --flow=1 $CYLC_WORKFLOW_ID 20100101T0800Z/foo + cylc set $CYLC_WORKFLOW_ID 20100101T0800Z/foo """ [[foo]] script = false diff --git a/tests/functional/spawn-on-demand/05-stop-flow/flow.cylc b/tests/functional/spawn-on-demand/05-stop-flow/flow.cylc index cac2ddf8009..2c2d1996009 100644 --- a/tests/functional/spawn-on-demand/05-stop-flow/flow.cylc +++ b/tests/functional/spawn-on-demand/05-stop-flow/flow.cylc @@ -10,5 +10,5 @@ [[bar]] script = """ cylc stop --flow=1 ${CYLC_WORKFLOW_ID} - cylc__job__poll_grep_workflow_log 'Command actioned: stop' + cylc__job__poll_grep_workflow_log 'Command "stop" actioned' """ diff --git a/tests/functional/spawn-on-demand/06-stop-flow-2/flow.cylc b/tests/functional/spawn-on-demand/06-stop-flow-2/flow.cylc index 9c7da97e974..8aefe4c5a44 100644 --- a/tests/functional/spawn-on-demand/06-stop-flow-2/flow.cylc +++ b/tests/functional/spawn-on-demand/06-stop-flow-2/flow.cylc @@ -14,13 +14,13 @@ script = """ if (( CYLC_TASK_SUBMIT_NUMBER == 2 )); then cylc stop --flow=1 ${CYLC_WORKFLOW_ID} - cylc__job__poll_grep_workflow_log "Command actioned: stop" + cylc__job__poll_grep_workflow_log 'Command "stop" actioned' fi """ [[baz]] script = """ if (( CYLC_TASK_SUBMIT_NUMBER == 1 )); then cylc trigger --flow=new --meta=other "${CYLC_WORKFLOW_ID}//1/foo" - cylc__job__poll_grep_workflow_log -E "1/bar running job:02.* => succeeded" + cylc__job__poll_grep_workflow_log -E "1/bar/02\(2\):running.* => succeeded" fi """ diff --git a/tests/functional/spawn-on-demand/07-abs-triggers/flow.cylc b/tests/functional/spawn-on-demand/07-abs-triggers/flow.cylc index 8084e5c0abe..3f621ccc1e4 100644 --- a/tests/functional/spawn-on-demand/07-abs-triggers/flow.cylc +++ b/tests/functional/spawn-on-demand/07-abs-triggers/flow.cylc @@ -16,7 +16,7 @@ script = """ # Ensure that 1,2/bar are spawned by 1,2/foo and not by 2/start # (so the scheduler must update their prereqs when 2/start finishes). - cylc__job__poll_grep_workflow_log -E "2/bar .* spawned" + cylc__job__poll_grep_workflow_log -E "2/bar.* added to active task pool" """ [[foo]] [[bar]] diff --git a/tests/functional/spawn-on-demand/09-set-outputs.t b/tests/functional/spawn-on-demand/09-set-outputs.t index 41c6b5b6b87..57d34e15024 100644 --- a/tests/functional/spawn-on-demand/09-set-outputs.t +++ b/tests/functional/spawn-on-demand/09-set-outputs.t @@ -16,7 +16,7 @@ # along with this program. If not, see . #------------------------------------------------------------------------------- -# Check that "cylc set-outputs" works like it says on the tin. +# Check that "cylc set" works like it says on the tin. . "$(dirname "$0")/test_header" set_test_number 2 reftest diff --git a/tests/functional/spawn-on-demand/09-set-outputs/flow.cylc b/tests/functional/spawn-on-demand/09-set-outputs/flow.cylc index 1d1d7e7e061..85138715d1b 100644 --- a/tests/functional/spawn-on-demand/09-set-outputs/flow.cylc +++ b/tests/functional/spawn-on-demand/09-set-outputs/flow.cylc @@ -1,8 +1,6 @@ -# Test that `cylc set-outputs` has the same effect as natural output +# Test that `cylc set` has the same effect as natural output # completion: i.e. that downstream children are spawned as normal. -# DEBUG mode required: we search for "task proxy removed" in the log. - [scheduler] [[events]] abort on stall timeout = True @@ -35,7 +33,7 @@ [[foo]] # Hang about until setter is finished. script = """ - cylc__job__poll_grep_workflow_log -E "1/setter .* => succeeded" + cylc__job__poll_grep_workflow_log -E "1/setter.* => succeeded" """ [[bar]] script = true @@ -43,11 +41,11 @@ # (To the rescue). script = """ # Set foo outputs while it still exists in the pool. - cylc set-outputs --flow=2 --output=out1 --output=out2 "${CYLC_WORKFLOW_ID}//1/foo" + cylc set --flow=2 --output=out1 --output=out2 "${CYLC_WORKFLOW_ID}//1/foo" # Set bar outputs after it is gone from the pool. - cylc__job__poll_grep_workflow_log -E "1/bar .*task proxy removed" - cylc set-outputs --flow=2 --output=out1 --output=out2 "${CYLC_WORKFLOW_ID}//1/bar" + cylc__job__poll_grep_workflow_log -E "1/bar.* completed" + cylc set --flow=2 --output=out1 --output=out2 "${CYLC_WORKFLOW_ID}//1/bar" """ [[qux, quw, fux, fuw]] script = true diff --git a/tests/functional/spawn-on-demand/10-retrigger/flow.cylc b/tests/functional/spawn-on-demand/10-retrigger/flow.cylc index 2bdd4365a07..7e9149ce3c9 100644 --- a/tests/functional/spawn-on-demand/10-retrigger/flow.cylc +++ b/tests/functional/spawn-on-demand/10-retrigger/flow.cylc @@ -18,7 +18,7 @@ """ [[triggerer]] script = """ - cylc__job__poll_grep_workflow_log -E '1/oops running .* \(received\)failed' + cylc__job__poll_grep_workflow_log -E '1/oops/01:running.* \(received\)failed' cylc trigger "${CYLC_WORKFLOW_ID}//1/oops" """ [[foo, bar]] diff --git a/tests/functional/spawn-on-demand/11-hold-not-spawned/flow.cylc b/tests/functional/spawn-on-demand/11-hold-not-spawned/flow.cylc index 890d73f78be..c47ca3c93c4 100644 --- a/tests/functional/spawn-on-demand/11-hold-not-spawned/flow.cylc +++ b/tests/functional/spawn-on-demand/11-hold-not-spawned/flow.cylc @@ -15,6 +15,6 @@ script = true [[stopper]] script = """ - cylc__job__poll_grep_workflow_log "\[1/holdee .* holding \(as requested earlier\)" -E + cylc__job__poll_grep_workflow_log "\[1/holdee.* holding \(as requested earlier\)" -E cylc stop $CYLC_WORKFLOW_ID """ diff --git a/tests/functional/spawn-on-demand/12-set-outputs-no-reflow.t b/tests/functional/spawn-on-demand/12-set-outputs-cont-flow.t similarity index 93% rename from tests/functional/spawn-on-demand/12-set-outputs-no-reflow.t rename to tests/functional/spawn-on-demand/12-set-outputs-cont-flow.t index 41c6b5b6b87..36c8bd100b3 100644 --- a/tests/functional/spawn-on-demand/12-set-outputs-no-reflow.t +++ b/tests/functional/spawn-on-demand/12-set-outputs-cont-flow.t @@ -16,7 +16,7 @@ # along with this program. If not, see . #------------------------------------------------------------------------------- -# Check that "cylc set-outputs" works like it says on the tin. +# Check that "cylc set" continues a flow by default. . "$(dirname "$0")/test_header" set_test_number 2 reftest diff --git a/tests/functional/spawn-on-demand/12-set-outputs-no-reflow/flow.cylc b/tests/functional/spawn-on-demand/12-set-outputs-cont-flow/flow.cylc similarity index 70% rename from tests/functional/spawn-on-demand/12-set-outputs-no-reflow/flow.cylc rename to tests/functional/spawn-on-demand/12-set-outputs-cont-flow/flow.cylc index 315007c52fa..353d6d1f41f 100644 --- a/tests/functional/spawn-on-demand/12-set-outputs-no-reflow/flow.cylc +++ b/tests/functional/spawn-on-demand/12-set-outputs-cont-flow/flow.cylc @@ -1,5 +1,5 @@ -# Test that `cylc set-outputs` does not cause reflow by default -# Task setter should cause bar to run, but not subsequently baz. +# Test that `cylc set` continues the active flow by default +# Task "setter" should cause bar to run, then subsequently baz. [scheduler] [[events]] @@ -21,5 +21,5 @@ script = true [[setter]] script = """ - cylc set-outputs "${CYLC_WORKFLOW_ID}//1/foo" + cylc set --output=succeeded "${CYLC_WORKFLOW_ID}//1/foo" """ diff --git a/tests/functional/spawn-on-demand/12-set-outputs-no-reflow/reference.log b/tests/functional/spawn-on-demand/12-set-outputs-cont-flow/reference.log similarity index 79% rename from tests/functional/spawn-on-demand/12-set-outputs-no-reflow/reference.log rename to tests/functional/spawn-on-demand/12-set-outputs-cont-flow/reference.log index 2322cc234da..3c7b498cc8b 100644 --- a/tests/functional/spawn-on-demand/12-set-outputs-no-reflow/reference.log +++ b/tests/functional/spawn-on-demand/12-set-outputs-cont-flow/reference.log @@ -3,3 +3,4 @@ Final point: 1 1/foo -triggered off [] 1/setter -triggered off ['1/foo'] 1/bar -triggered off ['1/foo'] +1/baz -triggered off ['1/bar'] diff --git a/tests/functional/spawn-on-demand/14-trigger-flow-blocker/flow.cylc b/tests/functional/spawn-on-demand/14-trigger-flow-blocker/flow.cylc index 0849b69bb75..9c205e301c5 100644 --- a/tests/functional/spawn-on-demand/14-trigger-flow-blocker/flow.cylc +++ b/tests/functional/spawn-on-demand/14-trigger-flow-blocker/flow.cylc @@ -23,7 +23,7 @@ cylc trigger --flow=none $CYLC_WORKFLOW_ID//3/foo elif ((CYLC_TASK_CYCLE_POINT == 3)); then # Run until I get merged. - cylc__job__poll_grep_workflow_log -E "3/foo .* merged in flow\(s\) 1" + cylc__job__poll_grep_workflow_log -E "3/foo.* merged in flow\(s\) 1" fi """ [[bar]] diff --git a/tests/functional/spawn-on-demand/18-submitted.t b/tests/functional/spawn-on-demand/18-submitted.t index de5041f4ca1..30f022ebafd 100644 --- a/tests/functional/spawn-on-demand/18-submitted.t +++ b/tests/functional/spawn-on-demand/18-submitted.t @@ -40,7 +40,7 @@ reftest_run for number in 1 2 3; do grep_workflow_log_ok \ "${TEST_NAME_BASE}-a${number}" \ - "${number}/a${number} .* did not complete required outputs: \['submitted'\]" + "${number}/a${number}.* did not complete required outputs: \['submitted'\]" done purge diff --git a/tests/functional/spawn-on-demand/19-submitted-compat.t b/tests/functional/spawn-on-demand/19-submitted-compat.t index d529dfb4183..98c603d55a7 100644 --- a/tests/functional/spawn-on-demand/19-submitted-compat.t +++ b/tests/functional/spawn-on-demand/19-submitted-compat.t @@ -51,7 +51,7 @@ grep_workflow_log_ok \ 'Backward compatibility mode ON' grep_workflow_log_ok \ "${TEST_NAME_BASE}-a-complete" \ - '\[1/a running job:01 flows:1\] => succeeded' + '\[1/a/01:running\] => succeeded' grep_workflow_log_ok \ "${TEST_NAME_BASE}-b-incomplete" \ "1/b did not complete required outputs: \['submitted', 'succeeded'\]" diff --git a/tests/functional/special/08-clock-trigger-retry.t b/tests/functional/special/08-clock-trigger-retry.t index d4f591d4870..fa3a65ee150 100644 --- a/tests/functional/special/08-clock-trigger-retry.t +++ b/tests/functional/special/08-clock-trigger-retry.t @@ -42,7 +42,7 @@ workflow_run_ok "${TEST_NAME_BASE}-run" cylc play --no-detach "$WORKFLOW_NAME" log_scan "${TEST_NAME_BASE}-log-scan" \ "${WORKFLOW_RUN_DIR}/log/scheduler/log" 2 1 \ - "\[20150101.*/foo .* job:01 .* retrying in PT5S" \ + "\[20150101.*/foo.* retrying in PT5S" \ "xtrigger satisfied: _cylc_retry_20150101" # (if task resubmits immediately instead of waiting PT5S, xtrigger msg will not appear) diff --git a/tests/functional/startup/00-state-summary.t b/tests/functional/startup/00-state-summary.t index a4a02208899..76d16bbd25d 100644 --- a/tests/functional/startup/00-state-summary.t +++ b/tests/functional/startup/00-state-summary.t @@ -30,7 +30,7 @@ run_ok "${TEST_NAME}" cylc validate "${WORKFLOW_NAME}" cylc play --no-detach "${WORKFLOW_NAME}" > /dev/null 2>&1 # Restart with a failed task and a succeeded task. cylc play "${WORKFLOW_NAME}" -poll_grep_workflow_log -E '1/foo .* \(polled\)failed' +poll_grep_workflow_log -E '1/foo.* \(polled\)failed' cylc dump "${WORKFLOW_NAME}" > dump.out TEST_NAME=${TEST_NAME_BASE}-grep # State summary should not just say "Initializing..." diff --git a/tests/functional/triggering/19-and-suicide/flow.cylc b/tests/functional/triggering/19-and-suicide/flow.cylc index cf7ae49d129..670c361fc96 100644 --- a/tests/functional/triggering/19-and-suicide/flow.cylc +++ b/tests/functional/triggering/19-and-suicide/flow.cylc @@ -16,7 +16,7 @@ [[t0]] # https://github.com/cylc/cylc-flow/issues/2655 # "1/t2" should not suicide on "1/t1:failed" - script = cylc__job__poll_grep_workflow_log -E '1/t1 .* \(received\)failed' + script = cylc__job__poll_grep_workflow_log -E '1/t1.* \(received\)failed' [[t1]] script = false [[t2]] diff --git a/tests/functional/triggering/21-expire.t b/tests/functional/triggering/21-expire.t new file mode 100644 index 00000000000..aaacdf807b0 --- /dev/null +++ b/tests/functional/triggering/21-expire.t @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# 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 . +#------------------------------------------------------------------------------- +# Test expire triggering +. "$(dirname "$0")/test_header" +set_test_number 2 +reftest +exit diff --git a/tests/functional/triggering/21-expire/flow.cylc b/tests/functional/triggering/21-expire/flow.cylc new file mode 100644 index 00000000000..2a5336cb5e8 --- /dev/null +++ b/tests/functional/triggering/21-expire/flow.cylc @@ -0,0 +1,22 @@ +[scheduling] + initial cycle point = 1999 + [[special tasks]] + clock-expire = foo1(PT0S), foo2(PT0S), bar1(PT0S), x(PT0S) + [[graph]] + # Expire: foo1, foo2, bar1, x + # Run: y, bar2, baz, qux + R1 = """ + x? + FOO? + BAR? + x:expire? => y + FOO:expire-all? => baz + BAR:expire-any? => qux + """ +[runtime] + [[FOO, BAR]] + [[foo1, foo2]] + inherit = FOO + [[bar1, bar2]] + inherit = BAR + [[x, y, baz, qux]] diff --git a/tests/functional/triggering/21-expire/reference.log b/tests/functional/triggering/21-expire/reference.log new file mode 100644 index 00000000000..8ba5edca688 --- /dev/null +++ b/tests/functional/triggering/21-expire/reference.log @@ -0,0 +1,4 @@ +19990101T0000Z/bar2 -triggered off [] in flow 1 +19990101T0000Z/baz -triggered off ['19990101T0000Z/foo1', '19990101T0000Z/foo2'] in flow 1 +19990101T0000Z/qux -triggered off ['19990101T0000Z/bar1'] in flow 1 +19990101T0000Z/y -triggered off ['19990101T0000Z/x'] in flow 1 diff --git a/tests/functional/xtriggers/03-sequence.t b/tests/functional/xtriggers/03-sequence.t index f45af9b1caf..31e34c21522 100644 --- a/tests/functional/xtriggers/03-sequence.t +++ b/tests/functional/xtriggers/03-sequence.t @@ -49,7 +49,7 @@ run_ok "${TEST_NAME_BASE}-val" cylc validate "${WORKFLOW_NAME}" # Run workflow; it will stall waiting on the never-satisfied xtriggers. cylc play "${WORKFLOW_NAME}" -poll_grep_workflow_log -E '2025/start .* => succeeded' +poll_grep_workflow_log -E '2025/start.* => succeeded' cylc show "${WORKFLOW_NAME}//2026/foo" | grep -E '^ - xtrigger' > 2026.foo.log diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index ca7ee981bd9..1af3e2af312 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -24,6 +24,7 @@ from typing import List, TYPE_CHECKING, Set, Tuple, Union from cylc.flow.config import WorkflowConfig +from cylc.flow.id import Tokens from cylc.flow.option_parsers import Options from cylc.flow.pathutil import get_cylc_run_dir from cylc.flow.rundb import CylcWorkflowDAO @@ -544,7 +545,9 @@ def complete(): The scheduler to await. tokens_list: If specified, this will wait for the tasks represented by these - tokens to be marked as completed by the task pool. + tokens to be marked as completed by the task pool. Can use + relative task ids as strings (e.g. '1/a') rather than tokens for + convenience. stop_mode: If tokens_list is not provided, this will wait for the scheduler to be shutdown with the specified mode (default = AUTO, i.e. @@ -561,20 +564,26 @@ def complete(): """ async def _complete( schd, - *tokens_list, + *tokens_list: Union[Tokens, str], stop_mode=StopMode.AUTO, - timeout=60, - ): + timeout: int = 60, + ) -> None: start_time = time() - tokens_list = [tokens.task for tokens in tokens_list] + + _tokens_list: List[Tokens] = [] + for tokens in tokens_list: + if isinstance(tokens, str): + tokens = Tokens(tokens, relative=True) + _tokens_list.append(tokens.task) # capture task completion remove_if_complete = schd.pool.remove_if_complete def _remove_if_complete(itask): + nonlocal _tokens_list ret = remove_if_complete(itask) - if ret and itask.tokens.task in tokens_list: - tokens_list.remove(itask.tokens.task) + if ret and itask.tokens.task in _tokens_list: + _tokens_list.remove(itask.tokens.task) return ret schd.pool.remove_if_complete = _remove_if_complete @@ -595,8 +604,8 @@ def _set_stop(mode=None): schd._set_stop = _set_stop # determine the completion condition - if tokens_list: - condition = lambda: bool(tokens_list) + if _tokens_list: + condition = lambda: bool(_tokens_list) else: condition = lambda: bool(not has_shutdown) @@ -604,9 +613,9 @@ def _set_stop(mode=None): while condition(): # allow the main loop to advance await asyncio.sleep(0) - if time() - start_time > timeout: + if (time() - start_time) > timeout: raise Exception( - f'Timeout waiting for {", ".join(map(str, tokens_list))}' + f'Timeout waiting for {", ".join(map(str, _tokens_list))}' ) # restore regular shutdown logic diff --git a/tests/integration/scripts/test_completion_server.py b/tests/integration/scripts/test_completion_server.py new file mode 100644 index 00000000000..db0fb2a57d6 --- /dev/null +++ b/tests/integration/scripts/test_completion_server.py @@ -0,0 +1,206 @@ +# 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 . + +"""Integration tests for the "cylc completion-server command. + +See also the more extensive unit tests for this module. +""" + +from cylc.flow.scripts.completion_server import ( + _list_prereqs_and_outputs, + complete_cylc, +) + + +def setify(coro): + """Cast returned lists to sets for coroutines. + + Convenience function to use when you want to test output not order. + """ + async def _coro(*args, **kwargs): + nonlocal coro + ret = await coro(*args, **kwargs) + if isinstance(ret, list): + return set(ret) + return ret + return _coro + + +async def test_list_prereqs_and_outputs(flow, scheduler, start): + """Test the success cases for listing task prereqs/outputs. + + The error cases are tested in a unit test (doesn't require a running + scheduler). + """ + _complete_cylc = setify(complete_cylc) # Note: results are un-ordered + + id_ = flow({ + 'scheduler': { + 'allow implicit tasks': 'True', + }, + 'scheduling': { + 'initial cycle point': '1', + 'cycling mode': 'integer', + 'graph': { + 'P1': ''' + a => b + c => d + b[-P1] => b + ''' + }, + }, + 'runtime': { + 'a': {}, + 'b': { + 'outputs': { + 'foo': 'abc def ghi', + } + } + } + }) + schd = scheduler(id_) + async with start(schd): + await schd.update_data_structure() + b1 = schd.tokens.duplicate(cycle='1', task='b') + d1 = schd.tokens.duplicate(cycle='1', task='d') + e1 = schd.tokens.duplicate(cycle='1', task='e') # does not exist + + # list prereqs (b1) + assert await _complete_cylc('cylc', 'set', b1.id, '--pre', '') == { + # keywords + 'all', + # intra-cycle dependency + '1/a:succeeded', + # inter-cycle dependency + '0/b:succeeded', + } + + # list outputs (b1) + assert await _complete_cylc('cylc', 'set', b1.id, '--out', '') == { + # regular task outputs + 'failed', + 'started', + 'submit-failed', + 'submitted', + 'succeeded', + # custom task outputs + 'foo', + } + + # list prereqs (d1) + assert await _complete_cylc('cylc', 'set', d1.id, '--pre', '') == { + # keywords + 'all', + # d1 prereqs + '1/c:succeeded', + } + + # list prereqs for multiple (b1, d1) + assert await _complete_cylc( + 'cylc', + 'set', + b1.id, + d1.id, + '--pre', + '', + ) == { + # keywords + 'all', + # b1 prereqs + '1/a:succeeded', + '0/b:succeeded', + # d1 prereqs + '1/c:succeeded', + } + + # list prereqs for multiple (b1, d1) - alternative format + assert await _complete_cylc( + 'cylc', + 'set', + f'{schd.id}//', + f'//{b1.relative_id}', + f'//{d1.relative_id}', + '--pre', + '', + ) == { + # keywords + 'all', + # b1 prereqs + '1/a:succeeded', + '0/b:succeeded', + # d1 prereqs + '1/c:succeeded', + } + + # list outputs for a non-existant task + assert await _complete_cylc('cylc', 'set', e1.id, '--out', '') == set() + + # list outputs for a non-existant workflow + assert await _complete_cylc( + 'cylc', + 'set', + # this invalid workflow shouldn't prevent it from returning values + # for the valid one + 'no-such-workflow//', + f'{schd.id}//', + f'//{b1.relative_id}', + f'//{d1.relative_id}', + '--pre', + '', + ) == { + # keywords + 'all', + # b1 prereqs + '1/a:succeeded', + '0/b:succeeded', + # d1 prereqs + '1/c:succeeded', + } + + # start a second workflow to test multi-workflow functionality + id2 = flow({ + 'scheduling': { + 'graph': { + 'R1': ''' + x => z + ''' + } + }, + 'runtime': {'x': {}, 'z': {}}, + }) + schd2 = scheduler(id2) + async with start(schd2): + await schd2.update_data_structure() + z1 = schd2.tokens.duplicate(cycle='1', task='z') + + # list prereqs for multiple tasks in multiple workflows + # (it should combine the results from both workflows) + assert await _complete_cylc( + 'cylc', + 'set', + b1.id, + z1.id, + '--pre', + '', + ) == { + # keywords + 'all', + # workflow1//1/b prereqs + '0/b:succeeded', + '1/a:succeeded', + # workflow2//1/z prereqs + '1/x:succeeded' + } diff --git a/tests/integration/scripts/test_set.py b/tests/integration/scripts/test_set.py new file mode 100644 index 00000000000..7dbb240533a --- /dev/null +++ b/tests/integration/scripts/test_set.py @@ -0,0 +1,151 @@ +# 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 . + +"""Test "cylc set" functionality. + +Note: see also functional tests +""" + +from cylc.flow.cycling.integer import IntegerPoint +from cylc.flow.data_store_mgr import TASK_PROXIES +from cylc.flow.task_state import TASK_STATUS_WAITING, TASK_STATUS_SUCCEEDED + + +async def test_set_parentless_spawning( + flow, + scheduler, + run, + complete, +): + """Ensure that setting outputs does not interfere with parentless spawning. + + Setting outputs manually causes the logic to follow a different code + pathway to "natural" output satisfaction. If we're not careful this could + lead to "premature shutdown" (i.e. the scheduler thinks it's finished when + it isn't), this test makes sure that's not the case. + """ + id_ = flow({ + 'scheduling': { + 'initial cycle point': '1', + 'cycling mode': 'integer', + 'runahead limit': 'P0', + 'graph': {'P1': 'a => z'}, + }, + }) + schd = scheduler(id_, paused_start=False) + async with run(schd): + # mark cycle 1 as succeeded + schd.pool.set(['1/a', '1/z'], ['succeeded'], None, ['1']) + + # the parentless task "a" should be spawned out to the runahead limit + assert [ + itask.identity for itask in schd.pool.get_tasks() + ] == ['2/a', '3/a'] + + # the workflow should run on to the next cycle + await complete(schd, '2/a', timeout=5) + + +async def test_rerun_incomplete( + flow, + scheduler, + run, + complete, + reflog, +): + """Incomplete tasks should be re-run.""" + id_ = flow({ + 'scheduling': { + 'graph': {'R1': 'a => z'}, + }, + 'runtime': { + # register a custom output + 'a': {'outputs': {'x': 'x'}}, + }, + }) + schd = scheduler(id_, paused_start=False) + async with run(schd): + # generate 1/a:x but do not complete 1/a + schd.pool.set(['1/a'], ['x'], None, ['1']) + triggers = reflog(schd) + await complete(schd) + + assert triggers == { + # the task 1/a should have been run despite the earlier + # setting of the "x" output + ('1/a', None), + ('1/z', ('1/a',)), + } + + +async def test_data_store( + flow, + scheduler, + start, +): + """Test that manually set prereqs/outputs are applied to the data store.""" + id_ = flow({ + 'scheduling': { + 'graph': {'R1': 'a => z'}, + }, + 'runtime': { + # register a custom output + 'a': {'outputs': {'x': 'x'}}, + }, + }) + schd = scheduler(id_) + async with start(schd): + await schd.update_data_structure() + data = schd.data_store_mgr.data[schd.tokens.id] + task_a = data[TASK_PROXIES][ + schd.pool.get_task(IntegerPoint('1'), 'a').tokens.id + ] + + # set the 1/a:succeeded prereq of 1/z + schd.pool.set(['1/z'], None, ['1/a:succeeded'], ['1']) + task_z = data[TASK_PROXIES][ + schd.pool.get_task(IntegerPoint('1'), 'z').tokens.id + ] + await schd.update_data_structure() + assert task_z.prerequisites[0].satisfied is True + + # set 1/a:x the task should be waiting with output x satisfied + schd.pool.set(['1/a'], ['x'], None, ['1']) + await schd.update_data_structure() + assert task_a.state == TASK_STATUS_WAITING + assert task_a.outputs['x'].satisfied is True + assert task_a.outputs['succeeded'].satisfied is False + + # set 1/a:succeeded the task should be succeeded with output x sat + schd.pool.set(['1/a'], ['succeeded'], None, ['1']) + await schd.update_data_structure() + assert task_a.state == TASK_STATUS_SUCCEEDED + assert task_a.outputs['x'].satisfied is True + assert task_a.outputs['succeeded'].satisfied is True + + +async def test_incomplete_detection( + one_conf, + flow, + scheduler, + start, + log_filter, +): + """It should detect and log finished tasks left with incomplete outputs.""" + schd = scheduler(flow(one_conf)) + async with start(schd) as log: + schd.pool.set(['1/one'], ['failed'], None, ['1']) + assert log_filter(log, contains='1/one did not complete') diff --git a/tests/integration/test_config.py b/tests/integration/test_config.py index acf24d17eaf..dd8da5edb24 100644 --- a/tests/integration/test_config.py +++ b/tests/integration/test_config.py @@ -263,6 +263,13 @@ def test_parse_special_tasks_families(flow, scheduler, validate, section): with pytest.raises(WorkflowConfigError) as exc_ctx: config = validate(id_) assert 'external triggers must be used only once' in str(exc_ctx.value) + + elif section == 'clock-expire': + with pytest.raises(WorkflowConfigError) as exc_ctx: + config = validate(id_) + assert ( + 'Clock-expire must be visible in the graph' in str(exc_ctx.value) + ) else: config = validate(id_) assert set(config.cfg['scheduling']['special tasks'][section]) == { diff --git a/tests/integration/test_data_store_mgr.py b/tests/integration/test_data_store_mgr.py index 21b95280443..615050c3dd5 100644 --- a/tests/integration/test_data_store_mgr.py +++ b/tests/integration/test_data_store_mgr.py @@ -171,14 +171,14 @@ async def test_delta_task_state(harness): w_id = schd.data_store_mgr.workflow_id schd.data_store_mgr.data[w_id] = data assert TASK_STATUS_FAILED not in set(collect_states(data, TASK_PROXIES)) - for itask in schd.pool.get_all_tasks(): + for itask in schd.pool.get_tasks(): itask.state.reset(TASK_STATUS_FAILED) schd.data_store_mgr.delta_task_state(itask) assert TASK_STATUS_FAILED in set(collect_states( schd.data_store_mgr.updated, TASK_PROXIES)) # put things back the way we found them - for itask in schd.pool.get_all_tasks(): + for itask in schd.pool.get_tasks(): itask.state.reset(TASK_STATUS_WAITING) schd.data_store_mgr.delta_task_state(itask) await schd.update_data_structure() @@ -191,7 +191,7 @@ async def test_delta_task_held(harness): schd.pool.hold_tasks('*') await schd.update_data_structure() assert True in {t.is_held for t in data[TASK_PROXIES].values()} - for itask in schd.pool.get_all_tasks(): + for itask in schd.pool.get_tasks(): itask.state.reset(is_held=False) schd.data_store_mgr.delta_task_held(itask) assert True not in { @@ -269,7 +269,7 @@ async def test_update_data_structure(harness): assert TASK_STATUS_FAILED not in set(collect_states(data, FAMILY_PROXIES)) assert TASK_STATUS_FAILED not in data[WORKFLOW].state_totals assert len({t.is_held for t in data[TASK_PROXIES].values()}) == 2 - for itask in schd.pool.get_all_tasks(): + for itask in schd.pool.get_tasks(): itask.state.reset(TASK_STATUS_FAILED) schd.data_store_mgr.delta_task_state(itask) schd.data_store_mgr.update_data_structure() @@ -288,15 +288,17 @@ async def test_update_data_structure(harness): def test_delta_task_prerequisite(harness): """Test delta_task_prerequisites.""" schd, data = harness - schd.pool.force_spawn_children([ - t.identity - for t in schd.pool.get_all_tasks() - ], (TASK_STATUS_SUCCEEDED,), "flow1") + schd.pool.set( + [t.identity for t in schd.pool.get_tasks()], + [(TASK_STATUS_SUCCEEDED,)], + [], + "all" + ) assert all({ p.satisfied for t in schd.data_store_mgr.updated[TASK_PROXIES].values() for p in t.prerequisites}) - for itask in schd.pool.get_all_tasks(): + for itask in schd.pool.get_tasks(): # set prereqs as not-satisfied for prereq in itask.state.prerequisites: prereq._all_satisfied = False diff --git a/tests/integration/test_examples.py b/tests/integration/test_examples.py index 882d4c10163..5cdb1f21552 100644 --- a/tests/integration/test_examples.py +++ b/tests/integration/test_examples.py @@ -138,7 +138,7 @@ async def test_task_pool(one, start): async with start(one): # pump the scheduler's heart manually one.pool.release_runahead_tasks() - assert len(one.pool.main_pool) == 1 + assert len(one.pool.active_tasks) == 1 async def test_exception(one, run, log_filter): diff --git a/tests/integration/test_reload.py b/tests/integration/test_reload.py index d9de0826dfb..5bd07c17af4 100644 --- a/tests/integration/test_reload.py +++ b/tests/integration/test_reload.py @@ -99,11 +99,11 @@ def change_state(_=0): [ # the task should have entered the preparing state before the # reload was requested - '[1/foo waiting(queued) job:00 flows:1] => preparing(queued)', + '[1/foo:waiting(queued)] => preparing(queued)', # the reload should have put the workflow into the paused state - 'PAUSING the workflow now: Reloading workflow', + 'Pausing the workflow: Reloading workflow', # reload should have waited for the task to submit - '[1/foo preparing(queued) job:00 flows:1]' + '[1/foo/00:preparing(queued)]' ' => submitted(queued)', # before then reloading the workflow config 'Reloading the workflow definition.', diff --git a/tests/integration/test_resolvers.py b/tests/integration/test_resolvers.py index 1190cd70978..17e09983dc3 100644 --- a/tests/integration/test_resolvers.py +++ b/tests/integration/test_resolvers.py @@ -20,12 +20,11 @@ import pytest -from cylc.flow import CYLC_LOG from cylc.flow.data_store_mgr import EDGES, TASK_PROXIES from cylc.flow.id import Tokens +from cylc.flow import CYLC_LOG from cylc.flow.network.resolvers import Resolvers from cylc.flow.scheduler import Scheduler -from cylc.flow.workflow_status import StopMode @pytest.fixture @@ -217,44 +216,32 @@ async def test_mutation_mapper(mock_flow): """Test the mapping of mutations to internal command methods.""" meta = {} response = await mock_flow.resolvers._mutation_mapper('pause', {}, meta) - assert response is None + assert response[0] is True # (True, command-uuid-str) with pytest.raises(ValueError): await mock_flow.resolvers._mutation_mapper('non_exist', {}, meta) -@pytest.mark.asyncio -async def test_stop( - one: Scheduler, run: Callable, log_filter: Callable, -): - """Test the stop resolver.""" - async with run(one) as log: - resolvers = Resolvers( - one.data_store_mgr, - schd=one - ) - resolvers.stop(StopMode.REQUEST_CLEAN) - await one.process_command_queue() - assert log_filter( - log, level=logging.INFO, contains="Command actioned: stop" - ) - assert one.stop_mode == StopMode.REQUEST_CLEAN - - -async def test_command_logging(mock_flow, caplog): - """It should log the command, with user name if not owner.""" - caplog.set_level(logging.INFO, logger=CYLC_LOG) - owner = mock_flow.owner - other = f"not-{mock_flow.owner}" - - command = "stop" - mock_flow.resolvers._log_command(command, owner) - assert caplog.records[-1].msg == f"[command] {command}" - mock_flow.resolvers._log_command(command, other) - msg1 = f"[command] {command} (issued by {other})" - assert caplog.records[-1].msg == msg1 - - command = "put_messages" - mock_flow.resolvers._log_command(command, owner) - assert caplog.records[-1].msg == msg1 # (prev message, i.e. not logged). - mock_flow.resolvers._log_command(command, other) - assert caplog.records[-1].msg == f"[command] {command} (issued by {other})" +async def test_command_logging(mock_flow, caplog, log_filter): + """The command log message should include non-owner name.""" + + meta = {} + + caplog.set_level(logging.INFO, CYLC_LOG) + + await mock_flow.resolvers._mutation_mapper("stop", {}, meta) + assert log_filter(caplog, contains='Command "stop" received') + + # put_messages: only log for owner + kwargs = { + "task_job": "1/foo/01", + "event_time": "bedtime", + "messages": [[logging.CRITICAL, "it's late"]] + } + meta["auth_user"] = mock_flow.owner + await mock_flow.resolvers._mutation_mapper("put_messages", kwargs, meta) + assert not log_filter(caplog, contains='Command "put_messages" received:') + + meta["auth_user"] = "Dr Spock" + await mock_flow.resolvers._mutation_mapper("put_messages", kwargs, meta) + assert log_filter( + caplog, contains='Command "put_messages" received from Dr Spock') diff --git a/tests/integration/test_scheduler.py b/tests/integration/test_scheduler.py index 6befe2b34d1..d120fa1421e 100644 --- a/tests/integration/test_scheduler.py +++ b/tests/integration/test_scheduler.py @@ -24,7 +24,6 @@ from cylc.flow.exceptions import CylcError from cylc.flow.parsec.exceptions import ParsecError from cylc.flow.scheduler import Scheduler, SchedulerStop -from cylc.flow.task_outputs import TASK_OUTPUT_SUCCEEDED from cylc.flow.task_state import ( TASK_STATUS_WAITING, TASK_STATUS_SUBMIT_FAILED, @@ -212,7 +211,7 @@ async def test_no_poll_waiting_tasks( log: pytest.LogCaptureFixture async with start(one) as log: # Test assumes start up with a waiting task. - task = (one.pool.get_all_tasks())[0] + task = (one.pool.get_tasks())[0] assert task.state.status == TASK_STATUS_WAITING polled_tasks = capture_polling(one) @@ -325,7 +324,7 @@ async def test_uuid_unchanged_on_restart( cf_uuid = uuid_re.findall(contact_file.read_text()) assert cf_uuid == [schd.uuid_str] - + async def test_restart_timeout( flow, one_conf, @@ -348,11 +347,14 @@ async def test_restart_timeout( id_ = flow(one_conf) # run the workflow to completion - schd = scheduler(id_) - async with start(schd): - for itask in schd.pool.get_all_tasks(): - itask.state_reset(TASK_OUTPUT_SUCCEEDED) - schd.pool.spawn_on_output(itask, TASK_OUTPUT_SUCCEEDED) + # (by setting the only task to completed) + schd = scheduler(id_, paused_start=False) + async with start(schd) as log: + for itask in schd.pool.get_tasks(): + # (needed for job config in sim mode:) + schd.task_job_mgr.submit_task_jobs( + schd.workflow, [itask], None, None) + schd.pool.set([itask.identity], None, None, ['all']) # restart the completed workflow schd = scheduler(id_) diff --git a/tests/integration/test_task_job_mgr.py b/tests/integration/test_task_job_mgr.py index b085162a1da..e1338a7ae77 100644 --- a/tests/integration/test_task_job_mgr.py +++ b/tests/integration/test_task_job_mgr.py @@ -99,7 +99,6 @@ async def test_run_job_cmd_no_hosts_error( # killing the task should not result in an error... schd.task_job_mgr.kill_task_jobs( - schd.workflow, schd.pool.get_tasks() ) diff --git a/tests/integration/test_task_pool.py b/tests/integration/test_task_pool.py index aec35516514..7a0071b61c0 100644 --- a/tests/integration/test_task_pool.py +++ b/tests/integration/test_task_pool.py @@ -14,21 +14,32 @@ # You should have received a copy of the GNU General Public License # along with this program. If not, see . -from copy import deepcopy import logging -from typing import AsyncGenerator, Callable, Iterable, List, Tuple, Union +from typing import ( + TYPE_CHECKING, + AsyncGenerator, + Callable, + Iterable, + List, + Tuple, + Union +) import pytest from pytest import param +from json import loads from cylc.flow import CYLC_LOG -from cylc.flow.cycling import PointBase from cylc.flow.cycling.integer import IntegerPoint from cylc.flow.cycling.iso8601 import ISO8601Point +from cylc.flow.task_events_mgr import TaskEventsManager 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_outputs import ( + TASK_OUTPUT_STARTED, + TASK_OUTPUT_SUCCEEDED +) + +from cylc.flow.flow_mgr import FLOW_ALL, FLOW_NONE from cylc.flow.task_state import ( TASK_STATUS_WAITING, TASK_STATUS_PREPARING, @@ -41,6 +52,11 @@ TASK_STATUSES_ALL, ) +if TYPE_CHECKING: + from cylc.flow.cycling import PointBase + from cylc.flow.scheduler import Scheduler + from cylc.flow.task_proxy import TaskProxy + # NOTE: foo and bar have no parents so at start-up (even with the workflow # paused) they get spawned out to the runahead limit. 2/pub spawns # immediately too, because we spawn autospawn absolute-triggered tasks as @@ -83,8 +99,17 @@ } +def pool_get_task_ids( + pool: List['TaskProxy'] +) -> List[str]: + """Return sorted list of IDs of tasks in a task pool.""" + return sorted( + [itask.identity for itask in pool.get_tasks()] + ) + + def get_task_ids( - name_point_list: Iterable[Tuple[str, Union[PointBase, str, int]]] + name_point_list: Iterable[Tuple[str, Union['PointBase', str, int]]] ) -> List[str]: """Helper function to return sorted task identities from a list of (name, point) tuples.""" @@ -116,14 +141,14 @@ def assert_expected_log( @pytest.fixture(scope='module') async def mod_example_flow( mod_flow: Callable, mod_scheduler: Callable, mod_run: Callable -) -> Scheduler: +) -> 'Scheduler': """Return a scheduler for interrogating its task pool. 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. """ id_ = mod_flow(EXAMPLE_FLOW_CFG) - schd: Scheduler = mod_scheduler(id_, paused_start=True) + schd: 'Scheduler' = mod_scheduler(id_, paused_start=True) async with mod_run(schd): pass return schd @@ -135,7 +160,7 @@ async def example_flow( scheduler: Callable, start, caplog: pytest.LogCaptureFixture, -) -> AsyncGenerator[Scheduler, None]: +) -> AsyncGenerator['Scheduler', None]: """Return a scheduler for interrogating its task pool. This is function-scoped so slower than mod_example_flow; only use this @@ -145,7 +170,7 @@ async def example_flow( # set up caplog and do schd.install()/.initialise()/.configure() instead caplog.set_level(logging.INFO, CYLC_LOG) id_ = flow(EXAMPLE_FLOW_CFG) - schd: Scheduler = scheduler(id_) + schd: 'Scheduler' = scheduler(id_) async with start(schd): yield schd @@ -153,14 +178,14 @@ async def example_flow( @pytest.fixture(scope='module') async def mod_example_flow_2( mod_flow: Callable, mod_scheduler: Callable, mod_run: Callable -) -> Scheduler: +) -> 'Scheduler': """Return a scheduler for interrogating its task pool. 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. """ id_ = mod_flow(EXAMPLE_FLOW_2_CFG) - schd: Scheduler = mod_scheduler(id_, paused_start=True) + schd: 'Scheduler' = mod_scheduler(id_, paused_start=True) async with mod_run(schd): pass return schd @@ -216,9 +241,8 @@ async def test_filter_task_proxies( expected_task_ids: List[str], expected_bad_items: List[str], expected_warnings: List[str], - mod_example_flow: Scheduler, - caplog: pytest.LogCaptureFixture, - monkeypatch, + mod_example_flow: 'Scheduler', + caplog: pytest.LogCaptureFixture ) -> None: """Test TaskPool.filter_task_proxies(). @@ -241,68 +265,6 @@ async def test_filter_task_proxies( assert_expected_log(caplog, expected_warnings) -@pytest.mark.parametrize( - 'items, expected_task_ids, expected_bad_items, expected_warnings', - [ - param( - ['*:waiting'], - ['1/waz', '1/foo', '1/bar', '2/foo', '2/bar', '2/pub', '3/foo', - '3/bar', '4/foo', '4/bar', '5/foo', '5/bar'], [], [], - id="Task state" - ), - ] -) -async def test_filter_task_proxies_hidden( - items: List[str], - expected_task_ids: List[str], - expected_bad_items: List[str], - expected_warnings: List[str], - mod_example_flow: Scheduler, - caplog: pytest.LogCaptureFixture, - monkeypatch, -) -> None: - """Test TaskPool.filter_task_proxies(). - - This is the same as test_filter_task_proxies except we artificially add a - new proxy "1/waz" to the hidden pool. Filtering should find a single copy - each of the hidden and main pool tasks. - - See GitHub #4909: a bug in filtering was doubling up tasks in cycle points - that appeared in both pools. - - The NOTE before EXAMPLE_FLOW_CFG above explains which tasks should be - expected for the tests here. - - Params: - items: Arg passed to filter_task_proxies(). - expected_task_ids: IDs of the TaskProxys that are expected to be - returned, of the form "{point}/{name}"/ - expected_bad_items: Expected to be returned. - expected_warnings: Expected to be logged. - """ - monkeypatch.setattr( - # make Tokens objects mutable to allow deepcopy to work on TaskProxy - # objects - 'cylc.flow.id.Tokens.__setitem__', - lambda self, key, value: dict.__setitem__(self, key, value), - ) - - caplog.set_level(logging.WARNING, CYLC_LOG) - task_pool = mod_example_flow.pool - - # Duplicate a task proxy, rename it, and add it to the hidden pool. - a_task = deepcopy(task_pool.get_tasks()[0]) - a_task.identity = "1/waz" - task_pool.hidden_pool.setdefault(a_task.point, {}) - task_pool.hidden_pool[a_task.point][a_task.identity] = a_task - - itasks, _, bad_items = task_pool.filter_task_proxies(items) - task_ids = [itask.identity for itask in itasks] - assert sorted(task_ids) == sorted(expected_task_ids) - assert sorted(bad_items) == sorted(expected_bad_items) - assert_expected_log(caplog, expected_warnings) - - @pytest.mark.parametrize( 'items, expected_task_ids, expected_warnings', [ @@ -353,7 +315,7 @@ async def test_match_taskdefs( items: List[str], expected_task_ids: List[str], expected_warnings: List[str], - mod_example_flow: Scheduler, + mod_example_flow: 'Scheduler', caplog: pytest.LogCaptureFixture ) -> None: """Test TaskPool.match_taskdefs(). @@ -415,7 +377,7 @@ async def test_hold_tasks( items: List[str], expected_tasks_to_hold_ids: List[str], expected_warnings: List[str], - example_flow: Scheduler, caplog: pytest.LogCaptureFixture, + example_flow: 'Scheduler', caplog: pytest.LogCaptureFixture, db_select: Callable ) -> None: """Test TaskPool.hold_tasks(). @@ -434,7 +396,7 @@ async def test_hold_tasks( task_pool = example_flow.pool n_warnings = task_pool.hold_tasks(items) - for itask in task_pool.get_all_tasks(): + for itask in task_pool.get_tasks(): hold_expected = itask.identity in expected_tasks_to_hold_ids assert itask.state.is_held is hold_expected @@ -448,7 +410,7 @@ async def test_hold_tasks( async def test_release_held_tasks( - example_flow: Scheduler, db_select: Callable + example_flow: 'Scheduler', db_select: Callable ) -> None: """Test TaskPool.release_held_tasks(). @@ -463,7 +425,7 @@ async def test_release_held_tasks( task_pool = example_flow.pool expected_tasks_to_hold_ids = sorted(['1/foo', '1/bar', '3/asd']) task_pool.hold_tasks(expected_tasks_to_hold_ids) - for itask in task_pool.get_all_tasks(): + for itask in task_pool.get_tasks(): hold_expected = itask.identity in expected_tasks_to_hold_ids assert itask.state.is_held is hold_expected assert get_task_ids(task_pool.tasks_to_hold) == expected_tasks_to_hold_ids @@ -472,7 +434,7 @@ async def test_release_held_tasks( # Test task_pool.release_held_tasks(['1/foo', '3/asd']) - for itask in task_pool.get_all_tasks(): + for itask in task_pool.get_tasks(): assert itask.state.is_held is (itask.identity == '1/bar') expected_tasks_to_hold_ids = sorted(['1/bar']) @@ -494,7 +456,7 @@ async def test_release_held_tasks( async def test_hold_point( hold_after_point: int, expected_held_task_ids: List[str], - example_flow: Scheduler, db_select: Callable + example_flow: 'Scheduler', db_select: Callable ) -> None: """Test TaskPool.set_hold_point() and .release_hold_point()""" expected_held_task_ids = sorted(expected_held_task_ids) @@ -506,7 +468,7 @@ async def test_hold_point( assert ('holdcp', str(hold_after_point)) in db_select( example_flow, True, 'workflow_params') - for itask in task_pool.get_all_tasks(): + for itask in task_pool.get_tasks(): hold_expected = itask.identity in expected_held_task_ids assert itask.state.is_held is hold_expected @@ -521,7 +483,7 @@ async def test_hold_point( ('holdcp', None) ] - for itask in task_pool.get_all_tasks(): + for itask in task_pool.get_tasks(): assert itask.state.is_held is False assert task_pool.tasks_to_hold == set() @@ -604,7 +566,7 @@ async def test_reload_stopcp( } } } - schd: Scheduler = scheduler(flow(cfg)) + schd: 'Scheduler' = scheduler(flow(cfg)) async with start(schd): assert str(schd.pool.stop_point) == '2020' await schd.command_reload_workflow() @@ -612,7 +574,7 @@ async def test_reload_stopcp( async def test_runahead_after_remove( - example_flow: Scheduler + example_flow: 'Scheduler' ) -> None: """The runahead limit should be recomputed after tasks are removed. @@ -634,7 +596,7 @@ async def test_load_db_bad_platform( ): """Test that loading an unavailable platform from the database doesn't cause calamitous failure.""" - schd: Scheduler = scheduler(flow(one_conf)) + schd: 'Scheduler' = scheduler(flow(one_conf)) async with start(schd): result = schd.pool.load_db_task_pool_for_restart(0, ( @@ -645,7 +607,7 @@ async def test_load_db_bad_platform( def list_tasks(schd): - """Return a list of task pool tasks (incl hidden pool tasks). + """Return a sorted list of task pool tasks. Returns a list in the format: [ @@ -655,7 +617,7 @@ def list_tasks(schd): """ return sorted( (itask.tokens['cycle'], itask.tokens['task'], itask.state.status) - for itask in schd.pool.get_all_tasks() + for itask in schd.pool.get_tasks() ) @@ -746,8 +708,8 @@ async def test_restart_prereqs( assert list_tasks(schd) == expected_1 # Mark 1/a as succeeded and spawn 1/z - schd.pool.get_all_tasks()[0].state_reset('succeeded') - schd.pool.spawn_on_output(schd.pool.get_all_tasks()[0], 'succeeded') + task_a = schd.pool.get_tasks()[0] + schd.pool.task_events_mgr.process_message(task_a, 1, 'succeeded') assert list_tasks(schd) == expected_2 # Save our progress @@ -770,7 +732,9 @@ async def test_restart_prereqs( schd.data_store_mgr.update_data_structure() # Check resulting dependencies of task z - task_z = schd.pool.get_all_tasks()[0] + task_z = [ + t for t in schd.pool.get_tasks() if t.tdef.name == "z" + ][0] assert sorted( ( p.satisfied @@ -867,8 +831,8 @@ async def test_reload_prereqs( assert list_tasks(schd) == expected_1 # Mark 1/a as succeeded and spawn 1/z - schd.pool.get_all_tasks()[0].state_reset('succeeded') - schd.pool.spawn_on_output(schd.pool.get_all_tasks()[0], 'succeeded') + task_a = schd.pool.get_tasks()[0] + schd.pool.task_events_mgr.process_message(task_a, 1, 'succeeded') assert list_tasks(schd) == expected_2 # Modify flow.cylc to add a new dependency on "z" @@ -880,7 +844,9 @@ async def test_reload_prereqs( assert list_tasks(schd) == expected_3 # Check resulting dependencies of task z - task_z = schd.pool.get_all_tasks()[0] + task_z = [ + t for t in schd.pool.get_tasks() if t.tdef.name == "z" + ][0] assert sorted( ( p.satisfied @@ -904,10 +870,9 @@ async def _test_restart_prereqs_sat(): ] # Mark both as succeeded and spawn 1/c - for itask in schd.pool.get_all_tasks(): - itask.state_reset('succeeded') - schd.pool.spawn_on_output(itask, 'succeeded') - schd.workflow_db_mgr.put_insert_task_outputs(itask) + for itask in schd.pool.get_tasks(): + schd.pool.task_events_mgr.process_message(itask, 1, 'succeeded') + schd.workflow_db_mgr.put_update_task_outputs(itask) schd.pool.remove_if_complete(itask) schd.workflow_db_mgr.process_queued_ops() assert list_tasks(schd) == [ @@ -922,7 +887,7 @@ async def _test_restart_prereqs_sat(): ] # Check resulting dependencies of task z - task_c = schd.pool.get_all_tasks()[0] + task_c = schd.pool.get_tasks()[0] assert sorted( (*key, satisfied) for prereq in task_c.state.prerequisites @@ -1018,8 +983,8 @@ async def test_runahead_limit_for_sequence_before_start_cycle( ): """It should obey the runahead limit. - Ensure the runahead limit is computed correctly for sequences that begin - before the start cycle. + Ensure the runahead limit is computed correctly for sequences before the + start cycle See https://github.com/cylc/cylc-flow/issues/5603 """ @@ -1079,8 +1044,12 @@ async def test_db_update_on_removal( async with start(schd): task_a = schd.pool.get_tasks()[0] + # set it to running (and submitted implied) + schd.pool.set( + [task_a.identity], [TASK_OUTPUT_STARTED], None, ['all']) + # set the task to running - task_a.state_reset('running') + schd.pool.task_events_mgr.process_message(task_a, 1, 'started') # update the db await schd.update_data_structure() @@ -1092,7 +1061,7 @@ async def test_db_update_on_removal( ] # mark the task as succeeded and allow it to be removed from the pool - task_a.state_reset('succeeded') + schd.pool.task_events_mgr.process_message(task_a, 1, 'succeeded') schd.pool.remove_if_complete(task_a) # update the DB, note no new tasks have been added to the pool @@ -1110,8 +1079,7 @@ async def test_no_flow_tasks_dont_spawn( ): """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". + No-flow tasks (i.e `--flow=none`) are not attached to any "flow". See https://github.com/cylc/cylc-flow/issues/5613 """ @@ -1128,9 +1096,13 @@ async def test_no_flow_tasks_dont_spawn( 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) + + # set as no-flow: + task_a.flow_nums = set() + + # Set as completed: should not spawn children. + schd.pool.set([task_a.identity], None, None, [FLOW_NONE]) for flow_nums, force, pool in ( # outputs yielded from a no-flow task should not spawn downstreams @@ -1153,16 +1125,13 @@ async def test_no_flow_tasks_dont_spawn( 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 + for itask in schd.pool.get_tasks() ] == pool @@ -1176,23 +1145,16 @@ async def test_task_proxy_remove_from_queues( # Set up a scheduler with a non-default queue: one_conf['scheduling'] = { 'queues': {'queue_two': {'members': 'one, control'}}, - 'graph': {'R1': 'two & one & hidden & control & hidden_control'}, + 'graph': {'R1': 'two & one & control'}, } schd = scheduler(flow(one_conf)) async with start(schd): # Get a list of itasks: itasks = schd.pool.get_tasks() - point = itasks[0].point for itask in itasks: id_ = itask.identity - # Move some tasks to the hidden_pool to ensure that these are - # removed too: - if 'hidden' in itask.identity: - schd.pool.hidden_pool.setdefault(point, {id_: itask}) - del schd.pool.main_pool[point][id_] - # The meat of the test - remove itask from pool if it # doesn't have "control" in the name: if 'control' not in id_: @@ -1203,12 +1165,11 @@ async def test_task_proxy_remove_from_queues( name: [itask.identity for itask in queue.deque] for name, queue in schd.pool.task_queue_mgr.queues.items()} - assert queues_after['default'] == ['1/hidden_control'] assert queues_after['queue_two'] == ['1/control'] async def test_runahead_offset_start( - mod_example_flow_2: Scheduler + mod_example_flow_2: 'Scheduler' ) -> None: """Late-start recurrences should not break the runahead limit at start-up. @@ -1229,20 +1190,19 @@ async def test_detect_incomplete_tasks( If a task finishes without completing all required outputs, then it should be marked as incomplete. """ - incomplete_final_task_states = [ - TASK_STATUS_FAILED, - TASK_STATUS_EXPIRED, - TASK_STATUS_SUBMIT_FAILED, - ] + final_task_states = { + TASK_STATUS_FAILED: TaskEventsManager.EVENT_FAILED, + TASK_STATUS_EXPIRED: TaskEventsManager.EVENT_EXPIRED, + TASK_STATUS_SUBMIT_FAILED: TaskEventsManager.EVENT_SUBMIT_FAILED + } id_ = flow({ 'scheduler': { 'allow implicit tasks': 'True', }, 'scheduling': { 'graph': { - # a workflow with one task for each of the incomplete final - # task states - 'R1': '\n'.join(incomplete_final_task_states) + # a workflow with one task for each of the final task states + 'R1': '\n'.join(final_task_states.keys()) } } }) @@ -1251,14 +1211,326 @@ async def test_detect_incomplete_tasks( itasks = schd.pool.get_tasks() for itask in itasks: # spawn the output corresponding to the task - schd.pool.spawn_on_output(itask, itask.tdef.name) + schd.pool.task_events_mgr.process_message( + itask, 1, + final_task_states[itask.tdef.name] + ) # ensure that it is correctly identified as incomplete assert itask.state.outputs.get_incomplete() assert itask.state.outputs.is_incomplete() - assert log_filter( - log, contains=f"[{itask}] did not complete required outputs:") - # the task should not have been removed - assert itask in schd.pool.get_tasks() + if itask.tdef.name == TASK_STATUS_EXPIRED: + assert log_filter( + log, + contains=f"[{itask}] removed (expired)" + ) + # the task should have been removed + assert itask not in schd.pool.get_tasks() + else: + assert log_filter( + log, + contains=f"[{itask}] did not complete required outputs:" + ) + # the task should not have been removed + assert itask in schd.pool.get_tasks() + + +async def test_future_trigger_final_point( + flow, + scheduler, + start, + log_filter, +): + """Check spawning of future-triggered tasks: foo[+P1] => bar. + + Don't spawn if a prerequisite reaches beyond the final cycle point. + + """ + id_ = flow( + { + 'scheduler': { + 'allow implicit tasks': 'True', + }, + 'scheduling': { + 'cycling mode': 'integer', + 'initial cycle point': 1, + 'final cycle point': 1, + 'graph': { + 'P1': "foo\n foo[+P1] & bar => baz" + } + } + } + ) + schd = scheduler(id_) + async with start(schd) as log: + for itask in schd.pool.get_tasks(): + schd.pool.spawn_on_output(itask, "succeeded") + assert log_filter( + log, + regex=( + ".*1/baz.*not spawned: a prerequisite is beyond" + r" the workflow stop point \(1\)" + ) + ) + + +async def test_set_failed_complete( + flow, + scheduler, + start, + one_conf, + log_filter, + db_select: Callable +): + """Test manual completion of an incomplete failed task.""" + id_ = flow(one_conf) + schd = scheduler(id_) + async with start(schd) as log: + one = schd.pool.get_tasks()[0] + one.state_reset(is_queued=False) + + schd.pool.task_events_mgr.process_message(one, 1, "failed") + assert log_filter( + log, regex="1/one.* setting missed output: submitted") + assert log_filter( + log, regex="1/one.* setting missed output: started") + assert log_filter( + log, regex="failed.* did not complete required outputs") + + # Set failed task complete via default "set" args. + schd.pool.set([one.identity], None, None, ['all']) + + assert log_filter( + log, contains='output 1/one:succeeded completed') + + db_outputs = db_select( + schd, True, 'task_outputs', 'outputs', + **{'name': 'one'} + ) + assert ( + sorted(loads((db_outputs[0])[0])) == [ + "failed", "started", "submitted", "succeeded" + ] + ) + + +async def test_set_prereqs( + flow, + scheduler, + start, + log_filter, +): + """Check manual setting of prerequisites. + + """ + id_ = flow( + { + 'scheduler': { + 'allow implicit tasks': 'True', + }, + 'scheduling': { + 'graph': { + 'R1': "foo & bar => baz" + } + } + } + ) + schd = scheduler(id_) + + async with start(schd) as log: + + # it should start up with 1/foo and 1/bar + assert ( + pool_get_task_ids(schd.pool) == ["1/bar", "1/foo"] + ) + + # set one prereq of future task baz + schd.pool.set(["1/baz"], None, ["1/foo:succeeded"], ['all']) + + assert ( + pool_get_task_ids(schd.pool) == [ + "1/bar", "1/baz", "1/foo" + ] + ) + + # get the 1/baz task proxy + baz = schd.pool.get_task(IntegerPoint("1"), "baz") + assert not baz.state.prerequisites_all_satisfied() + + # set its other prereq + schd.pool.set(["1/baz"], None, ["1/bar:succeeded"], ['all']) + + # it should now be fully satisfied + assert baz.state.prerequisites_all_satisfied() + + # try to set an invalid prereq + schd.pool.set(["1/baz"], None, ["1/qux:succeeded"], ['all']) + assert log_filter( + log, contains="1/baz does not depend on 1/qux:succeeded") + + +async def test_set_outputs_live( + flow, + scheduler, + start, + log_filter, +): + """Check manual set outputs in an active (spawned) task. + + """ + id_ = flow( + { + 'scheduler': { + 'allow implicit tasks': 'True', + }, + 'scheduling': { + 'graph': { + 'R1': """ + foo:x => bar + foo => baz + foo:y + """ + } + }, + 'runtime': { + 'foo': { + 'outputs': { + 'x': 'x', + 'y': 'y' + } + } + } + } + ) + schd = scheduler(id_) + + async with start(schd) as log: + + # it should start up with just 1/foo + assert pool_get_task_ids(schd.pool) == ["1/foo"] + + # fake failed + foo = schd.pool.get_task(IntegerPoint("1"), "foo") + foo.state_reset(is_queued=False) + schd.pool.task_events_mgr.process_message(foo, 1, 'failed') + + # set foo:x: it should spawn bar but not baz + schd.pool.set(["1/foo"], ["x"], None, ['all']) + assert ( + pool_get_task_ids(schd.pool) == ["1/bar", "1/foo"] + ) + + # set foo:succeed: it should spawn baz but foo remains incomplete. + schd.pool.set(["1/foo"], ["succeeded"], None, ['all']) + assert ( + pool_get_task_ids(schd.pool) == ["1/bar", "1/baz", "1/foo"] + ) + + # it should complete implied outputs (submitted, started) too + assert log_filter( + log, contains="setting missed output: submitted") + assert log_filter( + log, contains="setting missed output: started") + + # set foo (default: all required outputs): complete y. + schd.pool.set(["1/foo"], None, None, ['all']) + assert ( + pool_get_task_ids(schd.pool) == ["1/bar", "1/baz"] + ) + assert log_filter( + log, contains="[1/foo/00:succeeded] completed") + + +async def test_set_outputs_future( + flow, + scheduler, + start, + log_filter, +): + """Check manual setting of future task outputs. + + """ + id_ = flow( + { + 'scheduler': { + 'allow implicit tasks': 'True', + }, + 'scheduling': { + 'graph': { + 'R1': "a => b => c" + } + } + } + ) + schd = scheduler(id_) + + async with start(schd) as log: + + # it should start up with just 1/a + assert pool_get_task_ids(schd.pool) == ["1/a"] + + # setting future task b succeeded should spawn c but not b + schd.pool.set(["1/b"], ["succeeded"], None, ['all']) + assert ( + pool_get_task_ids(schd.pool) == ["1/a", "1/c"] + ) + + # try to set an invalid output + schd.pool.set(["1/b"], ["shrub"], None, ['all']) + assert log_filter(log, contains="output 1/b:shrub not found") + + +async def test_prereq_satisfaction( + flow, + scheduler, + start, + log_filter, +): + """Check manual setting of future task outputs. + + """ + id_ = flow( + { + 'scheduler': { + 'allow implicit tasks': 'True', + }, + 'scheduling': { + 'graph': { + 'R1': "a:x & a:y => b" + } + }, + 'runtime': { + 'a': { + 'outputs': { + 'x': 'x', + 'y': 'y' + } + } + } + } + ) + schd = scheduler(id_) + async with start(schd) as log: + # it should start up with just 1/a + assert pool_get_task_ids(schd.pool) == ["1/a"] + # spawn b + schd.pool.set(["1/a"], ["x"], None, ['all']) + assert ( + pool_get_task_ids(schd.pool) == ["1/a", "1/b"] + ) + + b = schd.pool.get_task(IntegerPoint("1"), "b") + + assert not b.is_waiting_prereqs_done() + + # set valid and invalid prerequisites, check log. + b.satisfy_me(["1/a:x", "1/a:y", "1/a:z", "1/a:w"]) + assert log_filter(log, contains="1/b does not depend on 1/a:z") + assert log_filter(log, contains="1/b does not depend on 1/a:w") + assert not log_filter(log, contains="1/b does not depend on 1/a:x") + assert not log_filter(log, contains="1/b does not depend on 1/a:y") + + assert b.is_waiting_prereqs_done() + @pytest.mark.parametrize('compat_mode', ['compat-mode', 'normal-mode']) @@ -1284,6 +1556,7 @@ async def test_compute_runahead( """ if cycling_mode == 'integer': + config = { 'scheduler': { 'allow implicit tasks': 'True', @@ -1408,32 +1681,40 @@ async def test_runahead_future_trigger( async with start(schd, level=logging.DEBUG): assert str(schd.pool.runahead_limit_point) == '20010103' schd.pool.release_runahead_tasks() - for itask in schd.pool.get_all_tasks(): + for itask in schd.pool.get_tasks(): schd.pool.spawn_on_output(itask, 'succeeded') # future trigger raises the limit by one cycle point assert str(schd.pool.runahead_limit_point) == '20010104' -async def test_compute_runahead_against_task_state( - flow, - scheduler, - start, - monkeypatch, -): - """For each task status check whether changing the oldest task - to that status will cause compute_runahead to make a change. - """ - states = [ +@pytest.mark.parametrize( + 'status, expected', + [ # (Status, Are we expecting an update?) (TASK_STATUS_WAITING, False), (TASK_STATUS_EXPIRED, True), (TASK_STATUS_PREPARING, False), - (TASK_STATUS_SUBMIT_FAILED, True), + (TASK_STATUS_SUBMIT_FAILED, False), (TASK_STATUS_SUBMITTED, False), (TASK_STATUS_RUNNING, False), (TASK_STATUS_FAILED, True), (TASK_STATUS_SUCCEEDED, True) ] +) +async def test_runahead_c7_compat_task_state( + flow, + scheduler, + start, + monkeypatch, + status, + expected +): + """For each task status check whether changing the oldest task + to that status will cause compute_runahead to make a change. + + Compat mode: Cylc 7 ignored failed tasks but not submit-failed! + + """ config = { 'scheduler': { 'allow implicit tasks': 'True', @@ -1452,21 +1733,19 @@ def max_cycle(tasks): return max([int(t.tokens.get("cycle")) for t in tasks]) monkeypatch.setattr( - 'cylc.flow.flags.cylc7_back_compat', - True) + 'cylc.flow.flags.cylc7_back_compat', True) monkeypatch.setattr( 'cylc.flow.task_events_mgr.TaskEventsManager._insert_task_job', lambda *_: True) schd = scheduler(flow(config)) async with start(schd): - for task_status, new_runahead in states: - before = max_cycle(schd.pool.get_tasks()) - itask = schd.pool.get_task(ISO8601Point(f'{before - 2:04}'), 'a') - schd.task_events_mgr.process_message( - itask, - logging.INFO, - task_status, - ) - after = max_cycle(schd.pool.get_tasks()) - assert bool(before != after) == new_runahead + before = max_cycle(schd.pool.get_tasks()) + itask = schd.pool.get_task(ISO8601Point(f'{before - 2:04}'), 'a') + schd.task_events_mgr.process_message( + itask, + logging.DEBUG, + status, + ) + after = max_cycle(schd.pool.get_tasks()) + assert bool(before != after) == expected diff --git a/tests/integration/test_trigger.py b/tests/integration/test_trigger.py index e008caa4a7c..e0d0e0eaf76 100644 --- a/tests/integration/test_trigger.py +++ b/tests/integration/test_trigger.py @@ -37,7 +37,7 @@ async def test_trigger_invalid(mod_one, start, log_filter, flow_strs): """Ensure invalid flow values are rejected.""" async with start(mod_one) as log: log.clear() - assert mod_one.pool.force_trigger_tasks(['*'], flow_strs) == 0 + assert mod_one.pool.force_trigger_tasks(['*'], flow_strs) == None assert len(log_filter(log, level=logging.WARN)) == 1 diff --git a/tests/integration/tui/screenshots/test_auto_expansion.later-time.html b/tests/integration/tui/screenshots/test_auto_expansion.later-time.html index f5a19fd428d..2b7137f8b3d 100644 --- a/tests/integration/tui/screenshots/test_auto_expansion.later-time.html +++ b/tests/integration/tui/screenshots/test_auto_expansion.later-time.html @@ -1,9 +1,9 @@
Cylc Tui   workflows filtered (W - edit, E - reset)                             
                                                                                 
 - ~cylc                                                                         
-   - one - paused                                                               
-      - ̿○ 1                                                                     
-           ̿○ b                                                                  
+   - one - paused 1■                                                            
+      - ̿● 1                                                                     
+         + ̿●  b                                                                
       - ̿○ 2                                                                     
          - ̿○ A                                                                  
               ̿○ a                                                               
diff --git a/tests/integration/tui/screenshots/test_online_mutation.hold-mutation-selected.html b/tests/integration/tui/screenshots/test_online_mutation.hold-mutation-selected.html
index 34be2ffa0ce..af0a063a14f 100644
--- a/tests/integration/tui/screenshots/test_online_mutation.hold-mutation-selected.html
+++ b/tests/integration/tui/screenshots/test_online_mutation.hold-mutation-selected.html
@@ -9,7 +9,7 @@
                  < log                                      >                 
                  < poll                                     >                 
                  < release                                  >                 
-                 < show                                     >                 
+                 < set                                      >                 
                                                                               
 quit: q  help:  q to close                                     ↥ ↧ Home End   
 filter tasks: T────────────────────────────────────────────────               
diff --git a/tests/integration/tui/screenshots/test_show.fail.html b/tests/integration/tui/screenshots/test_show.fail.html
index f788e5b3a55..66a4836bc01 100644
--- a/tests/integration/tui/screenshots/test_show.fail.html
+++ b/tests/integration/tui/screenshots/test_show.fail.html
@@ -21,7 +21,7 @@
                  <                                                           
                  <                                                           
                  <                                                           
-                                                                             
+                 <                                                           
                                                                              
                                                                              
                                                                              
diff --git a/tests/integration/tui/screenshots/test_show.success.html b/tests/integration/tui/screenshots/test_show.success.html
index afdcd1a73b4..fe016f285df 100644
--- a/tests/integration/tui/screenshots/test_show.success.html
+++ b/tests/integration/tui/screenshots/test_show.success.html
@@ -17,7 +17,6 @@
                  state: waiting                                               
                  prerequisites: (None)                                        
                  outputs: ('-': not completed)                                
-                   - 1/foo expired                                            
                    - 1/foo submitted                                          
                    - 1/foo submit-failed                                      
                    - 1/foo started                                            
@@ -36,6 +35,7 @@
                                                                                 
                                                                                 
                                                                                 
+                                                                                
 quit: q  help: h  context: enter  tree: - ← + →  navigation: ↑ ↓ ↥ ↧ Home End   
 filter tasks: T f s r R  filter workflows: W E p                                
 
\ No newline at end of file diff --git a/tests/integration/tui/test_app.py b/tests/integration/tui/test_app.py index 908866fc948..8931dfbd348 100644 --- a/tests/integration/tui/test_app.py +++ b/tests/integration/tui/test_app.py @@ -19,6 +19,11 @@ import urwid from cylc.flow.cycling.integer import IntegerPoint +from cylc.flow.task_outputs import ( + TASK_OUTPUT_SUBMITTED, + TASK_OUTPUT_STARTED, + TASK_OUTPUT_SUCCEEDED +) from cylc.flow.task_state import ( # TASK_STATUS_RUNNING, TASK_STATUS_SUCCEEDED, @@ -332,9 +337,10 @@ async def test_auto_expansion(flow, scheduler, start, rakiura): for task in ('a', 'b'): itask = schd.pool.get_task(IntegerPoint('1'), task) - itask.state_reset(TASK_STATUS_SUCCEEDED) - schd.pool.spawn_on_output(itask, TASK_STATUS_SUCCEEDED) + schd.pool._set_outputs_itask(itask, [TASK_OUTPUT_SUCCEEDED]) + await schd.update_data_structure() + schd.update_data_store() rk.compare_screenshot( 'later-time', diff --git a/tests/integration/tui/test_mutations.py b/tests/integration/tui/test_mutations.py index 18da88d227d..fd7a58675fd 100644 --- a/tests/integration/tui/test_mutations.py +++ b/tests/integration/tui/test_mutations.py @@ -27,7 +27,8 @@ async def gen_commands(schd): while True: await asyncio.sleep(0.1) if not schd.command_queue.empty(): - yield schd.command_queue.get() + # (ignore first item: command UUID) + yield schd.command_queue.get()[1:] async def test_online_mutation( @@ -76,7 +77,7 @@ async def test_online_mutation( command = None async for command in gen_commands(schd): break - assert command == ('hold', (['1/one'],), {}) + assert command == ('hold', [], {'tasks': ['1/one']}) # close the dialogue and re-run the hold mutation rk.user_input('q', 'q', 'enter') diff --git a/tests/integration/tui/test_show.py b/tests/integration/tui/test_show.py index c664cdd1393..063fd4b8781 100644 --- a/tests/integration/tui/test_show.py +++ b/tests/integration/tui/test_show.py @@ -48,7 +48,7 @@ async def test_show(flow, scheduler, start, rakiura, monkeypatch): rk.user_input('down', 'down', 'enter') # select the "show" context option - rk.user_input(*(['down'] * 6), 'enter') + rk.user_input(*(['down'] * 7), 'enter') rk.compare_screenshot( 'success', 'the show output should be displayed', @@ -63,7 +63,7 @@ def cli_cmd_fail(*args, **kwargs): ) # select the "show" context option - rk.user_input('q', 'enter', *(['down'] * 6), 'enter') + rk.user_input('q', 'enter', *(['down'] * 7), 'enter') rk.compare_screenshot( 'fail', 'the error should be displayed', diff --git a/tests/integration/validate/test_outputs.py b/tests/integration/validate/test_outputs.py index a91393366b5..5675372a09f 100644 --- a/tests/integration/validate/test_outputs.py +++ b/tests/integration/validate/test_outputs.py @@ -50,6 +50,7 @@ 'foo+bar', # keywords 'required', + 'optional', 'all', # built-in qualifiers 'succeeded', diff --git a/tests/unit/scripts/test_completion_server.py b/tests/unit/scripts/test_completion_server.py index 186e13b7272..5d1c5394e8f 100644 --- a/tests/unit/scripts/test_completion_server.py +++ b/tests/unit/scripts/test_completion_server.py @@ -20,6 +20,7 @@ from cylc.flow.id import Tokens from cylc.flow.network.scan import scan from cylc.flow.scripts.completion_server import ( + _list_prereqs_and_outputs, server, complete_cylc, complete_command, @@ -540,7 +541,7 @@ async def test_list_dir(tmp_path, monkeypatch): # => list $PWD assert { str(path) - for path in await _list_dir(None, None) + for path in await _list_dir(None) } == {'x/'} # no trailing `/` at the end of the path @@ -548,7 +549,7 @@ async def test_list_dir(tmp_path, monkeypatch): # => list the parent assert { str(path) - for path in await _list_dir(None, 'x') + for path in await _list_dir('x') } == {'x/'} # # trailing `/` at the end of the path @@ -556,14 +557,14 @@ async def test_list_dir(tmp_path, monkeypatch): # # => list dir path assert { str(path) - for path in await _list_dir(None, 'x/') + for path in await _list_dir('x/') } == {'x/y/', 'x/z'} # "y" is a dir, "z" is a file # listing a file # => noting to list, just return the file assert { str(path) - for path in await _list_dir(None, 'x/z/') + for path in await _list_dir('x/z/') } == {'x/z'} # --- absolute paths --- @@ -574,7 +575,7 @@ async def test_list_dir(tmp_path, monkeypatch): assert { # '/'.join(path.rsplit('/', 2)[-2:]) path.replace(str(tmp_path), '') - for path in await _list_dir(None, str(tmp_path / 'x')) + for path in await _list_dir(str(tmp_path / 'x')) } == {'/x/'} # trailing `/` at the end of the path @@ -582,14 +583,14 @@ async def test_list_dir(tmp_path, monkeypatch): # => list dir path assert { path.replace(str(tmp_path), '') - for path in await _list_dir(None, str(tmp_path / 'x') + '/') + for path in await _list_dir(str(tmp_path / 'x') + '/') } == {'/x/y/', '/x/z'} # "y" is a dir, "z" is a file # listing a file # => noting to list, just return the file assert { path.replace(str(tmp_path), '') - for path in await _list_dir(None, str(tmp_path / 'x' / 'z') + '/') + for path in await _list_dir(str(tmp_path / 'x' / 'z') + '/') } == {'/x/z'} @@ -599,12 +600,12 @@ async def test_list_flows(): Currently this only provides the textural options i.e. it doesn't list "flows" running in a workflow, yet... """ - assert 'all' in await list_flows(None, None) + assert 'all' in await list_flows(None) async def test_list_colours(): """Test listing values for the --color option.""" - assert 'always' in await list_colours(None, None) + assert 'always' in await list_colours(None) async def test_cli_detokenise(): @@ -715,3 +716,16 @@ def _get_current_completion_script_version(_script, lang): out, err = capsys.readouterr() assert not out # never write to stdout assert not err + + +async def test_prereqs_and_outputs(): + """Test the error cases for listing task prereqs/outputs. + + The succeess cases are tested in an integration test (requires a running + scheduler). + """ + # if no tokens are provided, no prereqs or outputs are returned + assert await _list_prereqs_and_outputs([]) == ([], []) + + # if an invalid workflow is provided, we can't list anything + assert await _list_prereqs_and_outputs([Tokens(workflow='no-such-workflow')]) == ([], []) diff --git a/tests/unit/scripts/test_trigger.py b/tests/unit/scripts/test_trigger.py index d464bda0a34..87f392d73f8 100644 --- a/tests/unit/scripts/test_trigger.py +++ b/tests/unit/scripts/test_trigger.py @@ -18,12 +18,17 @@ from optparse import Values import pytest -from typing import Iterable, Optional, Tuple, Type +from typing import Optional, Tuple, Type from cylc.flow.exceptions import InputError from cylc.flow.option_parsers import Options -from cylc.flow.flow_mgr import FLOW_ALL, FLOW_NEW, FLOW_NONE -from cylc.flow.scripts.trigger import get_option_parser, _validate +from cylc.flow.flow_mgr import ( + FLOW_ALL, + FLOW_NEW, + FLOW_NONE, + validate_flow_opts +) +from cylc.flow.scripts.trigger import get_option_parser Opts = Options(get_option_parser()) @@ -39,6 +44,13 @@ ), None ), + ( + Opts( + flow=None, + flow_wait=False + ), + None + ), ( Opts( flow=[FLOW_NEW], @@ -57,17 +69,6 @@ "Multiple flow options must all be integer valued" ) ), - ( - Opts( - flow=[FLOW_ALL], - flow_wait=False, - flow_descr="the quick brown fox" - ), - ( - InputError, - "Metadata is only for new flows" - ) - ), ( Opts( flow=["cheese"], @@ -75,7 +76,7 @@ ), ( InputError, - "Flow values must be integer, 'all', 'new', or 'none'" + "Flow values must be an integer, or 'all', 'new', or 'none'" ) ), ( @@ -117,7 +118,7 @@ def test_validate( if expected_err: err, msg = expected_err with pytest.raises(err) as exc: - _validate(opts) + validate_flow_opts(opts) assert msg in str(exc.value) else: - _validate(opts) + validate_flow_opts(opts) diff --git a/tests/unit/test_flow_mgr.py b/tests/unit/test_flow_mgr.py index 5fea08c1b97..52d865e861b 100644 --- a/tests/unit/test_flow_mgr.py +++ b/tests/unit/test_flow_mgr.py @@ -43,25 +43,54 @@ def test_all( patch_datetime_now, caplog: pytest.LogCaptureFixture, ): + """Test flow number management.""" + db_mgr = WorkflowDatabaseManager() flow_mgr = FlowMgr(db_mgr) caplog.set_level(logging.INFO, CYLC_LOG) - count = 1 + # automatic: expect flow number 1 meta = "the quick brown fox" - msg1 = f"flow: {count} ({meta}) {FAKE_NOW}" - assert flow_mgr.get_new_flow(meta) == count + assert flow_mgr.get_flow_num(None, meta) == 1 + msg1 = f"flow: 1 ({meta}) {FAKE_NOW}" assert f"New {msg1}" in caplog.messages - count = 2 - meta = "jumped over the lazy" - msg2 = f"flow: {count} ({meta}) {FAKE_NOW}" - assert flow_mgr.get_new_flow(meta) == count + # automatic: expect 2 + meta = "jumped over the lazy dog" + assert flow_mgr.get_flow_num(None, meta) == 2 + msg2 = f"flow: 2 ({meta}) {FAKE_NOW}" assert f"New {msg2}" in caplog.messages + # give flow 2: not a new flow + meta = "jumped over the moon" + assert flow_mgr.get_flow_num(2, meta) == 2 + msg3 = f"flow: 2 ({meta}) {FAKE_NOW}" + assert f"New {msg3}" not in caplog.messages + assert f"Ignoring flow metadata \"{meta}\": 2 is not a new flow" in caplog.messages + + # give flow 4: new flow + meta = "jumped over the cheese" + assert flow_mgr.get_flow_num(4, meta) == 4 + msg4 = f"flow: 4 ({meta}) {FAKE_NOW}" + assert f"New {msg4}" in caplog.messages + + # automatic: expect 3 + meta = "jumped over the log" + assert flow_mgr.get_flow_num(None, meta) == 3 + msg5 = f"flow: 3 ({meta}) {FAKE_NOW}" + assert f"New {msg5}" in caplog.messages + + # automatic: expect 5 (skip over 4) + meta = "crawled under the log" + assert flow_mgr.get_flow_num(None, meta) == 5 + msg6 = f"flow: 5 ({meta}) {FAKE_NOW}" + assert f"New {msg6}" in caplog.messages flow_mgr._log() assert ( "Flows:\n" f"{msg1}\n" - f"{msg2}" + f"{msg2}\n" + f"{msg4}\n" + f"{msg5}\n" + f"{msg6}" ) in caplog.messages diff --git a/tests/unit/test_graph_parser.py b/tests/unit/test_graph_parser.py index ddd443a3597..ca19b2060b4 100644 --- a/tests/unit/test_graph_parser.py +++ b/tests/unit/test_graph_parser.py @@ -778,6 +778,10 @@ def test_family_optional_outputs(qual, task_output): "FAM => foo", # bare family on LHS "Illegal family trigger" ], + [ + "FAM:expire-all => foo", + "must be optional" + ], ] ) def test_family_trigger_errors(graph, error): @@ -819,6 +823,10 @@ def test_family_trigger_errors(graph, error): "a:finish? => b", "Pseudo-output a:finished can't be optional", ], + [ + "a:expire => b", + "must be optional", + ], ] ) def test_task_optional_output_errors_order( diff --git a/tests/unit/test_id_match.py b/tests/unit/test_id_match.py index d26e85b092d..8727023a864 100644 --- a/tests/unit/test_id_match.py +++ b/tests/unit/test_id_match.py @@ -43,6 +43,7 @@ def _task_proxy(id_, hier): hier.append('root') tdef = create_autospec(TaskDef, namespace_hierarchy=hier) tdef.name = tokens['task'] + tdef.expiration_offset = None return TaskProxy( Tokens('~user/workflow'), tdef, @@ -127,7 +128,7 @@ def test_filter_ids_task_mode(task_pool, ids, matched, not_matched): {} ) - _matched, _not_matched = filter_ids([pool], ids) + _matched, _not_matched = filter_ids(pool, ids) assert [get_task_id(itask) for itask in _matched] == matched assert _not_matched == not_matched @@ -188,21 +189,21 @@ def test_filter_ids_cycle_mode(task_pool, ids, matched, not_matched): {} ) - _matched, _not_matched = filter_ids([pool], ids, out=IDTokens.Cycle) + _matched, _not_matched = filter_ids(pool, ids, out=IDTokens.Cycle) assert _matched == [IntegerPoint(i) for i in matched] assert _not_matched == not_matched def test_filter_ids_invalid(caplog): """Ensure invalid IDs are handled elegantly.""" - matched, not_matched = filter_ids([{}], ['#']) + matched, not_matched = filter_ids({}, ['#']) assert matched == [] assert not_matched == ['#'] assert caplog.record_tuples == [ ('cylc', 30, 'No active tasks matching: #'), ] caplog.clear() - matched, not_matched = filter_ids([{}], ['#'], warn=False) + matched, not_matched = filter_ids({}, ['#'], warn=False) assert caplog.record_tuples == [] @@ -216,7 +217,7 @@ def test_filter_ids_pattern_match_off(task_pool): ) _matched, _not_matched = filter_ids( - [pool], + pool, ['1/a'], out=IDTokens.Task, pattern_match=False, @@ -238,7 +239,7 @@ def test_filter_ids_toggle_pattern_matching(task_pool, caplog): # ensure pattern matching works _matched, _not_matched = filter_ids( - [pool], + pool, ids, out=IDTokens.Task, pattern_match=True, @@ -249,7 +250,7 @@ def test_filter_ids_toggle_pattern_matching(task_pool, caplog): # ensure pattern matching can be disabled caplog.clear() _matched, _not_matched = filter_ids( - [pool], + pool, ids, out=IDTokens.Task, pattern_match=False, @@ -285,7 +286,7 @@ def test_filter_ids_namespace_hierarchy(task_pool, ids, matched, not_matched): ) _matched, _not_matched = filter_ids( - [pool], + pool, ids, pattern_match=False, ) diff --git a/tests/unit/test_prerequisite.py b/tests/unit/test_prerequisite.py index d8981c2febf..410176e508c 100644 --- a/tests/unit/test_prerequisite.py +++ b/tests/unit/test_prerequisite.py @@ -18,6 +18,7 @@ from cylc.flow.cycling.loader import ISO8601_CYCLING_TYPE, get_point from cylc.flow.prerequisite import Prerequisite +from cylc.flow.id import Tokens @pytest.fixture @@ -61,10 +62,10 @@ def test_satisfied(prereq): ('2001', 'd', 'custom'): False, } # mark two prerequisites as satisfied - prereq.satisfy_me({ - ('2000', 'b', 'succeeded'), - ('2000', 'c', 'succeeded'), - }) + prereq.satisfy_me([ + Tokens('2000/b:succeeded', relative=True), + Tokens('2000/c:succeeded', relative=True), + ]) assert prereq.satisfied == { # the pre-initial dependency should be marked as satisfied ('1999', 'a', 'succeeded'): 'satisfied naturally', diff --git a/tests/unit/test_subprocpool.py b/tests/unit/test_subprocpool.py index 981ca8e75f9..48ff0c3db3b 100644 --- a/tests/unit/test_subprocpool.py +++ b/tests/unit/test_subprocpool.py @@ -328,6 +328,7 @@ def test__run_command_exit_add_to_badhosts_log(caplog, mock_ctx): SimpleNamespace( name='t', dependencies={}, sequential='', external_triggers=[], xtrig_labels={}, + expiration_offset=None, outputs={ 'submitted': [None, None], 'submit-failed': [None, None] }, diff --git a/tests/unit/test_task_state.py b/tests/unit/test_task_state.py index 82e19f9e215..a64056ef574 100644 --- a/tests/unit/test_task_state.py +++ b/tests/unit/test_task_state.py @@ -79,58 +79,6 @@ def test_reset(state, is_held, should_reset): assert tstate.status == state -@pytest.mark.parametrize( - 'before,after,outputs', - [ - ( - (TASK_STATUS_WAITING, False), - (TASK_STATUS_SUCCEEDED, False), - ['submitted', 'started', 'succeeded'] - ), - ( - (TASK_STATUS_WAITING, False), - (TASK_STATUS_FAILED, False), - ['submitted', 'started', 'failed'] - ), - ( - (TASK_STATUS_WAITING, False), - (TASK_STATUS_FAILED, None), # no change to is_held - ['submitted', 'started', 'failed'] - ), - ( - (TASK_STATUS_WAITING, False), - (None, False), # no change to status - [] - ), - # only reset task outputs if not setting task to held - # https://github.com/cylc/cylc-flow/pull/2116 - ( - (TASK_STATUS_WAITING, False), - (TASK_STATUS_FAILED, True), - [] - ), - # only reset task outputs if not setting task to held - # https://github.com/cylc/cylc-flow/pull/2116 - ( - (TASK_STATUS_WAITING, False), - (TASK_STATUS_SUCCEEDED, True), - [] - ) - ] -) -def test_reset_outputs(before, after, outputs): - """Test that outputs are reset correctly on state changes.""" - tdef = TaskDef('foo', {}, 'live', '123', '123') - - orig_status, orig_is_held = before - new_status, new_is_held = after - - tstate = TaskState(tdef, '123', orig_status, orig_is_held) - assert tstate.outputs.get_completed() == [] - tstate.reset(status=new_status, is_held=new_is_held) - assert tstate.outputs.get_completed() == outputs - - def test_task_prereq_duplicates(set_cycling_type): """Test prerequisite duplicates from multiple recurrences are discarded."""