From a4c1f0d8617c8f9b745c1b902c5a2e2d7cb718c0 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 14 Feb 2024 18:49:47 +0000 Subject: [PATCH 01/25] Remove broken hard-coded usage tracking IP address. This is broken in two ways: * Hard-coding IP addresses is bad internet citizenship, especially into address blocks that are not owned by the Parsl project. If name service lookup fails, it is bad practice to fall back to a hard-coded address. * The hard-coded IP address goes into a variable that is then not used anywhere, so the hard-coded IP address functionality doesn't actually work. --- parsl/usage_tracking/usage.py | 20 ++++---------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/parsl/usage_tracking/usage.py b/parsl/usage_tracking/usage.py index b89be61b4e..c584855196 100644 --- a/parsl/usage_tracking/usage.py +++ b/parsl/usage_tracking/usage.py @@ -32,7 +32,7 @@ def run(*args, **kwargs): @async_process -def udp_messenger(domain_name: str, UDP_IP: str, UDP_PORT: int, sock_timeout: int, message: str) -> None: +def udp_messenger(domain_name: str, UDP_PORT: int, sock_timeout: int, message: str) -> None: """Send UDP messages to usage tracker asynchronously This multiprocessing based messenger was written to overcome the limitations @@ -40,7 +40,6 @@ def udp_messenger(domain_name: str, UDP_IP: str, UDP_PORT: int, sock_timeout: in Args: - domain_name (str) : Domain name string - - UDP_IP (str) : IP address YYY.YYY.YYY.YYY - UDP_PORT (int) : UDP port to send out on - sock_timeout (int) : Socket timeout """ @@ -49,15 +48,7 @@ def udp_messenger(domain_name: str, UDP_IP: str, UDP_PORT: int, sock_timeout: in try: encoded_message = bytes(message, "utf-8") - if domain_name: - try: - UDP_IP = socket.gethostbyname(domain_name) - except Exception: - # (False, "Domain lookup failed, defaulting to {0}".format(UDP_IP)) - pass - - if UDP_IP is None: - raise Exception("UDP_IP is None") + UDP_IP = socket.gethostbyname(domain_name) if UDP_PORT is None: raise Exception("UDP_PORT is None") @@ -84,7 +75,7 @@ class UsageTracker: """ - def __init__(self, dfk, ip='52.3.111.203', port=50077, + def __init__(self, dfk, port=50077, domain_name='tracking.parsl-project.org'): """Initialize usage tracking unless the user has opted-out. @@ -99,18 +90,15 @@ def __init__(self, dfk, ip='52.3.111.203', port=50077, - dfk (DFK object) : Data Flow Kernel object KWargs: - - ip (string) : IP address - port (int) : Port number, Default:50077 - domain_name (string) : Domain name, will override IP Default: tracking.parsl-project.org """ self.domain_name = domain_name - self.ip = ip # The sock timeout will only apply to UDP send and not domain resolution self.sock_timeout = 5 self.UDP_PORT = port - self.UDP_IP = None self.procs = [] self.dfk = dfk self.config = self.dfk.config @@ -188,7 +176,7 @@ def send_UDP_message(self, message: str) -> None: """Send UDP message.""" if self.tracking_enabled: try: - proc = udp_messenger(self.domain_name, self.UDP_IP, self.UDP_PORT, self.sock_timeout, message) + proc = udp_messenger(self.domain_name, self.UDP_PORT, self.sock_timeout, message) self.procs.append(proc) except Exception as e: logger.debug("Usage tracking failed: {}".format(e)) From 647fc0567e0e9aac0d54f3b31636ab44b6733a7b Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 16 Feb 2024 16:12:34 +0000 Subject: [PATCH 02/25] upgrade mypy periodically might help with 3.12 typechecking with paramspecs? (not needed for master because paramspec is too new... 3.10) --- parsl/executors/flux/executor.py | 5 ++++- parsl/utils.py | 22 +++++++++++++++------- test-requirements.txt | 2 +- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/parsl/executors/flux/executor.py b/parsl/executors/flux/executor.py index dfea11acb3..7bb81460b9 100644 --- a/parsl/executors/flux/executor.py +++ b/parsl/executors/flux/executor.py @@ -227,7 +227,10 @@ def __init__( # add a ``weakref.finalize()`` function for joining the executor thread weakref.finalize( self, - lambda x, y: x.set() or y.join(), + # can't use these here in a lambda because it gives a mypy error with mypy 1.8.0: + # mypy doesn't like None or None as a const value here. it's nto a "type error", + # more a "redundant calculation of definitely None" + lambda x, y: x.set() or y.join(), # type: ignore[func-returns-value] self._stop_event, self._submission_thread, ) diff --git a/parsl/utils.py b/parsl/utils.py index ae1e272bf6..2718a3983b 100644 --- a/parsl/utils.py +++ b/parsl/utils.py @@ -7,7 +7,7 @@ import time from contextlib import contextmanager from types import TracebackType -from typing import Any, Callable, List, Sequence, Tuple, Union, Generator, IO, AnyStr, Dict, Optional +from typing import Any, Callable, Iterator, List, Sequence, Tuple, Union, Generator, IO, Dict, Optional import typeguard from typing_extensions import Type @@ -113,15 +113,14 @@ def get_std_fname_mode( fdname: str, stdfspec: Union[os.PathLike, str, Tuple[str, str], Tuple[os.PathLike, str]] ) -> Tuple[str, str]: - import parsl.app.errors as pe if isinstance(stdfspec, (str, os.PathLike)): fname = stdfspec mode = 'a+' elif isinstance(stdfspec, tuple): - if len(stdfspec) != 2: - msg = (f"std descriptor {fdname} has incorrect tuple length " - f"{len(stdfspec)}") - raise pe.BadStdStreamFile(msg, TypeError('Bad Tuple Length')) + # mypy can detect that the tuple is length 2 + # (when upgrading from 1.5.1 to 1.8.0 and Python 3.12 but can typeguard check that, + # which is what I want this check for...?) + # typeguard (at least 4.1.5, and 2.13.3) can detect tuple length fname, mode = stdfspec return str(fname), mode @@ -135,8 +134,17 @@ def wait_for_file(path: str, seconds: int = 10) -> Generator[None, None, None]: yield +# with upgrade from mypy 1.5.1 to 1.8.0 +# parsl/utils.py:137: error: Argument 1 to "contextmanager" has +# incompatible type "Callable[[str, str, int], Generator[IO[AnyStr], None, None]]"; +# expected "Callable[[str, str, int], Iterator[IO[Never]]]" [arg-type] +# so it used to return Generator[IO[AnyStr], None, None] +# but mypy thins it should return Iterator[IO[Never]] +# except with that annotation the same mypy fails... with parsl/utils.py:142: error: +# Invalid type argument value for "IO" [type-var] + @contextmanager -def time_limited_open(path: str, mode: str, seconds: int = 1) -> Generator[IO[AnyStr], None, None]: +def time_limited_open(path: str, mode: str, seconds: int = 1) -> Iterator[IO[Any]]: with wait_for_file(path, seconds): logger.debug("wait_for_file yielded") f = open(path, mode) diff --git a/test-requirements.txt b/test-requirements.txt index 3958c45d3b..a022e20466 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -8,7 +8,7 @@ mock>=1.0.0 types-mock nbsphinx sphinx_rtd_theme -mypy==1.5.1 +mypy==1.8.0 types-python-dateutil types-requests types-paramiko From add3210ba3a24ab7d888080a8249143f0007fd15 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 31 Jan 2024 13:56:23 +0000 Subject: [PATCH 03/25] Remove unused codepath from HighThroughputExecutor scale_in The codepath would be used when scale_in is called with: force=False max_idletime=None and would pick blocks from the (weirdly sorted - see issue #2120) list of blocks which are currently idle. This PR removes that unused codepath, merging the choice of forced/non-forced scale-in and idle time specification into a single parameter that indicates "do not scale in blocks that have not been idle this long". --- parsl/executors/high_throughput/executor.py | 36 +++++++++------------ parsl/jobs/job_status_poller.py | 8 +++-- parsl/jobs/strategy.py | 4 +-- 3 files changed, 22 insertions(+), 26 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index d76a1896f2..a81b3fb6d6 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -667,7 +667,7 @@ def create_monitoring_info(self, status): def workers_per_node(self) -> Union[int, float]: return self._workers_per_node - def scale_in(self, blocks, force=True, max_idletime=None): + def scale_in(self, blocks, max_idletime=None): """Scale in the number of active blocks by specified amount. The scale in method here is very rude. It doesn't give the workers @@ -680,18 +680,14 @@ def scale_in(self, blocks, force=True, max_idletime=None): blocks : int Number of blocks to terminate and scale_in by - force : Bool - Used along with blocks to indicate whether blocks should be terminated by force. - - When force = True, we will kill blocks regardless of the blocks being busy + max_idletime: float + A time to indicate how long a block should be idle to be a + candidate for scaling in. - When force = False, only idle blocks will be terminated. If the - number of idle blocks < ``blocks``, then fewer than ``blocks`` - blocks will be terminated. + If None then blocks will be force scaled in even if they are busy. - max_idletime: float - A time to indicate how long a block can be idle. - Used along with force = False to kill blocks that have been idle for that long. + If a float, then only idle blocks will be terminated, which may be less than + the requested number. Returns ------- @@ -711,18 +707,16 @@ def scale_in(self, blocks, force=True, max_idletime=None): sorted_blocks = sorted(block_info.items(), key=lambda item: (item[1][1], item[1][0])) logger.debug(f"Scale in selecting from {len(sorted_blocks)} blocks") - if force is True: + if max_idletime is None: block_ids_to_kill = [x[0] for x in sorted_blocks[:blocks]] else: - if not max_idletime: - block_ids_to_kill = [x[0] for x in sorted_blocks if x[1][0] == 0][:blocks] - else: - block_ids_to_kill = [] - for x in sorted_blocks: - if x[1][1] > max_idletime and x[1][0] == 0: - block_ids_to_kill.append(x[0]) - if len(block_ids_to_kill) == blocks: - break + block_ids_to_kill = [] + for x in sorted_blocks: + if x[1][1] > max_idletime and x[1][0] == 0: + block_ids_to_kill.append(x[0]) + if len(block_ids_to_kill) == blocks: + break + logger.debug("Selected idle block ids to kill: {}".format( block_ids_to_kill)) if len(block_ids_to_kill) < blocks: diff --git a/parsl/jobs/job_status_poller.py b/parsl/jobs/job_status_poller.py index 4a9132b8a9..50ea3a6332 100644 --- a/parsl/jobs/job_status_poller.py +++ b/parsl/jobs/job_status_poller.py @@ -72,11 +72,13 @@ def status(self) -> Dict[str, JobStatus]: def executor(self) -> BlockProviderExecutor: return self._executor - def scale_in(self, n, force=True, max_idletime=None): - if force and not max_idletime: + def scale_in(self, n, max_idletime=None): + + # this is a HighThroughputExecutor-specific interface violation + if max_idletime is None: block_ids = self._executor.scale_in(n) else: - block_ids = self._executor.scale_in(n, force=force, max_idletime=max_idletime) + block_ids = self._executor.scale_in(n, max_idletime=max_idletime) if block_ids is not None: new_status = {} for block_id in block_ids: diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index beb5e50249..b396d43e37 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -288,8 +288,8 @@ def _general_strategy(self, status_list, *, strategy_type): excess_slots = math.ceil(active_slots - (active_tasks * parallelism)) excess_blocks = math.ceil(float(excess_slots) / (tasks_per_node * nodes_per_block)) excess_blocks = min(excess_blocks, active_blocks - min_blocks) - logger.debug(f"Requesting scaling in by {excess_blocks} blocks") - exec_status.scale_in(excess_blocks, force=False, max_idletime=self.max_idletime) + logger.debug(f"Requesting scaling in by {excess_blocks} blocks with idle time {self.max_idletime}s") + exec_status.scale_in(excess_blocks, max_idletime=self.max_idletime) else: logger.error("This strategy does not support scaling in except for HighThroughputExecutor - taking no action") else: From c71b0a5d27c6889da7f4e70a75a1c675c045008c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 14 Feb 2024 11:19:57 +0000 Subject: [PATCH 04/25] this test only tests the behaviour from simple strategy, so turn it into a simple strategy test a different PR will introduce a variation that tests htex_auto_scale functionality --- parsl/tests/test_scaling/test_scale_down.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/tests/test_scaling/test_scale_down.py b/parsl/tests/test_scaling/test_scale_down.py index 6ebe441760..d606b987d0 100644 --- a/parsl/tests/test_scaling/test_scale_down.py +++ b/parsl/tests/test_scaling/test_scale_down.py @@ -39,7 +39,7 @@ def local_config(): ) ], max_idletime=0.5, - strategy='htex_auto_scale', + strategy='simple', ) From 5fbdfd64a448169a1d483fa1bb364ecc43b7c8f0 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 14 Feb 2024 10:46:29 +0000 Subject: [PATCH 05/25] test partial scaling-in (the scaling in the comes from one block being idle, not from the whole executor being idle) this is quite awkward ot test. what can we do? the current test has min_blocks 2, which means if we launch a single task (pretty much all we can), we'll not be able to distinguish min_blocks scaling from single task scaling. so a separate test perhaps with min_blocks 0? --- .../test_scale_down_htex_auto_scale.py | 140 ++++++++++++++++++ 1 file changed, 140 insertions(+) create mode 100644 parsl/tests/test_scaling/test_scale_down_htex_auto_scale.py diff --git a/parsl/tests/test_scaling/test_scale_down_htex_auto_scale.py b/parsl/tests/test_scaling/test_scale_down_htex_auto_scale.py new file mode 100644 index 0000000000..b7551251d8 --- /dev/null +++ b/parsl/tests/test_scaling/test_scale_down_htex_auto_scale.py @@ -0,0 +1,140 @@ +import logging +import time + +import pytest + +import parsl + +from parsl import File, python_app +from parsl.providers import LocalProvider +from parsl.channels import LocalChannel +from parsl.launchers import SingleNodeLauncher +from parsl.config import Config +from parsl.executors import HighThroughputExecutor + +logger = logging.getLogger(__name__) + +_max_blocks = 5 +_min_blocks = 0 + + +def local_config(): + return Config( + executors=[ + HighThroughputExecutor( + heartbeat_period=1, + heartbeat_threshold=2, + poll_period=100, + label="htex_local", + address="127.0.0.1", + max_workers=1, + encrypted=True, + provider=LocalProvider( + channel=LocalChannel(), + init_blocks=0, + max_blocks=_max_blocks, + min_blocks=_min_blocks, + launcher=SingleNodeLauncher(), + ), + ) + ], + max_idletime=0.5, + strategy='htex_auto_scale', + ) + + +@python_app +def sleep_TODO_events(): + import time + time.sleep(20) + + +@python_app +def waiting_app(ident: int, inputs=()): + import pathlib + import time + + # Approximate an Event by writing to files; the test logic will poll this file + with open(inputs[0], "a") as f: + f.write(f"Ready: {ident}\n") + + # Similarly, use Event approximation (file check!) by polling. + may_finish_file = pathlib.Path(inputs[1]) + while not may_finish_file.exists(): + time.sleep(0.01) + + +# see issue #1885 for details of failures of this test. +# at the time of issue #1885 this test was failing frequently +# in CI. +@pytest.mark.local +def test_scale_out(tmpd_cwd, try_assert): + dfk = parsl.dfk() + + num_managers = len(dfk.executors['htex_local'].connected_managers()) + + assert num_managers == 0, "Expected 0 managers at start" + assert dfk.executors['htex_local'].outstanding == 0, "Expected 0 tasks at start" + + ntasks = _max_blocks * 2 + ready_path = tmpd_cwd / "workers_ready" + finish_path = tmpd_cwd / "workers_may_continue" + ready_path.touch() + inputs = [File(str(ready_path)), File(str(finish_path))] + + futs = [waiting_app(i, inputs=inputs) for i in range(ntasks)] + + while ready_path.read_text().count("\n") < _max_blocks: + time.sleep(0.5) + + assert len(dfk.executors['htex_local'].connected_managers()) == _max_blocks + + finish_path.touch() # Approximation of Event, via files + [x.result() for x in futs] + + assert dfk.executors['htex_local'].outstanding == 0 + + # now we can launch one "long" task - and what should happen is that the connected_managers count "eventually" (?) converges to 1 and stays there. + + fut = sleep_TODO_events() + + def check_one_block(): + return len(dfk.executors['htex_local'].connected_managers()) == 1 + + try_assert( + check_one_block, + fail_msg="Expected 1 managers during a single long task", + timeout_ms=15000, + ) + + # the task should not have finished by the time we end up with 1 manager + assert not fut.done() + + # but now I want to test that we don't immediately converge to + # min_blocks but that, for example, there is one strategy pass or + # something like that? + + # this will give some strategy passes... is there an event driven way + # of doing this? + + time.sleep(10) + + # this interacts with the sleep in the task... it needs to be the right + # size to still be inside the task even after we've waited for the + # partial scale down to happen, so that the following assertion can fire + # properly + + assert check_one_block() + + fut.result() + + # now we should expect min_blocks scale down + + def check_min_blocks(): + return len(dfk.executors['htex_local'].connected_managers()) == _min_blocks + + try_assert( + check_min_blocks, + fail_msg=f"Expected {_min_blocks} managers when no tasks (min_blocks)", + timeout_ms=15000, + ) From ca2d8892a91538b15246d7c2a50c6e27b3b7178d Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 31 Jan 2024 14:43:00 +0000 Subject: [PATCH 06/25] uses human readable names not numeric indices for scale in code --- parsl/executors/high_throughput/executor.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index a81b3fb6d6..1dc6e6fc02 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -695,24 +695,24 @@ def scale_in(self, blocks, max_idletime=None): """ logger.debug(f"Scale in called, blocks={blocks}") managers = self.connected_managers() - block_info = {} # block id -> list( tasks, idle duration ) + block_info = {} # block id -> dict( "tasks", "idle" time in s) for manager in managers: if not manager['active']: continue b_id = manager['block_id'] if b_id not in block_info: - block_info[b_id] = [0, float('inf')] - block_info[b_id][0] += manager['tasks'] - block_info[b_id][1] = min(block_info[b_id][1], manager['idle_duration']) + block_info[b_id] = {'tasks': 0, 'idle': float('inf')} + block_info[b_id]['tasks'] += manager['tasks'] + block_info[b_id]['idle'] = min(block_info[b_id]['idle'], manager['idle_duration']) - sorted_blocks = sorted(block_info.items(), key=lambda item: (item[1][1], item[1][0])) + sorted_blocks = sorted(block_info.items(), key=lambda item: (item[1]['idle'], item[1]['tasks'])) logger.debug(f"Scale in selecting from {len(sorted_blocks)} blocks") if max_idletime is None: block_ids_to_kill = [x[0] for x in sorted_blocks[:blocks]] else: block_ids_to_kill = [] for x in sorted_blocks: - if x[1][1] > max_idletime and x[1][0] == 0: + if x[1]['idle'] > max_idletime and x[1]['tasks'] == 0: block_ids_to_kill.append(x[0]) if len(block_ids_to_kill) == blocks: break From 26131053a5fe90d7d96617823df724952347e59d Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 31 Jan 2024 13:25:03 +0000 Subject: [PATCH 07/25] make scale in pick longest idle blocks in preference to least idle blocks --- parsl/executors/high_throughput/executor.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index 1dc6e6fc02..8e5b90ee5d 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -705,7 +705,15 @@ def scale_in(self, blocks, max_idletime=None): block_info[b_id]['tasks'] += manager['tasks'] block_info[b_id]['idle'] = min(block_info[b_id]['idle'], manager['idle_duration']) - sorted_blocks = sorted(block_info.items(), key=lambda item: (item[1]['idle'], item[1]['tasks'])) + # This list should be sorted so that the most preferable blocks to + # scale in are at the start of the list. + + # The most preferred blocks are blocks with the longest idle time, so + # the key here starts with the negative of idle time, making longer + # idle times be lower numbers, so sorting first. + + sorted_blocks = sorted(block_info.items(), key=lambda item: (-item[1]['idle'], item[1]['tasks'])) + logger.debug(f"Scale in selecting from {len(sorted_blocks)} blocks") if max_idletime is None: block_ids_to_kill = [x[0] for x in sorted_blocks[:blocks]] From e0e28d51cd3025377be90320ebc7bdf526aed3bd Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 31 Jan 2024 15:05:28 +0000 Subject: [PATCH 08/25] scale-in should consider all known blocks, not just blocks that have at least one manager registered how to test this? make an htex with init_blocks=min_blocks=1 block and a worker command that does something like sleep forever (it needs to not fail, because failed blocks wont' get scaled in). and then shutdown without any tasks. in the old behaviour that unregistered block will not be scaled in, i think, but in the new behaviour it should be. --- parsl/executors/high_throughput/executor.py | 23 +++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index 8e5b90ee5d..e98f7db844 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -15,7 +15,7 @@ from parsl.serialize import pack_res_spec_apply_message, deserialize from parsl.serialize.errors import SerializationError, DeserializationError from parsl.app.errors import RemoteExceptionWrapper -from parsl.jobs.states import JobStatus, JobState +from parsl.jobs.states import JobStatus, JobState, TERMINAL_STATES from parsl.executors.high_throughput import zmq_pipes from parsl.executors.high_throughput import interchange from parsl.executors.errors import ( @@ -694,8 +694,27 @@ def scale_in(self, blocks, max_idletime=None): List of block IDs scaled in """ logger.debug(f"Scale in called, blocks={blocks}") - managers = self.connected_managers() block_info = {} # block id -> dict( "tasks", "idle" time in s) + + # First populate the list of blocks with blocks know to the provider + # mechanism. Then update that list with information from the + # interchange which will include idle times and task counts. Blocks + # with no interchange registration (for example because they are still + # in a batch queue) will be regarded as idle for a long time (so will + # be preferred when killing most idle blocks) in preference to any + # blocks which have already registered. + + provider_blocks = self.status() + for (b_id, job_status) in provider_blocks.items(): + + # skip blocks that have already finished + if job_status.state in TERMINAL_STATES: + continue + + if b_id not in block_info: + block_info[b_id] = {"tasks": 0, "idle": float('inf')} + + managers = self.connected_managers() for manager in managers: if not manager['active']: continue From 795762856e3f4dcd26be3bd713e408c2cc1a513d Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 20 Nov 2023 13:17:45 +0000 Subject: [PATCH 09/25] fiddle with submiterror * add retcode * use "job" not "task" in parsl: "job" refers to a batch system job / aka a block, and "task" refers to a user-supplied piece of code submitted to the DFK that is run by an executor --- parsl/providers/errors.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/parsl/providers/errors.py b/parsl/providers/errors.py index 03573d3b55..1290d07806 100644 --- a/parsl/providers/errors.py +++ b/parsl/providers/errors.py @@ -1,3 +1,5 @@ +from typing import Optional + from parsl.errors import ParslError import warnings @@ -51,7 +53,7 @@ class SubmitException(ExecutionProviderException): '''Raised by the submit() method of a provider if there is an error in launching a job. ''' - def __init__(self, job_name, message, stdout=None, stderr=None, retcode=None): + def __init__(self, job_name: str, message: str, stdout: Optional[str] = None, stderr: Optional[str] = None, retcode: Optional[int] = None): self.job_name = job_name self.message = message self.stdout = stdout From 2e898da2f9f8eef5fb517668934cf3e96eaa131c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 15 Nov 2023 13:49:36 +0000 Subject: [PATCH 10/25] make storage access a sequence for type-checker goodness remove typechecker note that typeguard appears to have evolved beyond, although in a way that makes me uncomfortable? make default_staging a sequence which can help guard against mutating it --- parsl/data_provider/data_manager.py | 6 +++--- parsl/executors/base.py | 17 ++++++----------- parsl/executors/high_throughput/executor.py | 2 +- parsl/executors/taskvine/executor.py | 6 +++--- parsl/tests/configs/local_threads_globus.py | 2 +- 5 files changed, 14 insertions(+), 19 deletions(-) diff --git a/parsl/data_provider/data_manager.py b/parsl/data_provider/data_manager.py index 3fa9204b5e..0a060b7f08 100644 --- a/parsl/data_provider/data_manager.py +++ b/parsl/data_provider/data_manager.py @@ -1,6 +1,6 @@ import logging from concurrent.futures import Future -from typing import Any, Callable, List, Optional, TYPE_CHECKING +from typing import Any, Callable, Optional, Sequence, TYPE_CHECKING from parsl.app.futures import DataFuture from parsl.data_provider.files import File @@ -16,7 +16,7 @@ # these will be shared between all executors that do not explicitly # override, so should not contain executor-specific state -default_staging: List[Staging] +default_staging: Sequence[Staging] default_staging = [NoOpFileStaging(), FTPSeparateTaskStaging(), HTTPSeparateTaskStaging()] @@ -39,7 +39,7 @@ def replace_task_stage_out(self, file: File, func: Callable, executor: str) -> C """This will give staging providers the chance to wrap (or replace entirely!) the task function.""" executor_obj = self.dfk.executors[executor] if hasattr(executor_obj, "storage_access") and executor_obj.storage_access is not None: - storage_access = executor_obj.storage_access # type: List[Staging] + storage_access: Sequence[Staging] = executor_obj.storage_access else: storage_access = default_staging diff --git a/parsl/executors/base.py b/parsl/executors/base.py index efc3375e6a..3a1e941fb0 100644 --- a/parsl/executors/base.py +++ b/parsl/executors/base.py @@ -29,17 +29,12 @@ class ParslExecutor(metaclass=ABCMeta): An executor may optionally expose: - storage_access: List[parsl.data_provider.staging.Staging] - a list of staging - providers that will be used for file staging. In the absence of this - attribute, or if this attribute is `None`, then a default value of - ``parsl.data_provider.staging.default_staging`` will be used by the - staging code. - - Typechecker note: Ideally storage_access would be declared on executor - __init__ methods as List[Staging] - however, lists are by default - invariant, not co-variant, and it looks like @typeguard cannot be - persuaded otherwise. So if you're implementing an executor and want to - @typeguard the constructor, you'll have to use List[Any] here. + storage_access: Optional[Sequence[parsl.data_provider.staging.Staging]] + A sequence of staging providers that will be used for file + staging. In the absence of this attribute, or if this + attribute is `None`, then a default value of + ``parsl.data_provider.staging.default_staging`` + will be used by the staging code. """ label: str = "undefined" diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index e98f7db844..e0574a844a 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -223,7 +223,7 @@ def __init__(self, worker_ports: Optional[Tuple[int, int]] = None, worker_port_range: Optional[Tuple[int, int]] = (54000, 55000), interchange_port_range: Optional[Tuple[int, int]] = (55000, 56000), - storage_access: Optional[List[Staging]] = None, + storage_access: Optional[Sequence[Staging]] = None, working_dir: Optional[str] = None, worker_debug: bool = False, cores_per_worker: float = 1.0, diff --git a/parsl/executors/taskvine/executor.py b/parsl/executors/taskvine/executor.py index cb958ff624..8d5a83df04 100644 --- a/parsl/executors/taskvine/executor.py +++ b/parsl/executors/taskvine/executor.py @@ -17,7 +17,7 @@ import itertools import uuid from concurrent.futures import Future -from typing import List, Optional, Union, Literal +from typing import Optional, Sequence, Union, Literal # Import Parsl constructs import parsl.utils as putils @@ -91,7 +91,7 @@ class TaskVineExecutor(BlockProviderExecutor, putils.RepresentationMixin): The Parsl provider that will spawn worker processes. Default to spawning one local vine worker process. - storage_access: List[Staging] + storage_access: Sequence[Staging] Define Parsl file staging providers for this executor. Default is None. """ @@ -106,7 +106,7 @@ def __init__(self, manager_config: TaskVineManagerConfig = TaskVineManagerConfig(), factory_config: TaskVineFactoryConfig = TaskVineFactoryConfig(), provider: Optional[ExecutionProvider] = LocalProvider(init_blocks=1), - storage_access: Optional[List[Staging]] = None): + storage_access: Optional[Sequence[Staging]] = None): # Set worker launch option for this executor if worker_launch_method == 'factory' or worker_launch_method == 'manual': diff --git a/parsl/tests/configs/local_threads_globus.py b/parsl/tests/configs/local_threads_globus.py index 21a85c1c83..74ed7f1478 100644 --- a/parsl/tests/configs/local_threads_globus.py +++ b/parsl/tests/configs/local_threads_globus.py @@ -10,7 +10,7 @@ # (i.e., user_opts['swan']['username'] -> 'your_username') from .user_opts import user_opts -storage_access = default_staging + [GlobusStaging( +storage_access = list(default_staging) + [GlobusStaging( endpoint_uuid=user_opts['globus']['endpoint'], endpoint_path=user_opts['globus']['path'] )] From d002313d1fe6fc6ce5576fa91964fce7a4c8950e Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Sat, 14 Oct 2023 17:13:53 +0000 Subject: [PATCH 11/25] fiddling with wqtaskfailrue exception --- parsl/executors/workqueue/executor.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/parsl/executors/workqueue/executor.py b/parsl/executors/workqueue/executor.py index 64a8268e8f..14e5e5c371 100644 --- a/parsl/executors/workqueue/executor.py +++ b/parsl/executors/workqueue/executor.py @@ -751,6 +751,8 @@ def _collect_work_queue_results(self): # If there are no results, then the task failed according to one of # work queue modes, such as resource exhaustion. ex = WorkQueueTaskFailure(task_report.reason, None) + if task_report.result is not None: + ex.__cause__ = task_report.result future.set_exception(ex) finally: logger.debug("Marking all outstanding tasks as failed") @@ -1008,7 +1010,7 @@ def _explain_work_queue_result(wq_task): if wq_result == wq.WORK_QUEUE_RESULT_SUCCESS: reason += "succesful execution with exit code {}".format(wq_task.return_status) elif wq_result == wq.WORK_QUEUE_RESULT_OUTPUT_MISSING: - reason += "The result file was not transfered from the worker.\n" + reason += "A result file was not transfered from the worker.\n" reason += "This usually means that there is a problem with the python setup,\n" reason += "or the wrapper that executes the function." reason += "\nTrace:\n" + str(wq_task.output) From 5e0f764e852dcb73bf2e12c4e6eed4458a43d730 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 15 Sep 2023 13:58:03 +0000 Subject: [PATCH 12/25] connected workers is used in strategy (and exposed as part of executor interface) only for printing a status line. this is awkward to statically typecheck, so remove it - and its awkward to typecheck because it's awkward to have this in the executor interface, or even in the block provider executor interface? an alternative is to have a HasConnectedWorkers interface type that gives the connected_workers attribute, but that's a lot of lines of code for something that is probably misabstracted to strategy anyway The htex connected workers info is useful, but that should be displayed elsewhere the now-unused/reused connected_workers property is made back into a method not a property, because it makes an RPC callout (so subject to hangs from a broken interchange), which is unclear when reading connected_workers as a property. TODO: put that connected workers logging into htex itself, rather than in strategy - probably into the interchange log? there's already a place in the interchange log - there's a debug message with manager counts every 10ms - could make that less frequent and at INFO level, rather tha nevery 10ms? --- parsl/executors/high_throughput/executor.py | 3 +-- parsl/jobs/strategy.py | 8 ++------ parsl/tests/manual_tests/test_memory_limits.py | 2 +- parsl/tests/scaling_tests/test_scale.py | 6 +++--- parsl/tests/site_tests/test_site.py | 2 +- 5 files changed, 8 insertions(+), 13 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index e0574a844a..df0a0fd1f9 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -565,8 +565,7 @@ def outstanding(self) -> int: and managers""" return self.command_client.run("OUTSTANDING_C") - @property - def connected_workers(self) -> int: + def _connected_workers(self) -> int: """Returns the count of workers across all connected managers""" return self.command_client.run("WORKERS") diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index b396d43e37..9bf961cc8c 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -204,12 +204,8 @@ def _general_strategy(self, status_list, *, strategy_type): logger.debug(f"Slot ratio calculation: active_slots = {active_slots}, active_tasks = {active_tasks}") - if hasattr(executor, 'connected_workers'): - logger.debug('Executor {} has {} active tasks, {}/{} running/pending blocks, and {} connected workers'.format( - label, active_tasks, running, pending, executor.connected_workers)) - else: - logger.debug('Executor {} has {} active tasks and {}/{} running/pending blocks'.format( - label, active_tasks, running, pending)) + logger.debug('Executor {} has {} active tasks and {}/{} running/pending blocks'.format( + label, active_tasks, running, pending)) # reset idle timer if executor has active tasks diff --git a/parsl/tests/manual_tests/test_memory_limits.py b/parsl/tests/manual_tests/test_memory_limits.py index be4507808f..d66f207874 100644 --- a/parsl/tests/manual_tests/test_memory_limits.py +++ b/parsl/tests/manual_tests/test_memory_limits.py @@ -55,7 +55,7 @@ def test_simple(mem_per_worker): # Prime a worker double(5).result() dfk = parsl.dfk() - connected = dfk.executors['htex_local'].connected_workers + connected = dfk.executors['htex_local']._connected_workers() print("Connected : ", connected) assert expected_workers == connected, "Expected {} workers, instead got {} workers".format(expected_workers, connected) diff --git a/parsl/tests/scaling_tests/test_scale.py b/parsl/tests/scaling_tests/test_scale.py index f024051ee0..ef4befd0b1 100755 --- a/parsl/tests/scaling_tests/test_scale.py +++ b/parsl/tests/scaling_tests/test_scale.py @@ -78,19 +78,19 @@ def test_platform(n=2, sleep=1): print("Trying to get executor : ", name) print("Executor : ", dfk.executors[name]) - print("Connected : ", dfk.executors[name].connected_workers) + print("Connected : ", dfk.executors[name]._connected_workers()) print("Outstanding : ", dfk.executors[name].outstanding) d = [] for i in range(0, n): x = platform(sleep=sleep) d.append(x) - print("Connected : ", dfk.executors[name].connected_workers) + print("Connected : ", dfk.executors[name]._connected_workers()) print("Outstanding : ", dfk.executors[name].outstanding) print(set([i.result()for i in d])) - print("Connected : ", dfk.executors[name].connected_workers) + print("Connected : ", dfk.executors[name]._connected_workers()) print("Outstanding : ", dfk.executors[name].outstanding) return True diff --git a/parsl/tests/site_tests/test_site.py b/parsl/tests/site_tests/test_site.py index 991a03fe46..a8b99f4e17 100644 --- a/parsl/tests/site_tests/test_site.py +++ b/parsl/tests/site_tests/test_site.py @@ -33,7 +33,7 @@ def test_platform(n=2, sleep_dur=10): print([i.result() for i in x]) print("Executor : ", dfk.executors[name]) - print("Connected : ", dfk.executors[name].connected_workers) + print("Connected : ", dfk.executors[name]._connected_workers()) print("Outstanding : ", dfk.executors[name].outstanding) d = [] From 4fe55c4dc0d831351bf0c7fe41fbc2aa35963188 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 29 Jun 2023 05:22:56 +0000 Subject: [PATCH 13/25] serialization performance test client serialise many times a few different structures deserialise many times a few different structures potentially do some serious stats (maybe theres a library to do that like criterion in haskell already? google for python microbenchmarking...) when deserializing the same list object, it's 10x slower... in practice, one thing I'd expect that caching to be relevant is repeated deserialization of function objects in htex executions (rather than caching of values - although that would sometimes be true too...) a more nuanced approach might allowed caching on the deserialisation side only for known-whitelisted types (such as function - but not sure about partial?) - which is the 'callable' distinction differently (and perhaps better?) however, as these tmings are don in the 20ns range, this may not really matter too much for htex... --- parsl/benchmark/serialization.py | 73 ++++++++++++++++++++++++++ parsl/benchmark/serialization_large.py | 35 ++++++++++++ 2 files changed, 108 insertions(+) create mode 100644 parsl/benchmark/serialization.py create mode 100644 parsl/benchmark/serialization_large.py diff --git a/parsl/benchmark/serialization.py b/parsl/benchmark/serialization.py new file mode 100644 index 0000000000..f67ba3a467 --- /dev/null +++ b/parsl/benchmark/serialization.py @@ -0,0 +1,73 @@ +import time +import parsl.serialize as s +import dill +import pickle + + +def prof(serializer, deserializer): + tot_iters = 100000 + + t_start = time.time() + for _ in range(0, tot_iters): + pass + t_end = time.time() + + print(f"time for null loop: {t_end - t_start}s") + print(f"time per iteration: {(t_end - t_start) / tot_iters}s") + + t_start = time.time() + for _ in range(0, tot_iters): + serializer(7) + t_end = time.time() + + print(f"time for serialize 7 loop: {t_end - t_start}s") + print(f"time per iteration: {(t_end - t_start) / tot_iters}s") + + t_start = time.time() + for n in range(0, tot_iters): + serializer(n) + t_end = time.time() + + print(f"time for serialize all ns loop: {t_end - t_start}s") + print(f"time per iteration: {(t_end - t_start) / tot_iters}s") + + t_start = time.time() + for _ in range(0, tot_iters): + serializer("hello") + t_end = time.time() + + print(f"time for serialize hello loop: {t_end - t_start}s") + print(f"time per iteration: {(t_end - t_start) / tot_iters}s") + + def f(): + """This is a test function to be serialized""" + return 100 + + try: + t_start = time.time() + for _ in range(0, tot_iters): + serializer(f) + t_end = time.time() + + print(f"time for serialize f loop: {t_end - t_start}s") + print(f"time per iteration: {(t_end - t_start) / tot_iters}s") + except Exception as e: + print(f"Exception in serialize f loop: {e}") + + t_start = time.time() + for n in range(0, tot_iters): + deserializer(serializer(n)) + t_end = time.time() + + print(f"time for serialize/deserialize all ns loop: {t_end - t_start}s") + print(f"time per iteration: {(t_end - t_start) / tot_iters}s") + + +if __name__ == "__main__": + print("parsl serialization benchmark") + print("parsl.serialize:") + prof(s.serialize, s.deserialize) + print("dill.dumps:") + prof(dill.dumps, dill.loads) + print("pickle.dumps:") + prof(pickle.dumps, pickle.loads) diff --git a/parsl/benchmark/serialization_large.py b/parsl/benchmark/serialization_large.py new file mode 100644 index 0000000000..912706f5b7 --- /dev/null +++ b/parsl/benchmark/serialization_large.py @@ -0,0 +1,35 @@ +# benchmarking of large serialization values + +import time +import random + +from parsl.serialize import serialize as serializer +from parsl.serialize import deserialize as deserializer + +iterations = 100 + +for large_count in [10, 100, 1000, 10000, 100000, 1000000, 10000000]: + + serialized_v = {} + for n in range(0, iterations): + large_v = [random.randint(0, 1000000) for _ in range(large_count)] + serialized_v[n] = serializer(large_v) + + print("") + print("--") + + print("test using same list object every time:") + print(f"large_count = {large_count}") + + t_start = time.time() + for n in range(0, iterations): + deserializer(serialized_v[n]) + t_end = time.time() + + print(f"time for deserialize all ns loop: {t_end - t_start}s") + + t_per_iter = (t_end - t_start) / iterations + print(f"time per iteration: {t_per_iter}s") + + t_per_element = t_per_iter / large_count + print(f"time per list element: {t_per_element}s") From 0b4d9b1ff7636f42064afb36e8ca67809bd7d7a9 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 16 May 2023 11:16:49 +0000 Subject: [PATCH 14/25] Prohibit adding duplicate-labelled executors The user-facing DFK add_executors call will raise a ConfigurationError, but the internally used Strategy.add_executors call performs this check only as an assertion, as it is not user facing. This is follow-on work from PR2712, which removes duplicate adding of executors. --- parsl/dataflow/dflow.py | 3 +++ parsl/jobs/strategy.py | 1 + 2 files changed, 4 insertions(+) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index 7df58a6217..f1e3101163 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -1112,6 +1112,9 @@ def _create_remote_dirs_over_channel(self, provider: ExecutionProvider, channel: def add_executors(self, executors): for executor in executors: + if executor.label in self.executors: + raise ConfigurationError("Executor {executor.label} already added") + executor.run_id = self.run_id executor.run_dir = self.run_dir executor.hub_address = self.hub_address diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index 9bf961cc8c..c9cd3c407e 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -144,6 +144,7 @@ def __init__(self, *, strategy: Optional[str], max_idletime: float) -> None: def add_executors(self, executors: Sequence[ParslExecutor]) -> None: for executor in executors: + assert executor.label not in self.executors self.executors[executor.label] = {'idle_since': None} def _strategy_noop(self, status: List[jsp.PollItem]) -> None: From e8683b60fb1b005a1c368df383041ebfb1c505d9 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 10 May 2023 07:06:27 +0000 Subject: [PATCH 15/25] Add type annotations to common test configurations This should get more type checking to happen inside the test suite. It reveals a number of mypy failures that come from broken (disabled) tests and which need fixing. Perhaps it could drive actually fixing some of those tests? --- parsl/tests/configs/htex_local.py | 2 +- parsl/tests/configs/htex_local_alternate.py | 2 +- parsl/tests/configs/local_threads.py | 2 +- parsl/tests/configs/local_threads_http_in_task.py | 2 +- parsl/tests/configs/workqueue_ex.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/parsl/tests/configs/htex_local.py b/parsl/tests/configs/htex_local.py index 0f09fce9dc..92d86d7ac1 100644 --- a/parsl/tests/configs/htex_local.py +++ b/parsl/tests/configs/htex_local.py @@ -6,7 +6,7 @@ from parsl.executors import HighThroughputExecutor -def fresh_config(): +def fresh_config() -> Config: return Config( executors=[ HighThroughputExecutor( diff --git a/parsl/tests/configs/htex_local_alternate.py b/parsl/tests/configs/htex_local_alternate.py index 2598a91b85..4093d9a54d 100644 --- a/parsl/tests/configs/htex_local_alternate.py +++ b/parsl/tests/configs/htex_local_alternate.py @@ -35,7 +35,7 @@ working_dir = os.getcwd() + "/" + "test_htex_alternate" -def fresh_config(): +def fresh_config() -> Config: return Config( executors=[ HighThroughputExecutor( diff --git a/parsl/tests/configs/local_threads.py b/parsl/tests/configs/local_threads.py index 4de2a7bd38..089cedb285 100644 --- a/parsl/tests/configs/local_threads.py +++ b/parsl/tests/configs/local_threads.py @@ -2,7 +2,7 @@ from parsl.executors.threads import ThreadPoolExecutor -def fresh_config(): +def fresh_config() -> Config: return Config( executors=[ThreadPoolExecutor()], ) diff --git a/parsl/tests/configs/local_threads_http_in_task.py b/parsl/tests/configs/local_threads_http_in_task.py index 71f718a5e4..d4f8bccc15 100644 --- a/parsl/tests/configs/local_threads_http_in_task.py +++ b/parsl/tests/configs/local_threads_http_in_task.py @@ -4,7 +4,7 @@ from parsl.executors.threads import ThreadPoolExecutor -def fresh_config(): +def fresh_config() -> Config: return Config( executors=[ ThreadPoolExecutor( diff --git a/parsl/tests/configs/workqueue_ex.py b/parsl/tests/configs/workqueue_ex.py index c64be32295..000182458c 100644 --- a/parsl/tests/configs/workqueue_ex.py +++ b/parsl/tests/configs/workqueue_ex.py @@ -6,7 +6,7 @@ from parsl.data_provider.file_noop import NoOpFileStaging -def fresh_config(): +def fresh_config() -> Config: return Config(executors=[WorkQueueExecutor(port=9000, coprocess=True, storage_access=[FTPInTaskStaging(), HTTPInTaskStaging(), NoOpFileStaging()])]) From c1bbb8e9bc96d3efbd9055621a5d51cae715e7df Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 1 May 2023 14:30:48 +0000 Subject: [PATCH 16/25] htex doesn't shut down its queue management thread at shutdown if running many htexes in sequence in a process, this results in a buildup of htex threads churning away i have some suspicion that this causes some hangs in CI... but also, I don't want these threads running in a many-sequential-DFK environment for general threads vs multiprocessing reliability this patch was written on the assumption that the timeout parameter caused the checking loop to repeatedy iterate so as to eventually discover is_alive is set to false; however, turns out that parameter is not implemented (see PR #2673) so I'll have to implement this in some different way --- parsl/executors/high_throughput/executor.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index df0a0fd1f9..1397f7513d 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -428,6 +428,7 @@ def _queue_management_worker(self): while not self.bad_state_is_set: try: + logger.debug("Waiting for an incoming_q message") msgs = self.incoming_q.get() except IOError as e: @@ -469,7 +470,12 @@ def _queue_management_worker(self): if 'result' in msg: result = deserialize(msg['result']) + # ^ if this raises an exception, queue management worker fails and + # parsl hangs. this is more relevant now allowing user-pluggable + # serialization and so user code exceptions can be raised here. + logger.debug("Setting result in future") task_fut.set_result(result) + logger.debug("Done setting result in future") elif 'exception' in msg: try: @@ -479,9 +485,13 @@ def _queue_management_worker(self): try: s.reraise() except Exception as e: + logger.debug("Setting exception in future") task_fut.set_exception(e) + logger.debug("Done setting exception in future") elif isinstance(s, Exception): + logger.debug("Setting exception in future") task_fut.set_exception(s) + logger.debug("Done setting exception in future") else: raise ValueError("Unknown exception-like type received: {}".format(type(s))) except Exception as e: @@ -794,4 +804,8 @@ def shutdown(self): logger.info("Attempting HighThroughputExecutor shutdown") self.interchange_proc.terminate() + + # this join isn't going to work because the queue management thread + # blocks on zmq read and doesn't discover is_alive is false... + # self._queue_management_thread.join() logger.info("Finished HighThroughputExecutor shutdown attempt") From 59972ca90f5d5d323fb284c3b5e55f0c535595b9 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 19 Jan 2023 10:50:43 +0000 Subject: [PATCH 17/25] Form WQ worker_command in __init__ to make type visible to mypy TODO: this breaks: the port hasn't been chosen by this time - it happens at startup --- parsl/executors/workqueue/executor.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/parsl/executors/workqueue/executor.py b/parsl/executors/workqueue/executor.py index 14e5e5c371..77e02ad96c 100644 --- a/parsl/executors/workqueue/executor.py +++ b/parsl/executors/workqueue/executor.py @@ -295,6 +295,8 @@ def __init__(self, if self.init_command != "": self.launch_cmd = self.init_command + "; " + self.launch_cmd + self.worker_command = self._construct_worker_command() + def _get_launch_command(self, block_id): # this executor uses different terminology for worker/launch # commands than in htex @@ -648,7 +650,6 @@ def initialize_scaling(self): """ # Start scaling in/out logger.debug("Starting WorkQueueExecutor with provider: %s", self.provider) - self.worker_command = self._construct_worker_command() self._patch_providers() if hasattr(self.provider, 'init_blocks'): From 9905fb2fb9b6174be06a97f98f7e79400a602211 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 10 Mar 2023 14:23:29 +0000 Subject: [PATCH 18/25] add invocation type annotations to see how well type checking of decorators works --- parsl/tests/test_python_apps/test_basic.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/tests/test_python_apps/test_basic.py b/parsl/tests/test_python_apps/test_basic.py index 1040d23d02..53d6d4e671 100644 --- a/parsl/tests/test_python_apps/test_basic.py +++ b/parsl/tests/test_python_apps/test_basic.py @@ -4,7 +4,7 @@ @python_app -def double(x): +def double(x: float) -> float: return x * 2 From f27715d272b7ceb44c787644eb3895275b63ae85 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 10 Feb 2023 17:51:32 +0000 Subject: [PATCH 19/25] Add typing for dfk.submit() this PR relies on earlier PRs which fix broken behaviour of ignore_for_cache vs AUTO_LOG, which was fixed in a few previous PRs. (TODO: numbers here) --- parsl/dataflow/memoization.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/parsl/dataflow/memoization.py b/parsl/dataflow/memoization.py index e4d657ccaa..0fde30cbb3 100644 --- a/parsl/dataflow/memoization.py +++ b/parsl/dataflow/memoization.py @@ -189,7 +189,11 @@ def make_hash(self, task: TaskRecord) -> str: logger.debug("Ignoring these kwargs for checkpointing: %s", ignore_list) for k in ignore_list: logger.debug("Ignoring kwarg %s", k) - del filtered_kw[k] + if k in filtered_kw: + del filtered_kw[k] + else: + # TODO make a parsl error? + raise ValueError(f"Cannot ignore keyword {k} which has not been supplied") if 'outputs' in task['kwargs']: outputs = task['kwargs']['outputs'] @@ -266,7 +270,7 @@ def update_memo(self, task: TaskRecord, r: Future[Any]) -> None: return if not isinstance(task['hashsum'], str): - logger.error("Attempting to update app cache entry but hashsum is not a string key") + logger.error(f"Attempting to update app cache entry but hashsum is not a string key: {task['hashsum']}") return if task['hashsum'] in self.memo_lookup_table: From baaec6b8f71e19eda0e1014b725f3c4d2f59898c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 19 Jan 2023 10:07:32 +0000 Subject: [PATCH 20/25] more wq typechecking - add asserts to reflect assumed behaviour i should check these asserts to see if they can disappear --- parsl/executors/workqueue/executor.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/parsl/executors/workqueue/executor.py b/parsl/executors/workqueue/executor.py index 77e02ad96c..8212f1f187 100644 --- a/parsl/executors/workqueue/executor.py +++ b/parsl/executors/workqueue/executor.py @@ -540,6 +540,7 @@ def _patch_providers(self): if isinstance(self.provider, CondorProvider): path_to_worker = shutil.which('work_queue_worker') self.worker_command = './' + self.worker_command + assert path_to_worker is not None self.provider.transfer_input_files.append(path_to_worker) if self.project_password_file: self.provider.transfer_input_files.append(self.project_password_file) @@ -621,6 +622,7 @@ def _prepare_package(self, fn, extra_pkgs): os.makedirs(pkg_dir, exist_ok=True) with tempfile.NamedTemporaryFile(suffix='.yaml') as spec: logger.info("Analyzing dependencies of %s", fn_name) + assert package_analyze_script is not None analyze_cmdline = [package_analyze_script, exec_parsl_function.__file__, '-', spec.name] for p in extra_pkgs: analyze_cmdline += ["--extra-pkg", p] @@ -637,6 +639,7 @@ def _prepare_package(self, fn, extra_pkgs): os.close(fd) logger.info("Creating dependency package for %s", fn_name) logger.debug("Writing deps for %s to %s", fn_name, tarball) + assert package_create_script is not None subprocess.run([package_create_script, spec.name, tarball], stdout=subprocess.DEVNULL, check=True) logger.debug("Done with conda-pack; moving %s to %s", tarball, pkg) os.rename(tarball, pkg) From 9104615316809f04f53e3e13f6709d671fbcef73 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 21 Mar 2023 14:07:18 +0000 Subject: [PATCH 21/25] move port mailbox into init so it can be seen by typechecker --- parsl/executors/workqueue/executor.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/parsl/executors/workqueue/executor.py b/parsl/executors/workqueue/executor.py index 8212f1f187..6502299415 100644 --- a/parsl/executors/workqueue/executor.py +++ b/parsl/executors/workqueue/executor.py @@ -279,6 +279,9 @@ def __init__(self, self.function_dir = function_dir self.coprocess = coprocess + self._port_mailbox: multiprocessing.Queue[int] + self._port_mailbox = multiprocessing.Queue() + if not self.address: self.address = socket.gethostname() @@ -325,8 +328,6 @@ def start(self): logger.debug("Starting WorkQueueExecutor") - self._port_mailbox = multiprocessing.Queue() - # Create a Process to perform WorkQueue submissions submit_process_kwargs = {"task_queue": self.task_queue, "launch_cmd": self.launch_cmd, From 3f80d8a8508d9cd34154bd368af628d919e81c61 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 9 Jan 2023 16:13:55 +0000 Subject: [PATCH 22/25] Squash benc-mypy history --- Makefile | 2 +- mypy-plugins/parsl_mypy.py | 56 ++++++++++++ mypy.ini | 84 +++++++++++++----- parsl/app/bash.py | 29 ++++-- parsl/app/python.py | 5 +- parsl/channels/errors.py | 46 +++++++--- parsl/channels/local/local.py | 6 +- parsl/channels/ssh/ssh.py | 88 +++++++++++++------ parsl/channels/ssh_il/ssh_il.py | 2 + parsl/data_provider/globus.py | 5 +- parsl/dataflow/dflow.py | 80 ++++++++++++----- parsl/dataflow/taskrecord.py | 2 - parsl/executors/base.py | 66 ++++++++++++-- parsl/executors/high_throughput/executor.py | 68 +++++++++----- .../high_throughput/process_worker_pool.py | 20 ++++- parsl/executors/status_handling.py | 50 +++++++---- parsl/executors/threads.py | 4 +- parsl/executors/workqueue/executor.py | 27 ++++-- parsl/jobs/error_handlers.py | 5 +- parsl/jobs/job_status_poller.py | 31 ++++--- parsl/jobs/strategy.py | 67 +++++++++++--- parsl/monitoring/monitoring.py | 35 ++++---- parsl/multiprocessing.py | 30 ++++--- parsl/providers/ad_hoc/ad_hoc.py | 21 +++-- parsl/providers/aws/aws.py | 3 +- parsl/providers/azure/azure.py | 5 +- parsl/providers/base.py | 21 ++++- parsl/providers/cluster_provider.py | 41 ++++++--- parsl/providers/condor/condor.py | 6 +- parsl/providers/googlecloud/googlecloud.py | 3 +- parsl/providers/kubernetes/kube.py | 12 +-- parsl/providers/local/local.py | 55 ++++++++---- parsl/providers/slurm/slurm.py | 3 + parsl/providers/torque/torque.py | 30 ++++--- parsl/tests/manual_tests/test_worker_count.py | 2 + .../tests/test_checkpointing/test_periodic.py | 3 + parsl/tests/test_python_apps/test_simple.py | 15 +++- parsl/usage_tracking/usage.py | 25 ++++-- parsl/utils.py | 13 ++- requirements.txt | 2 +- 40 files changed, 781 insertions(+), 287 deletions(-) create mode 100644 mypy-plugins/parsl_mypy.py diff --git a/Makefile b/Makefile index 560ee33747..53bd1a0369 100644 --- a/Makefile +++ b/Makefile @@ -45,7 +45,7 @@ clean_coverage: .PHONY: mypy mypy: ## run mypy checks - MYPYPATH=$(CWD)/mypy-stubs mypy parsl/ + PYTHONPATH=$(CWD)/mypy-plugins:$(PYTHONPATH) MYPYPATH=$(CWD)/mypy-stubs mypy --no-incremental parsl/ --show-traceback .PHONY: local_thread_test local_thread_test: ## run all tests with local_thread config diff --git a/mypy-plugins/parsl_mypy.py b/mypy-plugins/parsl_mypy.py new file mode 100644 index 0000000000..a88463132d --- /dev/null +++ b/mypy-plugins/parsl_mypy.py @@ -0,0 +1,56 @@ +from mypy.plugin import FunctionContext, Plugin +from mypy.types import Type +import mypy.nodes as nodes + +def plugin(v): + return ParslMypyPlugin + +class ParslMypyPlugin(Plugin): + def get_type_analyze_hook(self, t): + # print("BENC: gtah t={}".format(t)) + return None + + def get_function_hook(self, f): + if f == "parsl.app.app.python_appXXXX": + return python_app_function_hook + else: + return None + +def python_app_function_hook(ctx: FunctionContext) -> Type: + print("inside python_app function_hook") + print("ctx = {}".format(ctx)) + + # if python_app is being called with a function parameter (rather than + # None, the default) then the return type of the python_app decorator + # is a variation (with a Future added on the type of the decorated + # function...) + + if ctx.callee_arg_names[0] == "function": # will this always be at position 0? probably fragile to assume so, but this code does make that assumption + print(f"python_app called with a function supplied: {ctx.args[0]}") + function_node = ctx.args[0][0] + print(f"function node repr is {repr(function_node)} with type {type(function_node)}") + + # return the type of function_node - actually it needs modifying to have the Future wrapper added.... + if isinstance(function_node, nodes.TempNode): + print(f"temporary node has type {function_node.type}") + print(f"Python type of tempnode.type is {type(function_node.type)}") + print(ctx.api) + # return_type = ctx.api.named_type_or_none("concurrent.futures.Future", [function_node.type.ret_type]) + # return_type = ctx.api.named_generic_type("concurrent.futures.Future", [function_node.type.ret_type]) + # return_type = ctx.api.named_generic_type("builtins.list", [function_node.type.ret_type]) + return_type = function_node.type.ret_type + # return_type = ctx.default_return_type + print(f"plugin chosen return type is {return_type}") + return function_node.type.copy_modified(ret_type=return_type) + else: + print("function node is not specified as something this plugin understands") + return_type = ctx.default_return_type + return return_type + else: + print("python_app called without a function supplied") + # TODO: this should return a type that is aligned with the implementation: + # it's the type of the decorator, assuming that it will definitely be given + # a function this time? or something along those lines... + + print("will return ctx.default_return_type = {}".format(ctx.default_return_type)) + return ctx.default_return_type diff --git a/mypy.ini b/mypy.ini index cea93169f2..e083d97bb4 100644 --- a/mypy.ini +++ b/mypy.ini @@ -1,5 +1,5 @@ [mypy] -plugins = sqlalchemy.ext.mypy.plugin +plugins = sqlalchemy.ext.mypy.plugin, parsl_mypy enable_error_code = ignore-without-code no_implicit_reexport = True @@ -9,6 +9,8 @@ no_implicit_optional = True strict_equality = True warn_unused_ignores = True + +# there are some exceptions to this even in the more-strongly-typed sections warn_unreachable = True [mypy-non_existent.*] @@ -55,10 +57,54 @@ disallow_any_expr = True disallow_any_decorated = True [mypy-parsl.providers.base.*] +disallow_untyped_defs = True disallow_untyped_decorators = True check_untyped_defs = True disallow_subclassing_any = True + + +# modules to be checked mostly more strongly than default: + +[mypy-parsl.dataflow.dflow.*] +disallow_untyped_defs = True + +[mypy-parsl.jobs.*] +disallow_untyped_defs = True +disallow_any_decorated = True + +[mypy-parsl.jobs.strategy.*] +disallow_untyped_defs = True +disallow_any_decorated = True + +[mypy-parsl.jobs.job_status_poller.*] +disallow_untyped_defs = True + +# merge of #1877 introduced stuff that violates this so disabling pending perhaps further investigation +# disallow_any_expr = True + +disallow_any_decorated = True + +[mypy-parsl.config.*] disallow_untyped_defs = True +# Any has to be allowed because TaskRecord now forms part of the type signature of config, +# and task record has Any from the type of tasks args +#disallow_any_expr = True +#disallow_any_decorated = True + +[mypy-parsl.channels.base.*] +disallow_untyped_defs = True +disallow_any_expr = True + +[mypy-parsl.channels.ssh.*] +disallow_untyped_defs = True + +[mypy-parsl.launchers.*] +disallow_untyped_defs = True +disallow_any_decorated = True + +[mypy-parsl.executors.base.*] +disallow_untyped_defs = True +disallow_any_expr = True [mypy-parsl.serialize.*] disallow_untyped_decorators = True @@ -70,13 +116,23 @@ disallow_untyped_defs = True # parsl/serialize/proxystore.py:9: error: Class cannot subclass "Pickler" (has type "Any") disallow_subclassing_any = False -[mypy-parsl.executors.base.*] -disallow_untyped_defs = True -disallow_any_expr = True - [mypy-parsl.executors.high_throughput.interchange.*] disallow_untyped_defs = True +# modules to be checked more weakly than default: + +[mypy-parsl.executors.flux.*] +ignore_errors = True + +[mypy-parsl.executors.workqueue.*] +check_untyped_defs = True + +# this is generated object code: parsl_coprocess_stub +# should still be type checked, and we then rely on the +# work queue code generator to generate valid code. +[mypy-parsl.executors.workqueue.parsl_coprocess.*] +ignore_errors = True + [mypy-parsl.monitoring.*] disallow_untyped_decorators = True check_untyped_defs = True @@ -101,6 +157,8 @@ warn_unreachable = True [mypy-parsl.utils] disallow_untyped_defs = True +# imports from elsewhere that there are no stubs for: + [mypy-flask_sqlalchemy.*] ignore_missing_imports = True @@ -164,22 +222,6 @@ ignore_missing_imports = True [mypy-zmq.*] ignore_missing_imports = True -[mypy-mpi4py.*] -ignore_missing_imports = True - -[mypy-flask.*] -ignore_missing_imports = True - -# this is an internal undocumentated package -# of multiprocessing - trying to get Event -# to typecheck in monitoring, but it's not -# a top level class as far as mypy is concerned. -# but... when commented out seems ok? -# so lets see when happens when I try to merge -# in clean CI -#[mypy-multiprocessing.synchronization.*] -#ignore_missing_imports = True - [mypy-pandas.*] ignore_missing_imports = True diff --git a/parsl/app/bash.py b/parsl/app/bash.py index 6de57b96f1..ee426d0f67 100644 --- a/parsl/app/bash.py +++ b/parsl/app/bash.py @@ -3,20 +3,26 @@ from inspect import signature, Parameter import logging +from typing import Any, Callable, Dict, List, Optional, Sequence, Union +from typing_extensions import Literal + +from parsl.dataflow.futures import AppFuture from parsl.app.errors import wrap_error from parsl.app.app import AppBase -from parsl.dataflow.dflow import DataFlowKernelLoader +from parsl.dataflow.dflow import DataFlowKernel, DataFlowKernelLoader logger = logging.getLogger(__name__) -def remote_side_bash_executor(func, *args, **kwargs): +def remote_side_bash_executor(func: Callable[..., str], *args, **kwargs) -> int: """Executes the supplied function with *args and **kwargs to get a command-line to run, and then run that command-line using bash. """ import os import subprocess + from typing import List, cast import parsl.app.errors as pe + from parsl.data_provider.files import File from parsl.utils import get_std_fname_mode if hasattr(func, '__name__'): @@ -88,7 +94,8 @@ def open_std_fd(fdname): # TODO : Add support for globs here missing = [] - for outputfile in kwargs.get('outputs', []): + outputs = cast(List[File], kwargs.get('outputs', [])) + for outputfile in outputs: fpath = outputfile.filepath if not os.path.exists(fpath): @@ -102,7 +109,10 @@ def open_std_fd(fdname): class BashApp(AppBase): - def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ignore_for_cache=None): + def __init__(self, func: Callable[..., str], data_flow_kernel: Optional[DataFlowKernel] = None, + cache: bool = False, + executors: Union[List[str], Literal['all']] = 'all', + ignore_for_cache: Optional[Sequence[str]] = None) -> None: super().__init__(func, data_flow_kernel=data_flow_kernel, executors=executors, cache=cache, ignore_for_cache=ignore_for_cache) self.kwargs = {} @@ -120,10 +130,16 @@ def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ig # this is done to avoid passing a function type in the args which parsl.serializer # doesn't support remote_fn = partial(update_wrapper(remote_side_bash_executor, self.func), self.func) - remote_fn.__name__ = self.func.__name__ + + # parsl/app/bash.py:145: error: "partial[Any]" has no attribute "__name__" + # but... other parts of the code are relying on getting the __name__ + # of (?) an arbitrary Callable too (which is why we're setting the __name__ + # at all) + remote_fn.__name__ = self.func.__name__ # type: ignore[attr-defined] + self.wrapped_remote_function = wrap_error(remote_fn) - def __call__(self, *args, **kwargs): + def __call__(self, *args, **kwargs) -> AppFuture: """Handle the call to a Bash app. Args: @@ -136,6 +152,7 @@ def __call__(self, *args, **kwargs): App_fut """ + invocation_kwargs: Dict[str, Any] invocation_kwargs = {} invocation_kwargs.update(self.kwargs) invocation_kwargs.update(kwargs) diff --git a/parsl/app/python.py b/parsl/app/python.py index 27a2c464ba..af38731fa8 100644 --- a/parsl/app/python.py +++ b/parsl/app/python.py @@ -10,6 +10,7 @@ from parsl.app.errors import wrap_error from parsl.dataflow.dflow import DataFlowKernelLoader from parsl.utils import AutoCancelTimer +from parsl.dataflow.futures import AppFuture logger = logging.getLogger(__name__) @@ -36,7 +37,7 @@ def inject_exception(thread): class PythonApp(AppBase): """Extends AppBase to cover the Python App.""" - def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ignore_for_cache=None, join=False): + def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ignore_for_cache=None, join: bool = False) -> None: super().__init__( wrap_error(func), data_flow_kernel=data_flow_kernel, @@ -46,7 +47,7 @@ def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ig ) self.join = join - def __call__(self, *args, **kwargs): + def __call__(self, *args, **kwargs) -> AppFuture: """This is where the call to a python app is handled. Args: diff --git a/parsl/channels/errors.py b/parsl/channels/errors.py index 5bf5a0b280..2b004ecb5b 100644 --- a/parsl/channels/errors.py +++ b/parsl/channels/errors.py @@ -3,15 +3,27 @@ from parsl.errors import ParslError from typing import Optional +# vs PR 1846 +# there are differences in style between calling super.__init__ with +# parameters, or at all +# I think these are only stylistic + +# there are some semantic differences - eg removal of exceptions from +# channel error base and adding into only the subclasses which have +# exceptions (can't remember what motivated this specifically) + class ChannelError(ParslError): """ Base class for all exceptions Only to be invoked when only a more specific error is not available. + + vs PR 1846: differs in calling of super.__init__ and I've removed + the Exception parameter """ - def __init__(self, reason: str, e: Exception, hostname: str) -> None: + def __init__(self, reason: str, hostname: str) -> None: + super().__init__() self.reason = reason - self.e = e self.hostname = hostname def __repr__(self) -> str: @@ -31,9 +43,11 @@ class BadHostKeyException(ChannelError): hostname (string) ''' - def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("SSH channel could not be created since server's host keys could not be " - "verified", e, hostname) + # vs PR 1846: removal of 'e' parameter in superclass + # and stored in this class init instead + def __init__(self, e, hostname): + super().__init__("SSH channel could not be created since server's host keys could not be verified", hostname) + self.e = e class BadScriptPath(ChannelError): @@ -45,8 +59,10 @@ class BadScriptPath(ChannelError): hostname (string) ''' + # vs PR 1846, as with BadHostKeyException: remove e from superclass, store in this class def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("Inaccessible remote script dir. Specify script_dir", e, hostname) + super().__init__("Inaccessible remote script dir. Specify script_dir", hostname) + self.e = e class BadPermsScriptPath(ChannelError): @@ -58,8 +74,10 @@ class BadPermsScriptPath(ChannelError): hostname (string) ''' + "vs PR 1846, store exception locally" def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("User does not have permissions to access the script_dir", e, hostname) + super().__init__("User does not have permissions to access the script_dir", hostname) + self.e = e class FileExists(ChannelError): @@ -72,9 +90,12 @@ class FileExists(ChannelError): hostname (string) ''' + # vs PR 1846, PR 1846 uses .format instead of +filename. I previously kept the original behaviour + # but am adopting the .format style here def __init__(self, e: Exception, hostname: str, filename: Optional[str] = None) -> None: super().__init__("File name collision in channel transport phase: {}".format(filename), - e, hostname) + hostname) + self.e = e class AuthException(ChannelError): @@ -87,7 +108,8 @@ class AuthException(ChannelError): ''' def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("Authentication to remote server failed", e, hostname) + super().__init__("Authentication to remote server failed", hostname) + self.e = e class SSHException(ChannelError): @@ -100,7 +122,8 @@ class SSHException(ChannelError): ''' def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("Error connecting or establishing an SSH session", e, hostname) + super().__init__("Error connecting or establishing an SSH session", hostname) + self.e = e class FileCopyException(ChannelError): @@ -113,4 +136,5 @@ class FileCopyException(ChannelError): ''' def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("File copy failed due to {0}".format(e), e, hostname) + super().__init__("File copy failed due to {0}".format(e), hostname) + self.e = e diff --git a/parsl/channels/local/local.py b/parsl/channels/local/local.py index e933f0687f..d90b517034 100644 --- a/parsl/channels/local/local.py +++ b/parsl/channels/local/local.py @@ -10,6 +10,8 @@ logger = logging.getLogger(__name__) +from typing import Dict, Tuple, Optional + class LocalChannel(Channel, RepresentationMixin): ''' This is not even really a channel, since opening a local shell is not heavy @@ -32,7 +34,7 @@ def __init__(self, userhome=".", envs={}, script_dir=None): self._envs.update(envs) self.script_dir = script_dir - def execute_wait(self, cmd, walltime=None, envs={}): + def execute_wait(self, cmd: str, walltime: Optional[int] = None, envs: Dict[str, str] = {}) -> Tuple[int, str, str]: ''' Synchronously execute a commandline string on the shell. Args: @@ -73,7 +75,7 @@ def execute_wait(self, cmd, walltime=None, envs={}): return (retcode, stdout.decode("utf-8"), stderr.decode("utf-8")) - def push_file(self, source, dest_dir): + def push_file(self, source: str, dest_dir: str) -> str: ''' If the source files dirpath is the same as dest_dir, a copy is not necessary, and nothing is done. Else a copy is made. diff --git a/parsl/channels/ssh/ssh.py b/parsl/channels/ssh/ssh.py index 02c898ad68..0bdbff4326 100644 --- a/parsl/channels/ssh/ssh.py +++ b/parsl/channels/ssh/ssh.py @@ -1,6 +1,7 @@ import errno import logging import os +import typeguard import paramiko from parsl.channels.base import Channel @@ -9,11 +10,18 @@ logger = logging.getLogger(__name__) +from typing import Any, Dict, List, Tuple, Optional + class NoAuthSSHClient(paramiko.SSHClient): - def _auth(self, username, *args): - self._transport.auth_none(username) - return + def _auth(self, username: str, *args: List[Any]) -> None: + # swapped _internal variable for get_transport accessor + # method that I'm assuming without checking does the + # same thing. + transport = self.get_transport() + if transport is None: + raise RuntimeError("Expected a transport to be available") + transport.auth_none(username) class SSHChannel(Channel, RepresentationMixin): @@ -26,8 +34,18 @@ class SSHChannel(Channel, RepresentationMixin): ''' - def __init__(self, hostname, username=None, password=None, script_dir=None, envs=None, - gssapi_auth=False, skip_auth=False, port=22, key_filename=None, host_keys_filename=None): + @typeguard.typechecked + def __init__(self, + hostname: str, + username: Optional[str] = None, + password: Optional[str] = None, + script_dir: Optional[str] = None, + envs: Optional[Dict[str, str]] = None, + gssapi_auth: bool = False, + skip_auth: bool = False, + port: int = 22, + key_filename: Optional[str] = None, + host_keys_filename: Optional[str] = None): ''' Initialize a persistent connection to the remote system. We should know at this point whether ssh connectivity is possible @@ -50,29 +68,39 @@ def __init__(self, hostname, username=None, password=None, script_dir=None, envs self.username = username self.password = password self.port = port - self.script_dir = script_dir + + # if script_dir is a `str`, which it is from Channel, then can't + # assign None to it. Here and the property accessors are changed + # in benc-mypy to raise an error ratehr than return a None, + # because Channel-using code assumes that script_dir will always + # return a string and not a None. That assumption is not otherwise + # guaranteed by the type-system... + self._script_dir = None + if script_dir: + self.script_dir = script_dir + self.skip_auth = skip_auth self.gssapi_auth = gssapi_auth self.key_filename = key_filename self.host_keys_filename = host_keys_filename if self.skip_auth: - self.ssh_client = NoAuthSSHClient() + self.ssh_client: paramiko.SSHClient = NoAuthSSHClient() else: self.ssh_client = paramiko.SSHClient() self.ssh_client.load_system_host_keys(filename=host_keys_filename) self.ssh_client.set_missing_host_key_policy(paramiko.AutoAddPolicy()) - self.sftp_client = None + self.sftp_client: Optional[paramiko.SFTPClient] = None - self.envs = {} + self.envs = {} # type: Dict[str, str] if envs is not None: self.envs = envs - def _is_connected(self): + def _is_connected(self) -> bool: transport = self.ssh_client.get_transport() if self.ssh_client else None - return transport and transport.is_active() + return bool(transport and transport.is_active()) - def _connect(self): + def _connect(self) -> None: if not self._is_connected(): logger.debug(f"connecting to {self.hostname}:{self.port}") try: @@ -87,6 +115,8 @@ def _connect(self): key_filename=self.key_filename ) transport = self.ssh_client.get_transport() + if not transport: + raise RuntimeError("SSH client transport is None, despite connecting") self.sftp_client = paramiko.SFTPClient.from_transport(transport) except paramiko.BadHostKeyException as e: @@ -101,15 +131,17 @@ def _connect(self): except Exception as e: raise SSHException(e, self.hostname) - def _valid_sftp_client(self): + def _valid_sftp_client(self) -> paramiko.SFTPClient: self._connect() + if self.sftp_client is None: + raise RuntimeError("Internal consistency error: self.sftp_client should be valid but is not") return self.sftp_client - def _valid_ssh_client(self): + def _valid_ssh_client(self) -> paramiko.SSHClient: self._connect() return self.ssh_client - def prepend_envs(self, cmd, env={}): + def prepend_envs(self, cmd: str, env: Dict[str, str] = {}) -> str: env.update(self.envs) if len(env.keys()) > 0: @@ -117,7 +149,7 @@ def prepend_envs(self, cmd, env={}): return 'env {0} {1}'.format(env_vars, cmd) return cmd - def execute_wait(self, cmd, walltime=2, envs={}): + def execute_wait(self, cmd: str, walltime: int = 2, envs: Dict[str, str] = {}) -> Tuple[int, str, str]: ''' Synchronously execute a commandline string on the shell. Args: @@ -144,7 +176,7 @@ def execute_wait(self, cmd, walltime=2, envs={}): exit_status = stdout.channel.recv_exit_status() return exit_status, stdout.read().decode("utf-8"), stderr.read().decode("utf-8") - def push_file(self, local_source, remote_dir): + def push_file(self, local_source: str, remote_dir: str) -> str: ''' Transport a local file to a directory on a remote machine Args: @@ -184,7 +216,7 @@ def push_file(self, local_source, remote_dir): return remote_dest - def pull_file(self, remote_source, local_dir): + def pull_file(self, remote_source: str, local_dir: str) -> str: ''' Transport file on the remote side to a local directory Args: @@ -217,11 +249,12 @@ def pull_file(self, remote_source, local_dir): return local_dest - def close(self): + def close(self) -> bool: if self._is_connected(): - return self.ssh_client.close() + self.ssh_client.close() + return True - def isdir(self, path): + def isdir(self, path: str) -> bool: """Return true if the path refers to an existing directory. Parameters @@ -237,7 +270,7 @@ def isdir(self, path): return result - def makedirs(self, path, mode=0o700, exist_ok=False): + def makedirs(self, path: str, mode: int = 0o700, exist_ok: bool = False) -> None: """Create a directory on the remote side. If intermediate directories do not exist, they will be created. @@ -257,7 +290,7 @@ def makedirs(self, path, mode=0o700, exist_ok=False): self.execute_wait('mkdir -p {}'.format(path)) self._valid_sftp_client().chmod(path, mode) - def abspath(self, path): + def abspath(self, path: str) -> str: """Return the absolute path on the remote side. Parameters @@ -268,9 +301,12 @@ def abspath(self, path): return self._valid_sftp_client().normalize(path) @property - def script_dir(self): - return self._script_dir + def script_dir(self) -> str: + if self._script_dir: + return self._script_dir + else: + raise RuntimeError("scriptdir was not set") @script_dir.setter - def script_dir(self, value): + def script_dir(self, value: Optional[str]) -> None: self._script_dir = value diff --git a/parsl/channels/ssh_il/ssh_il.py b/parsl/channels/ssh_il/ssh_il.py index 0905334329..4bac6712c2 100644 --- a/parsl/channels/ssh_il/ssh_il.py +++ b/parsl/channels/ssh_il/ssh_il.py @@ -67,6 +67,8 @@ def __init__(self, hostname, username=None, password=None, script_dir=None, envs ''' transport = self.ssh_client.get_transport() + if transport is None: + raise RuntimeError("Expected transport to be available") il_password = getpass.getpass('Enter {0} Logon password :'.format(hostname)) transport.auth_password(username, il_password) diff --git a/parsl/data_provider/globus.py b/parsl/data_provider/globus.py index 50bddc5ecf..d01720a614 100644 --- a/parsl/data_provider/globus.py +++ b/parsl/data_provider/globus.py @@ -6,7 +6,7 @@ import typeguard from functools import partial -from typing import Optional +from typing import Any, Optional from parsl.app.app import python_app from parsl.utils import RepresentationMixin from parsl.data_provider.staging import Staging @@ -59,8 +59,11 @@ class Globus: - monitoring transfers. """ + authorizer: Any authorizer = None + TOKEN_FILE: str + @classmethod def init(cls): token_path = os.path.join(os.path.expanduser('~'), '.parsl') diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index f1e3101163..1fe14d7894 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -13,12 +13,15 @@ import datetime from getpass import getuser from typeguard import typechecked -from typing import Any, Callable, Dict, List, Optional, Sequence, Tuple, Union +from typing import cast, Any, Callable, Dict, Iterable, Optional, Union, List, Sequence, Tuple from uuid import uuid4 from socket import gethostname from concurrent.futures import Future from functools import partial +# mostly for type checking +from parsl.executors.base import ParslExecutor, FutureWithTaskID + import parsl from parsl.app.errors import RemoteExceptionWrapper from parsl.app.futures import DataFuture @@ -36,12 +39,12 @@ from parsl.jobs.job_status_poller import JobStatusPoller from parsl.jobs.states import JobStatus, JobState from parsl.usage_tracking.usage import UsageTracker -from parsl.executors.base import ParslExecutor from parsl.executors.status_handling import BlockProviderExecutor from parsl.executors.threads import ThreadPoolExecutor from parsl.monitoring import MonitoringHub from parsl.process_loggers import wrap_with_logs from parsl.providers.base import ExecutionProvider +from parsl.providers.base import Channeled, MultiChanneled from parsl.utils import get_version, get_std_fname_mode, get_all_checkpoints, Timer from parsl.monitoring.message_type import MessageType @@ -210,14 +213,29 @@ def _send_task_log_info(self, task_record: TaskRecord) -> None: task_log_info = self._create_task_log_info(task_record) self.monitoring.send(MessageType.TASK_INFO, task_log_info) - def _create_task_log_info(self, task_record): + def _create_task_log_info(self, task_record: TaskRecord) -> Dict[str, Any]: """ Create the dictionary that will be included in the log. """ - info_to_monitor = ['func_name', 'memoize', 'hashsum', 'fail_count', 'fail_cost', 'status', - 'id', 'time_invoked', 'try_time_launched', 'time_returned', 'try_time_returned', 'executor'] - task_log_info = {"task_" + k: task_record[k] for k in info_to_monitor} + # because self.tasks[task_id] is now a TaskRecord not a Dict[str,...], type checking + # can't do enough type checking if just iterating over this list of keys to copy + # and the assignments need to be written out explicitly. + + task_log_info = {} # type: Dict[str, Any] + + task_log_info["task_func_name"] = task_record['func_name'] + task_log_info["task_memoize"] = task_record['memoize'] + task_log_info["task_hashsum"] = task_record['hashsum'] + task_log_info["task_fail_count"] = task_record['fail_count'] + task_log_info["task_fail_cost"] = task_record['fail_cost'] + task_log_info["task_status"] = task_record['status'] + task_log_info["task_id"] = task_record['id'] + task_log_info["task_time_invoked"] = task_record['time_invoked'] + task_log_info["task_try_time_launched"] = task_record['try_time_launched'] + task_log_info["task_time_returned"] = task_record['time_returned'] + task_log_info["task_try_time_returned"] = task_record['try_time_returned'] + task_log_info["task_executor"] = task_record['executor'] task_log_info['run_id'] = self.run_id task_log_info['try_id'] = task_record['try_id'] task_log_info['timestamp'] = datetime.datetime.now() @@ -245,9 +263,8 @@ def _create_task_log_info(self, task_record): task_log_info['task_stderr'] = stderr_name task_log_info['task_fail_history'] = ",".join(task_record['fail_history']) task_log_info['task_depends'] = None - if task_record['depends'] is not None: - task_log_info['task_depends'] = ",".join([str(t.tid) for t in task_record['depends'] - if isinstance(t, AppFuture) or isinstance(t, DataFuture)]) + task_log_info['task_depends'] = ",".join([str(t.tid) for t in task_record['depends'] if isinstance(t, AppFuture) or isinstance(t, DataFuture)]) + task_log_info['task_joins'] = None if isinstance(task_record['joins'], list): @@ -560,7 +577,7 @@ def update_task_state(self, task_record: TaskRecord, new_state: States) -> None: """ with self.task_state_counts_lock: - if 'status' in task_record: + if hasattr(task_record, 'status'): self.task_state_counts[task_record['status']] -= 1 self.task_state_counts[new_state] += 1 task_record['status'] = new_state @@ -721,7 +738,7 @@ def launch_task(self, task_record: TaskRecord) -> Future: self._send_task_log_info(task_record) - if hasattr(exec_fu, "parsl_executor_task_id"): + if isinstance(exec_fu, FutureWithTaskID): logger.info(f"Parsl task {task_id} try {try_id} launched on executor {executor.label} with executor id {exec_fu.parsl_executor_task_id}") else: logger.info(f"Parsl task {task_id} try {try_id} launched on executor {executor.label}") @@ -739,6 +756,9 @@ def _add_input_deps(self, executor: str, args: Sequence[Any], kwargs: Dict[str, - executor (str) : executor where the app is going to be launched - args (List) : Positional args to app function - kwargs (Dict) : Kwargs to app function + - func : the function that will be invoked + + Returns: args, kwargs, (replacement, wrapping) function """ # Return if the task is a data management task, rather than doing @@ -825,7 +845,9 @@ def check_dep(d: Any) -> None: return depends - def _unwrap_futures(self, args, kwargs): + def _unwrap_futures(self, + args: Sequence[Any], + kwargs: Dict[str, Any]) -> Tuple[Sequence[Any], Dict[str, Any], Sequence[Tuple[Exception, str]]]: """This function should be called when all dependencies have completed. It will rewrite the arguments for that task, replacing each Future @@ -843,6 +865,10 @@ def _unwrap_futures(self, args, kwargs): a rewritten kwargs dict pairs of exceptions, task ids from any Futures which stored exceptions rather than results. + + TODO: mypy note: we take a *tuple* of args but return a *list* of args. + That's an (unintentional?) change of type of arg structure which leads me + to try to represent the args in TaskRecord as a Sequence """ dep_failures = [] @@ -857,7 +883,9 @@ def _unwrap_futures(self, args, kwargs): # then refer to the task ID. # Otherwise make a repr of the Future object. if hasattr(dep, 'task_record') and dep.task_record['dfk'] == self: - tid = "task " + repr(dep.task_record['id']) + d_tmp = cast(Any, dep) + tid = d_tmp.task_record['id'] + tid = "task " + repr(d_tmp.task_record['id']) else: tid = repr(dep) dep_failures.extend([(e, tid)]) @@ -872,7 +900,9 @@ def _unwrap_futures(self, args, kwargs): kwargs[key] = dep.result() except Exception as e: if hasattr(dep, 'task_record'): - tid = dep.task_record['id'] + # see note about protocol + d_tmp = cast(Any, dep) + tid = d_tmp.task_record['id'] else: tid = None dep_failures.extend([(e, tid)]) @@ -886,7 +916,8 @@ def _unwrap_futures(self, args, kwargs): new_inputs.extend([dep.result()]) except Exception as e: if hasattr(dep, 'task_record'): - tid = dep.task_record['id'] + d_tmp = cast(Any, dep) + tid = d_tmp.task_record['id'] else: tid = None dep_failures.extend([(e, tid)]) @@ -1110,7 +1141,7 @@ def _create_remote_dirs_over_channel(self, provider: ExecutionProvider, channel: channel.makedirs(channel.script_dir, exist_ok=True) - def add_executors(self, executors): + def add_executors(self, executors: Sequence[ParslExecutor]) -> None: for executor in executors: if executor.label in self.executors: raise ConfigurationError("Executor {executor.label} already added") @@ -1119,17 +1150,21 @@ def add_executors(self, executors): executor.run_dir = self.run_dir executor.hub_address = self.hub_address executor.hub_port = self.hub_interchange_port - if hasattr(executor, 'provider'): + if executor.provider is not None: # could be a protocol? if hasattr(executor.provider, 'script_dir'): executor.provider.script_dir = os.path.join(self.run_dir, 'submit_scripts') os.makedirs(executor.provider.script_dir, exist_ok=True) - if hasattr(executor.provider, 'channels'): + if isinstance(executor.provider, MultiChanneled): logger.debug("Creating script_dir across multiple channels") for channel in executor.provider.channels: self._create_remote_dirs_over_channel(executor.provider, channel) - else: + elif isinstance(executor.provider, Channeled): self._create_remote_dirs_over_channel(executor.provider, executor.provider.channel) + else: + raise ValueError(("Assuming executor.provider has channel(s) based on it " + "having provider/script_dir, but actually it isn't a " + "(Multi)Channeled instance. provider = {}").format(executor.provider)) self.executors[executor.label] = executor block_ids = executor.start() @@ -1226,6 +1261,8 @@ def cleanup(self) -> None: msg = executor.create_monitoring_info(new_status) logger.debug("Sending message {} to hub from DFK".format(msg)) self.monitoring.send(MessageType.BLOCK_INFO, msg) + else: + logger.error("There is no provider to perform scaling in") else: # and bad_state_is_set logger.warning(f"Not shutting down executor {executor.label} because it is in bad state") logger.info(f"Shutting down executor {executor.label}") @@ -1259,7 +1296,7 @@ def checkpoint(self, tasks: Optional[Sequence[TaskRecord]] = None) -> str: Kwargs: - tasks (List of task records) : List of task ids to checkpoint. Default=None - if set to None, we iterate over all tasks held by the DFK. + if set to None or [], we iterate over all tasks held by the DFK. .. note:: Checkpointing only works if memoization is enabled @@ -1271,6 +1308,7 @@ def checkpoint(self, tasks: Optional[Sequence[TaskRecord]] = None) -> str: """ with self.checkpoint_lock: if tasks: + checkpoint_queue: Iterable[TaskRecord] checkpoint_queue = tasks else: checkpoint_queue = self.checkpointable_tasks @@ -1301,7 +1339,7 @@ def checkpoint(self, tasks: Optional[Sequence[TaskRecord]] = None) -> str: hashsum = task_record['hashsum'] if not hashsum: continue - t = {'hash': hashsum, 'exception': None, 'result': app_fu.result()} + t = {'hash': hashsum, 'exception': None, 'result': app_fu.result()} # type: Dict[str, Any] # We are using pickle here since pickle dumps to a file in 'ab' # mode behave like a incremental log. diff --git a/parsl/dataflow/taskrecord.py b/parsl/dataflow/taskrecord.py index 34d5ef4ca5..afc3743775 100644 --- a/parsl/dataflow/taskrecord.py +++ b/parsl/dataflow/taskrecord.py @@ -5,8 +5,6 @@ from typing_extensions import TypedDict from concurrent.futures import Future - -# only for type checking: from typing import Any, Callable, Dict, Optional, List, Sequence, TYPE_CHECKING, Union if TYPE_CHECKING: diff --git a/parsl/executors/base.py b/parsl/executors/base.py index 3a1e941fb0..bf2ac728c2 100644 --- a/parsl/executors/base.py +++ b/parsl/executors/base.py @@ -1,9 +1,15 @@ -from abc import ABCMeta, abstractmethod +from __future__ import annotations +from abc import ABCMeta, abstractmethod, abstractproperty from concurrent.futures import Future -from typing import Any, Callable, Dict, Optional, List +from typing import Any, Callable, Dict, Optional, List, Sequence, Union from typing_extensions import Literal, Self from parsl.jobs.states import JobStatus +from parsl.data_provider.staging import Staging + +# for type checking: +from parsl.providers.base import ExecutionProvider +from typing_extensions import runtime_checkable, Protocol class ParslExecutor(metaclass=ABCMeta): @@ -43,12 +49,35 @@ class ParslExecutor(metaclass=ABCMeta): def __enter__(self) -> Self: return self + # mypy doesn't actually check that the below are defined by + # concrete subclasses - see github.com/python/mypy/issues/4426 + # and maybe PEP-544 Protocols + + def __init__(self) -> None: + self.label: str + self.radio_mode: str = "udp" + + self.provider: Optional[ExecutionProvider] = None + # this is wrong here. eg thread local executor has no provider. + # perhaps its better attached to the block scaling provider? + # cross-ref with notes of @property provider() in the + # nostatushandlingexecutor. + + # there's an abstraction problem here - what kind of executor should + # statically have this? for now I'll implement a protocol and assert + # the protocol holds, wherever the code makes that assumption. + # self.outstanding: int = None # what is this? used by strategy + self.working_dir: Optional[str] = None + self.storage_access: Optional[Sequence[Staging]] = None + self.run_id: Optional[str] = None + + # too lazy to figure out what the three Anys here should be def __exit__(self, exc_type: Any, exc_val: Any, exc_tb: Any) -> Literal[False]: self.shutdown() return False @abstractmethod - def start(self) -> Optional[List[str]]: + def start(self) -> Optional[Sequence[str]]: """Start the executor. Any spin-up operations (for example: starting thread pools) should be performed here. @@ -56,9 +85,8 @@ def start(self) -> Optional[List[str]]: pass @abstractmethod - def submit(self, func: Callable, resource_specification: Dict[str, Any], *args: Any, **kwargs: Any) -> Future: + def submit(self, func: Callable, resource_specification: Dict[str, Any], *args: Any, **kwargs: Dict[str, Any]) -> Future: """Submit. - The executor can optionally set a parsl_executor_task_id attribute on the Future that it returns, and in that case, parsl will log a relationship between the executor's task ID and parsl level try/task @@ -77,6 +105,13 @@ def shutdown(self) -> None: def create_monitoring_info(self, status: Dict[str, JobStatus]) -> List[object]: """Create a monitoring message for each block based on the poll status. + TODO: block_id_type should be an enumerated list of valid strings, rather than all strings + + TODO: there shouldn't be any default values for this - when it is invoked, it should be explicit which is needed? + Neither seems more natural to me than the other. + + TODO: internal vs external should be more clearly documented here + :return: a list of dictionaries mapping to the info of each block """ return [] @@ -120,3 +155,24 @@ def hub_port(self) -> Optional[int]: @hub_port.setter def hub_port(self, value: Optional[int]) -> None: self._hub_port = value + + +@runtime_checkable +class HasWorkersPerNode(Protocol): + """A marker type to indicate that the executor has a notion of workers per node. This maybe should merge into the block executor?""" + @abstractproperty + def workers_per_node(self) -> Union[int, float]: + pass + + +class HasOutstanding: + """A marker type to indicate that the executor has a count of outstanding tasks. This maybe should merge into the block executor?""" + @abstractproperty + def outstanding(self) -> int: + pass + + +class FutureWithTaskID(Future): + def __init__(self, task_id: str) -> None: + super().__init__() + self.parsl_executor_task_id = task_id diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index 1397f7513d..132b1eab82 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -8,7 +8,7 @@ import pickle from multiprocessing import Queue from typing import Dict, Sequence -from typing import List, Optional, Tuple, Union, Callable +from typing import Any, List, Optional, Tuple, Union, Callable import math import parsl.launchers @@ -17,6 +17,7 @@ from parsl.app.errors import RemoteExceptionWrapper from parsl.jobs.states import JobStatus, JobState, TERMINAL_STATES from parsl.executors.high_throughput import zmq_pipes +from parsl.executors.base import HasWorkersPerNode, FutureWithTaskID from parsl.executors.high_throughput import interchange from parsl.executors.errors import ( BadMessage, ScalingFailed, @@ -33,7 +34,7 @@ from parsl.addresses import get_all_addresses from parsl.process_loggers import wrap_with_logs -from parsl.multiprocessing import ForkProcess +from parsl.multiprocessing import forkProcess from parsl.utils import RepresentationMixin from parsl.providers import LocalProvider @@ -59,7 +60,7 @@ "--available-accelerators {accelerators}") -class HighThroughputExecutor(BlockProviderExecutor, RepresentationMixin): +class HighThroughputExecutor(BlockProviderExecutor, RepresentationMixin, HasWorkersPerNode): """Executor designed for cluster-scale The HighThroughputExecutor system has the following components: @@ -242,6 +243,8 @@ def __init__(self, block_error_handler: Union[bool, Callable[[BlockProviderExecutor, Dict[str, JobStatus]], None]] = True, encrypted: bool = False): + self._queue_management_thread: Optional[threading.Thread] + logger.debug("Initializing HighThroughputExecutor") BlockProviderExecutor.__init__(self, provider=provider, block_error_handler=block_error_handler) @@ -262,12 +265,12 @@ def __init__(self, mem_slots = max_workers cpu_slots = max_workers - if hasattr(self.provider, 'mem_per_node') and \ + if isinstance(self.provider, ExecutionProvider) and \ self.provider.mem_per_node is not None and \ mem_per_worker is not None and \ mem_per_worker > 0: mem_slots = math.floor(self.provider.mem_per_node / mem_per_worker) - if hasattr(self.provider, 'cores_per_node') and \ + if isinstance(self.provider, ExecutionProvider) and \ self.provider.cores_per_node is not None: cpu_slots = math.floor(self.provider.cores_per_node / cores_per_worker) @@ -288,6 +291,7 @@ def __init__(self, self.run_id = None # set to the correct run_id in dfk self.hub_address = None # set to the correct hub address in dfk self.hub_port = None # set to the correct hub port in dfk + self.worker_ports = worker_ports self.worker_port_range = worker_port_range self.interchange_port_range = interchange_port_range @@ -304,11 +308,15 @@ def __init__(self, assert mpi_launcher in VALID_LAUNCHERS, \ f"mpi_launcher must be set to one of {VALID_LAUNCHERS}" if self.enable_mpi_mode: - assert isinstance(self.provider.launcher, parsl.launchers.SingleNodeLauncher), \ - "mpi_mode requires the provider to be configured to use a SingleNodeLauncher" + # awkward to typecheck because a provider doesn't have launcher attribute, according to its type, + # but this code hopes that it does - ignore attr-defined on parsl.provider + assert self.provider is None or isinstance(self.provider.launcher, # type: ignore[attr-defined] + parsl.launchers.SingleNodeLauncher), "mpi_mode requires the provider to be configured to use a SingleNodeLauncher" self.mpi_launcher = mpi_launcher + self._executor_exception = None + if not launch_cmd: launch_cmd = DEFAULT_LAUNCH_CMD self.launch_cmd = launch_cmd @@ -325,7 +333,7 @@ def worker_logdir(self): return "{}/{}".format(self.worker_logdir_root, self.label) return self.logdir - def initialize_scaling(self): + def initialize_scaling(self) -> Sequence[str]: """Compose the launch command and scale out the initial blocks. """ debug_opts = "--debug" if self.worker_debug else "" @@ -336,6 +344,8 @@ def initialize_scaling(self): if self.address_probe_timeout: address_probe_timeout_string = "--address_probe_timeout={}".format(self.address_probe_timeout) + assert self.provider is not None + l_cmd = self.launch_cmd.format(debug=debug_opts, prefetch_capacity=self.prefetch_capacity, address_probe_timeout_string=address_probe_timeout_string, @@ -361,7 +371,7 @@ def initialize_scaling(self): logger.debug("Starting HighThroughputExecutor with provider:\n%s", self.provider) # TODO: why is this a provider property? - block_ids = [] + block_ids = [] # type: List[str] if hasattr(self.provider, 'init_blocks'): try: block_ids = self.scale_out(blocks=self.provider.init_blocks) @@ -370,7 +380,7 @@ def initialize_scaling(self): raise e return block_ids - def start(self): + def start(self) -> Sequence[str]: """Create the Interchange process and connect to it. """ if self.encrypted and self.cert_dir is None: @@ -403,7 +413,7 @@ def start(self): return block_ids @wrap_with_logs - def _queue_management_worker(self): + def _queue_management_worker(self) -> None: """Listen to the queue for task status messages and handle them. Depending on the message, tasks will be updated with results, exceptions, @@ -505,14 +515,14 @@ def _queue_management_worker(self): logger.info("Queue management worker finished") - def _start_local_interchange_process(self): + def _start_local_interchange_process(self) -> None: """ Starts the interchange process locally Starts the interchange process locally and uses an internal command queue to get the worker task and result ports that the interchange has bound to. """ - comm_q = Queue(maxsize=10) - self.interchange_proc = ForkProcess(target=interchange.starter, + comm_q = Queue(maxsize=10) # type: Queue[Any] + self.interchange_proc = forkProcess(target=interchange.starter, args=(comm_q,), kwargs={"client_ports": (self.outgoing_q.port, self.incoming_q.port, @@ -538,7 +548,7 @@ def _start_local_interchange_process(self): logger.error("Interchange has not completed initialization in 120s. Aborting") raise Exception("Interchange failed to start") - def _start_queue_management_thread(self): + def _start_queue_management_thread(self) -> None: """Method to start the management thread as a daemon. Checks if a thread already exists, then starts it. @@ -586,11 +596,11 @@ def connected_managers(self) -> List[Dict[str, typing.Any]]: """ return self.command_client.run("MANAGERS") - def connected_blocks(self) -> List[str]: + def connected_blocks(self) -> Sequence[str]: """List of connected block ids""" return self.command_client.run("CONNECTED_BLOCKS") - def _hold_block(self, block_id): + def _hold_block(self, block_id: str) -> None: """ Sends hold command to all managers which are in a specific block Parameters @@ -606,7 +616,7 @@ def _hold_block(self, block_id): logger.debug("Sending hold to manager: {}".format(manager['manager'])) self.hold_worker(manager['manager']) - def submit(self, func, resource_specification, *args, **kwargs): + def submit(self, func, resource_specification, *args, **kwargs) -> "Future[Any]": """Submits work to the outgoing_q. The outgoing_q is an external process listens on this @@ -627,7 +637,10 @@ def submit(self, func, resource_specification, *args, **kwargs): validate_resource_spec(resource_specification) if self.bad_state_is_set: - raise self.executor_exception + if self.executor_exception is None: + raise ValueError("Executor is in bad state, but no exception recorded") + else: + raise self.executor_exception self._task_counter += 1 task_id = self._task_counter @@ -637,8 +650,7 @@ def submit(self, func, resource_specification, *args, **kwargs): args_to_print = tuple([ar if len(ar := repr(arg)) < 100 else (ar[:100] + '...') for arg in args]) logger.debug("Pushing function {} to queue with args {}".format(func, args_to_print)) - fut = Future() - fut.parsl_executor_task_id = task_id + fut: Future = FutureWithTaskID(str(task_id)) self.tasks[task_id] = fut try: @@ -662,7 +674,7 @@ def create_monitoring_info(self, status): """ msg = [] for bid, s in status.items(): - d = {} + d: Dict[str, Any] = {} d['run_id'] = self.run_id d['status'] = s.status_name d['timestamp'] = datetime.datetime.now() @@ -676,13 +688,15 @@ def create_monitoring_info(self, status): def workers_per_node(self) -> Union[int, float]: return self._workers_per_node - def scale_in(self, blocks, max_idletime=None): + def scale_in(self, blocks: int, max_idletime: Optional[float] = None) -> List[str]: """Scale in the number of active blocks by specified amount. The scale in method here is very rude. It doesn't give the workers the opportunity to finish current tasks or cleanup. This is tracked in issue #530 + Exactly one of blocks or block_ids must be specified. + Parameters ---------- @@ -723,6 +737,7 @@ def scale_in(self, blocks, max_idletime=None): if b_id not in block_info: block_info[b_id] = {"tasks": 0, "idle": float('inf')} + block_ids_to_kill: List[str] managers = self.connected_managers() for manager in managers: if not manager['active']: @@ -766,6 +781,13 @@ def scale_in(self, blocks, max_idletime=None): # Potential issue with multiple threads trying to remove the same blocks to_kill = [self.blocks[bid] for bid in block_ids_to_kill if bid in self.blocks] + # who knows if we really have a provider here? provider can be none... + # in practice, it looks like it is guarded by a check around each + # call but that isn't enforced in the type system. + # This assert asserts that the caller has performed the relevant + # check for a provider existing before invoking scale_in. + assert self.provider is not None + r = self.provider.cancel(to_kill) job_ids = self._filter_scale_in_ids(to_kill, r) diff --git a/parsl/executors/high_throughput/process_worker_pool.py b/parsl/executors/high_throughput/process_worker_pool.py index 54460b561d..2d138fb499 100755 --- a/parsl/executors/high_throughput/process_worker_pool.py +++ b/parsl/executors/high_throughput/process_worker_pool.py @@ -10,7 +10,7 @@ import time import queue import uuid -from typing import Sequence, Optional, Dict, List +from typing import Any, Sequence, Optional, Dict, List import zmq import math @@ -21,6 +21,7 @@ from multiprocessing.sharedctypes import Synchronized from parsl import curvezmq + from parsl.process_loggers import wrap_with_logs from parsl.version import VERSION as PARSL_VERSION from parsl.app.errors import RemoteExceptionWrapper @@ -35,6 +36,8 @@ from parsl.executors.high_throughput.mpi_prefix_composer import compose_all, VALID_LAUNCHERS +logger = logging.getLogger("parsl") + HEARTBEAT_CODE = (2 ** 32) - 1 @@ -447,15 +450,26 @@ def handle_monitoring_messages(self, kill_event: threading.Event): logger.critical("Exiting") - def start(self): + def start(self) -> None: """ Start the worker processes. TODO: Move task receiving to a thread """ start = time.time() self._kill_event = threading.Event() + + # When upgrading from mypy 0.961 to 0.981, this change happens: + + # multiprocessing.Manager().dict() according to mypy, does not + # return a Dict, but instead a multiprocessing.managers.DictProxy + # parsl/executors/high_throughput/process_worker_pool.py:416: note: Revealed type is "multiprocessing.managers.DictProxy[Any, Any]" + # + # but this type inference gets figured out, so no need for explicit annotation, + # I think + self._tasks_in_progress = self._mp_manager.dict() + self.procs: Dict[Any, Any] self.procs = {} for worker_id in range(self.worker_count): p = self._start_worker(worker_id) @@ -598,7 +612,7 @@ def worker( logdir: str, debug: bool, mpi_launcher: str, -): +) -> None: """ Put request token into queue diff --git a/parsl/executors/status_handling.py b/parsl/executors/status_handling.py index 7cdedb156a..4154e3f526 100644 --- a/parsl/executors/status_handling.py +++ b/parsl/executors/status_handling.py @@ -6,7 +6,7 @@ from concurrent.futures import Future from typing import List, Any, Dict, Optional, Tuple, Union, Callable -from parsl.executors.base import ParslExecutor +from parsl.executors.base import ParslExecutor, HasOutstanding from parsl.executors.errors import BadStateException, ScalingFailed from parsl.jobs.states import JobStatus, JobState from parsl.jobs.error_handlers import simple_error_handler, noop_error_handler @@ -16,7 +16,11 @@ logger = logging.getLogger(__name__) -class BlockProviderExecutor(ParslExecutor): +# TODO: its not clear to me if block providers should always have outstanding? +# if they do, then a separate marker class is not needed. and if they don't, +# then HasOutstanding should be a superclass of individual implementations +# rather than the base BlockProviderExecutor. +class BlockProviderExecutor(ParslExecutor, HasOutstanding): """A base class for executors which scale using blocks. This base class is intended to help with executors which: @@ -48,7 +52,9 @@ def __init__(self, *, provider: Optional[ExecutionProvider], block_error_handler: Union[bool, Callable[[BlockProviderExecutor, Dict[str, JobStatus]], None]]): super().__init__() + # TODO: untangle having two provider attributes self._provider = provider + self.provider = provider self.block_error_handler: Callable[[BlockProviderExecutor, Dict[str, JobStatus]], None] if isinstance(block_error_handler, bool): if block_error_handler: @@ -97,10 +103,11 @@ def status_polling_interval(self): :return: the number of seconds to wait between calls to status() or zero if no polling should be done """ - if self._provider is None: - return 0 - else: - return self._provider.status_polling_interval + # this codepath is unreachable because execution provider is always set in init + # if self._provider is None: + # return 0 + # else: + return self._provider.status_polling_interval def _fail_job_async(self, block_id: Any, message: str): """Marks a job that has failed to start but would not otherwise be included in status() @@ -111,12 +118,14 @@ def _fail_job_async(self, block_id: Any, message: str): logger.info(f"Allocated block ID {block_id} for simulated failure") self._simulated_status[block_id] = JobStatus(JobState.FAILED, message) - @abstractproperty - def outstanding(self) -> int: - """This should return the number of tasks that the executor has been given to run (waiting to run, and running now)""" - - raise NotImplementedError("Classes inheriting from BlockProviderExecutor must implement " - "outstanding()") + # There's an abstraction problem here: is this a property of all executors or only executors + # which can be scaled with blocks? + # @abstractproperty + # def outstanding(self) -> int: + # """This should return the number of tasks that the executor has been given to run (waiting to run, and running now)""" + # + # raise NotImplementedError("Classes inheriting from BlockProviderExecutor must implement " + # "outstanding()") def status(self) -> Dict[str, JobStatus]: """Return the status of all jobs/blocks currently known to this executor. @@ -174,9 +183,11 @@ def handle_errors(self, status: Dict[str, JobStatus]) -> None: def tasks(self) -> Dict[object, Future]: return self._tasks - @property - def provider(self): - return self._provider + # this is defined as a regular attribute at the superclass level, + # which may or may not be correct. + # @property + # def provider(self): + # return self._provider def _filter_scale_in_ids(self, to_kill, killed): """ Filter out job id's that were not killed @@ -190,7 +201,7 @@ def scale_out(self, blocks: int = 1) -> List[str]: """ if not self.provider: raise ScalingFailed(self, "No execution provider available") - block_ids = [] + block_ids: List[str] = [] # is this true? is a block ID always a string (vs eg a POpen object?) logger.info(f"Scaling out by {blocks} blocks") for _ in range(blocks): block_id = str(self._block_id_counter.get_id()) @@ -220,6 +231,13 @@ def scale_in(self, blocks: int) -> List[str]: pass def _launch_block(self, block_id: str) -> Any: + + # there's no static type guarantee that there is a provider here but + # the code assumes there is, so to pass type checking, this assert + # will catch violations of that assumption, that otherwise would appear + # in later references to self.provider + assert self.provider is not None + launch_cmd = self._get_launch_command(block_id) job_name = f"parsl.{self.label}.block-{block_id}" logger.debug("Submitting to provider with job_name %s", job_name) diff --git a/parsl/executors/threads.py b/parsl/executors/threads.py index 97c26adb9e..04e30db06e 100644 --- a/parsl/executors/threads.py +++ b/parsl/executors/threads.py @@ -2,7 +2,7 @@ import typeguard import concurrent.futures as cf -from typing import List, Optional +from typing import Optional, Sequence from parsl.data_provider.staging import Staging from parsl.executors.base import ParslExecutor @@ -28,7 +28,7 @@ class ThreadPoolExecutor(ParslExecutor, RepresentationMixin): @typeguard.typechecked def __init__(self, label: str = 'threads', max_threads: Optional[int] = 2, - thread_name_prefix: str = '', storage_access: Optional[List[Staging]] = None, + thread_name_prefix: str = '', storage_access: Optional[Sequence[Staging]] = None, working_dir: Optional[str] = None): ParslExecutor.__init__(self) self.label = label diff --git a/parsl/executors/workqueue/executor.py b/parsl/executors/workqueue/executor.py index 6502299415..2d5be081b6 100644 --- a/parsl/executors/workqueue/executor.py +++ b/parsl/executors/workqueue/executor.py @@ -6,7 +6,7 @@ import threading import multiprocessing import logging -from concurrent.futures import Future +from parsl.executors.base import FutureWithTaskID from ctypes import c_bool import tempfile @@ -34,7 +34,7 @@ from parsl.utils import setproctitle import typeguard -from typing import Dict, List, Optional, Set, Union +from typing import Dict, List, Optional, Sequence, Set, Union from parsl.data_provider.staging import Staging from .errors import WorkQueueTaskFailure @@ -228,7 +228,7 @@ def __init__(self, port: int = WORK_QUEUE_DEFAULT_PORT, env: Optional[Dict] = None, shared_fs: bool = False, - storage_access: Optional[List[Staging]] = None, + storage_access: Optional[Sequence[Staging]] = None, use_cache: bool = False, source: bool = False, pack: bool = False, @@ -469,8 +469,7 @@ def submit(self, func, resource_specification, *args, **kwargs): input_files.append(self._register_file(maybe_file)) # Create a Future object and have it be mapped from the task ID in the tasks dictionary - fu = Future() - fu.parsl_executor_task_id = executor_task_id + fu = FutureWithTaskID(str(executor_task_id)) logger.debug("Getting tasks_lock to set WQ-level task entry") with self.tasks_lock: logger.debug("Got tasks_lock to set WQ-level task entry") @@ -656,12 +655,20 @@ def initialize_scaling(self): logger.debug("Starting WorkQueueExecutor with provider: %s", self.provider) self._patch_providers() - if hasattr(self.provider, 'init_blocks'): + # self.provider always has init_blocks - this check only needs to + # check that there is actually a provider specified. + if self.provider is not None: try: self.scale_out(blocks=self.provider.init_blocks) except Exception as e: logger.error("Initial block scaling out failed: {}".format(e)) raise e + # if hasattr(self.provider, 'init_blocks'): + # try: + # self.scale_out(blocks=self.provider.init_blocks) + # except Exception as e: + # logger.error("Initial block scaling out failed: {}".format(e)) + # raise e @property def outstanding(self) -> int: @@ -698,7 +705,9 @@ def shutdown(self, *args, **kwargs): collector thread, which shuts down the Work Queue system submission. """ logger.debug("Work Queue shutdown started") - self.should_stop.value = True + self.should_stop.value = True # type: ignore[attr-defined] + # issue https://github.com/python/typeshed/issues/8799 + # from mypy 0.981 onwards # Remove the workers that are still going kill_ids = [self.blocks[block] for block in self.blocks.keys()] @@ -719,7 +728,9 @@ def _collect_work_queue_results(self): """ logger.debug("Starting Collector Thread") try: - while not self.should_stop.value: + while not self.should_stop.value: # type: ignore[attr-defined] + # issue https://github.com/python/typeshed/issues/8799 + # from mypy 0.981 onwards if not self.submit_process.is_alive(): raise ExecutorError(self, "Workqueue Submit Process is not alive") diff --git a/parsl/jobs/error_handlers.py b/parsl/jobs/error_handlers.py index ca243fb694..249cec32d5 100644 --- a/parsl/jobs/error_handlers.py +++ b/parsl/jobs/error_handlers.py @@ -5,6 +5,7 @@ import parsl.executors.status_handling as status_handling from parsl.jobs.states import JobStatus, JobState from parsl.jobs.errors import TooManyJobFailuresError +from parsl.providers.base import ExecutionProvider def noop_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3) -> None: @@ -13,14 +14,14 @@ def noop_error_handler(executor: status_handling.BlockProviderExecutor, status: def simple_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3) -> None: (total_jobs, failed_jobs) = _count_jobs(status) - if hasattr(executor.provider, "init_blocks"): + if isinstance(executor.provider, ExecutionProvider): threshold = max(1, executor.provider.init_blocks) if total_jobs >= threshold and failed_jobs == total_jobs: executor.set_bad_state_and_fail_all(_get_error(status)) -def windowed_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): +def windowed_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3) -> None: sorted_status = [(key, status[key]) for key in sorted(status, key=lambda x: int(x))] current_window = dict(sorted_status[-threshold:]) total, failed = _count_jobs(current_window) diff --git a/parsl/jobs/job_status_poller.py b/parsl/jobs/job_status_poller.py index 50ea3a6332..c1a1486fbd 100644 --- a/parsl/jobs/job_status_poller.py +++ b/parsl/jobs/job_status_poller.py @@ -2,7 +2,7 @@ import parsl import time import zmq -from typing import Dict, List, Sequence, Optional +from typing import Any, cast, Dict, List, Sequence, Optional from parsl.jobs.states import JobStatus, JobState from parsl.jobs.strategy import Strategy @@ -72,13 +72,18 @@ def status(self) -> Dict[str, JobStatus]: def executor(self) -> BlockProviderExecutor: return self._executor - def scale_in(self, n, max_idletime=None): + def scale_in(self, n: int, max_idletime: Optional[float] = None) -> List[str]: # this is a HighThroughputExecutor-specific interface violation if max_idletime is None: block_ids = self._executor.scale_in(n) else: - block_ids = self._executor.scale_in(n, max_idletime=max_idletime) + # this cast is because ParslExecutor.scale_in doesn't have force or max_idletime parameters + # so we just hope that the actual executor happens to have them. + # see some notes in ParslExecutor about making the status handling superclass into a + # class that holds all the scaling methods, so that everything can be specialised + # to work on those. + block_ids = cast(Any, self._executor).scale_in(n, max_idletime=max_idletime) if block_ids is not None: new_status = {} for block_id in block_ids: @@ -87,14 +92,20 @@ def scale_in(self, n, max_idletime=None): self.send_monitoring_info(new_status) return block_ids - def scale_out(self, n): + def scale_out(self, n: int) -> List[str]: + logger.debug("BENC: in task status scale out") block_ids = self._executor.scale_out(n) - if block_ids is not None: - new_status = {} - for block_id in block_ids: - new_status[block_id] = JobStatus(JobState.PENDING) - self.send_monitoring_info(new_status) - self._status.update(new_status) + logger.debug("BENC: executor scale out has returned") + + # mypy - remove this if statement: block_ids is always a list according to the types. + # and so the else clause was failing with unreachable code. And this removed `if` + # would always fire, if that type annotation is true. + logger.debug(f"BENC: there were some block ids, {block_ids}, which will now be set to pending") + new_status = {} + for block_id in block_ids: + new_status[block_id] = JobStatus(JobState.PENDING) + self.send_monitoring_info(new_status) + self._status.update(new_status) return block_ids def __repr__(self) -> str: diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index c9cd3c407e..6749f888c8 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -7,11 +7,23 @@ import parsl.jobs.job_status_poller as jsp +from typing import Callable + +# this is used for testing a class to decide how to +# print a status line. That might be better done inside +# the executor class (i..e put the class specific behaviour +# inside the class, rather than testing class instance-ness +# here) + +# smells: testing class instance; importing a specific instance +# of a thing that should be generic + + from parsl.executors import HighThroughputExecutor from parsl.executors.base import ParslExecutor -from parsl.executors.status_handling import BlockProviderExecutor +from parsl.executors.base import HasWorkersPerNode, HasOutstanding from parsl.jobs.states import JobState -from parsl.process_loggers import wrap_with_logs +# from parsl.process_loggers import wrap_with_logs logger = logging.getLogger(__name__) @@ -129,15 +141,23 @@ def __init__(self, *, strategy: Optional[str], max_idletime: float) -> None: self.executors = {} self.max_idletime = max_idletime + self.strategies: Dict[Optional[str], Callable] self.strategies = {None: self._strategy_noop, 'none': self._strategy_noop, 'simple': self._strategy_simple, - 'htex_auto_scale': self._strategy_htex_auto_scale} + 'htex_auto_scale': self._strategy_htex_auto_scale + } if strategy is None: warnings.warn("literal None for strategy choice is deprecated. Use string 'none' instead.", DeprecationWarning) + # mypy note: with mypy 0.761, the type of self.strategize is + # correctly revealed inside this module, but isn't carried over + # when Strategy is used in other modules unless this specific + # type annotation is used. + + self.strategize: Callable self.strategize = self.strategies[strategy] logger.debug("Scaling strategy: {0}".format(strategy)) @@ -172,28 +192,33 @@ def _strategy_htex_auto_scale(self, status_list: List[jsp.PollItem]) -> None: """ self._general_strategy(status_list, strategy_type='htex') - @wrap_with_logs - def _general_strategy(self, status_list, *, strategy_type): + # can't do wrap with logs until I learn about paramspecs, because wrap_with_logs + # is not tightly typed enough to be allowed in this module yet. + # @wrap_with_logs + def _general_strategy(self, status_list: List[jsp.PollItem], strategy_type: str) -> None: logger.debug(f"general strategy starting with strategy_type {strategy_type} for {len(status_list)} executors") for exec_status in status_list: executor = exec_status.executor label = executor.label - if not isinstance(executor, BlockProviderExecutor): - logger.debug(f"Not strategizing for executor {label} because scaling not enabled") - continue logger.debug(f"Strategizing for executor {label}") # Tasks that are either pending completion + assert isinstance(executor, HasOutstanding) active_tasks = executor.outstanding status = exec_status.status + # The provider might not even be defined -- what's the behaviour in + # that case? + if executor.provider is None: + logger.error("Trying to strategize an executor that has no provider") + continue + # FIXME we need to handle case where provider does not define these # FIXME probably more of this logic should be moved to the provider min_blocks = executor.provider.min_blocks max_blocks = executor.provider.max_blocks - tasks_per_node = executor.workers_per_node nodes_per_block = executor.provider.nodes_per_block parallelism = executor.provider.parallelism @@ -201,9 +226,16 @@ def _general_strategy(self, status_list, *, strategy_type): running = sum([1 for x in status.values() if x.state == JobState.RUNNING]) pending = sum([1 for x in status.values() if x.state == JobState.PENDING]) active_blocks = running + pending - active_slots = active_blocks * tasks_per_node * nodes_per_block - logger.debug(f"Slot ratio calculation: active_slots = {active_slots}, active_tasks = {active_tasks}") + # TODO: if this isinstance doesn't fire, tasks_per_node and active_slots won't be + # set this iteration and either will be unset or will contain a previous executor's value. + # in both cases, this is wrong. but apparently mypy doesn't notice. + + if isinstance(executor, HasWorkersPerNode): + tasks_per_node = executor.workers_per_node + + active_slots = active_blocks * tasks_per_node * nodes_per_block + logger.debug(f"Slot ratio calculation: active_slots = {active_slots}, active_tasks = {active_tasks}") logger.debug('Executor {} has {} active tasks and {}/{} running/pending blocks'.format( label, active_tasks, running, pending)) @@ -233,9 +265,18 @@ def _general_strategy(self, status_list, *, strategy_type): logger.debug(f"Starting idle timer for executor. If idle time exceeds {self.max_idletime}s, blocks will be scaled in") self.executors[executor.label]['idle_since'] = time.time() + # ... this could be None, type-wise. So why aren't we seeing errors here? + # probably becaues usually if this is None, it will be because active_tasks>0, + # (although I can't see a clear proof that this will always be the case: + # could that setting to None have happened on a previous iteration?) + + # if idle_since is None, then that means not idle, which means should not + # go down the scale_in path idle_since = self.executors[executor.label]['idle_since'] - idle_duration = time.time() - idle_since - if idle_duration > self.max_idletime: + if idle_since is not None and (time.time() - idle_since) > self.max_idletime: + # restored this separate calculation even though making a single one + # ahead of time is better... + idle_duration = time.time() - idle_since # We have resources idle for the max duration, # we have to scale_in now. logger.debug(f"Idle time has reached {self.max_idletime}s for executor {label}; scaling in") diff --git a/parsl/monitoring/monitoring.py b/parsl/monitoring/monitoring.py index 893f2714fe..009f493e0c 100644 --- a/parsl/monitoring/monitoring.py +++ b/parsl/monitoring/monitoring.py @@ -9,12 +9,10 @@ import zmq import queue - import parsl.monitoring.remote -from parsl.multiprocessing import ForkProcess, SizedQueue -from multiprocessing import Process -from multiprocessing.queues import Queue +from parsl.multiprocessing import forkProcess, sizedQueue +from multiprocessing import Queue from parsl.utils import RepresentationMixin from parsl.process_loggers import wrap_with_logs from parsl.utils import setproctitle @@ -134,7 +132,8 @@ def __init__(self, Default: 30 seconds """ - self.logger = logger + # previously this was set in start() but logger exists at import so it can be set here and remove the optionality of self.logger's type + self.logger = logger # type: logging.Logger # Any is used to disable typechecking on uses of _dfk_channel, # because it is used in the code as if it points to a channel, but @@ -187,24 +186,24 @@ def start(self, run_id: str, run_dir: str) -> int: else: comm_q: Queue - comm_q = SizedQueue(maxsize=10) + comm_q = sizedQueue(maxsize=10) self.exception_q: Queue[Tuple[str, str]] - self.exception_q = SizedQueue(maxsize=10) + self.exception_q = sizedQueue(maxsize=10) self.priority_msgs: Queue[Tuple[Any, int]] - self.priority_msgs = SizedQueue() + self.priority_msgs = sizedQueue() self.resource_msgs: Queue[AddressedMonitoringMessage] - self.resource_msgs = SizedQueue() + self.resource_msgs = sizedQueue() self.node_msgs: Queue[AddressedMonitoringMessage] - self.node_msgs = SizedQueue() + self.node_msgs = sizedQueue() self.block_msgs: Queue[AddressedMonitoringMessage] - self.block_msgs = SizedQueue() + self.block_msgs = sizedQueue() - self.router_proc = ForkProcess(target=router_starter, + self.router_proc = forkProcess(target=router_starter, args=(comm_q, self.exception_q, self.priority_msgs, self.node_msgs, self.block_msgs, self.resource_msgs), kwargs={"hub_address": self.hub_address, "hub_port": self.hub_port, @@ -218,7 +217,7 @@ def start(self, run_id: str, run_dir: str) -> int: ) self.router_proc.start() - self.dbm_proc = ForkProcess(target=dbm_starter, + self.dbm_proc = forkProcess(target=dbm_starter, args=(self.exception_q, self.priority_msgs, self.node_msgs, self.block_msgs, self.resource_msgs,), kwargs={"logdir": self.logdir, "logging_level": logging.DEBUG if self.monitoring_debug else logging.INFO, @@ -230,11 +229,11 @@ def start(self, run_id: str, run_dir: str) -> int: self.dbm_proc.start() self.logger.info("Started the router process {} and DBM process {}".format(self.router_proc.pid, self.dbm_proc.pid)) - self.filesystem_proc = Process(target=filesystem_receiver, - args=(self.logdir, self.resource_msgs, run_dir), - name="Monitoring-Filesystem-Process", - daemon=True - ) + self.filesystem_proc = forkProcess(target=filesystem_receiver, + args=(self.logdir, self.resource_msgs, run_dir), + name="Monitoring-Filesystem-Process", + daemon=True + ) self.filesystem_proc.start() self.logger.info(f"Started filesystem radio receiver process {self.filesystem_proc.pid}") diff --git a/parsl/multiprocessing.py b/parsl/multiprocessing.py index a2dfdd3fe0..9e2ade5ebd 100644 --- a/parsl/multiprocessing.py +++ b/parsl/multiprocessing.py @@ -6,8 +6,6 @@ import multiprocessing.queues import platform -from typing import Callable, Type - logger = logging.getLogger(__name__) ForkContext = multiprocessing.get_context("fork") @@ -15,7 +13,20 @@ # maybe ForkProcess should be: Callable[..., Process] so as to make # it clear that it returns a Process always to the type checker? -ForkProcess: Type = ForkContext.Process +ForkProcess = multiprocessing.context.ForkProcess +SpawnProcess = multiprocessing.context.SpawnProcess + + +def forkProcess(*args, **kwargs) -> ForkProcess: + P = multiprocessing.get_context('fork').Process + # reveal_type(P) + return P(*args, **kwargs) + + +def spawnProcess(*args, **kwargs) -> SpawnProcess: + P = multiprocessing.get_context('spawn').Process + # reveal_type(P) + return P(*args, **kwargs) class MacSafeQueue(multiprocessing.queues.Queue): @@ -56,11 +67,10 @@ def empty(self): # SizedQueue should be constructable using the same calling # convention as multiprocessing.Queue but that entire signature # isn't expressible in mypy 0.790 -SizedQueue: Callable[..., multiprocessing.Queue] - +# SizedQueue: Callable[..., multiprocessing.Queue] -if platform.system() != 'Darwin': - import multiprocessing - SizedQueue = multiprocessing.Queue -else: - SizedQueue = MacSafeQueue +def sizedQueue(*args, **kwargs) -> multiprocessing.queues.Queue: + if platform.system() != 'Darwin': + return multiprocessing.Queue(*args, **kwargs) + else: + return MacSafeQueue(*args, **kwargs) diff --git a/parsl/providers/ad_hoc/ad_hoc.py b/parsl/providers/ad_hoc/ad_hoc.py index 63784998ab..1accbce7a7 100644 --- a/parsl/providers/ad_hoc/ad_hoc.py +++ b/parsl/providers/ad_hoc/ad_hoc.py @@ -5,14 +5,16 @@ from parsl.channels import LocalChannel from parsl.jobs.states import JobStatus, JobState from parsl.launchers import SimpleLauncher -from parsl.providers.base import ExecutionProvider +from parsl.providers.base import ExecutionProvider, MultiChanneled from parsl.providers.errors import ScriptPathError from parsl.utils import RepresentationMixin +from typing import Dict, Any, List + logger = logging.getLogger(__name__) -class AdHocProvider(ExecutionProvider, RepresentationMixin): +class AdHocProvider(ExecutionProvider, MultiChanneled, RepresentationMixin): """ Ad-hoc execution provider This provider is used to provision execution resources over one or more ad hoc nodes @@ -62,6 +64,7 @@ def __init__(self, self.nodes_per_block = 1 # Dictionary that keeps track of jobs, keyed on job_id + self.resources: Dict[Any, Dict[str, Any]] self.resources = {} self.least_loaded = self._least_loaded() @@ -184,15 +187,17 @@ def submit(self, command, tasks_per_node, job_name="parsl.adhoc"): if job_id is None: logger.warning("Channel failed to start remote command/retrieve PID") - self.resources[job_id] = {'job_id': job_id, - 'status': JobStatus(JobState.RUNNING), - 'cmd': final_cmd, - 'channel': channel, - 'remote_pid': remote_pid} + d = {'job_id': job_id, + 'status': JobStatus(JobState.RUNNING), + 'cmd': final_cmd, + 'channel': channel, + 'remote_pid': remote_pid} # type: Dict[str, Any] + + self.resources[job_id] = d return job_id - def status(self, job_ids): + def status(self, job_ids: List[Any]) -> List[JobStatus]: """ Get status of the list of jobs with job_ids Parameters diff --git a/parsl/providers/aws/aws.py b/parsl/providers/aws/aws.py index e3217dfd3e..75981ae4e1 100644 --- a/parsl/providers/aws/aws.py +++ b/parsl/providers/aws/aws.py @@ -3,6 +3,7 @@ import os import time from string import Template +from typing import List from parsl.errors import ConfigurationError from parsl.jobs.states import JobState, JobStatus @@ -600,7 +601,7 @@ def get_instance_state(self, instances=None): self.instance_states[instance['InstanceId']] = instance['State']['Name'] return self.instance_states - def status(self, job_ids): + def status(self, job_ids) -> List[JobStatus]: """Get the status of a list of jobs identified by their ids. Parameters diff --git a/parsl/providers/azure/azure.py b/parsl/providers/azure/azure.py index b33e9be237..80663b713d 100644 --- a/parsl/providers/azure/azure.py +++ b/parsl/providers/azure/azure.py @@ -3,6 +3,7 @@ import os import time from string import Template +from typing import Any, List from parsl.errors import ConfigurationError from parsl.jobs.states import JobState, JobStatus @@ -157,6 +158,8 @@ def __init__(self, self.launcher = launcher self.linger = linger self.resources = {} + + self.instances: List[Any] self.instances = [] env_specified = os.getenv("AZURE_CLIENT_ID") is not None and os.getenv( @@ -300,7 +303,7 @@ def submit(self, return vm_info.name - def status(self, job_ids): + def status(self, job_ids) -> List[JobStatus]: """Get the status of a list of jobs identified by their ids. Parameters ---------- diff --git a/parsl/providers/base.py b/parsl/providers/base.py index e4c7355445..edba6f49f2 100644 --- a/parsl/providers/base.py +++ b/parsl/providers/base.py @@ -1,6 +1,6 @@ from abc import ABCMeta, abstractmethod, abstractproperty import logging -from typing import Any, Dict, List, Optional +from typing import Any, Dict, List, Optional, Sequence from parsl.channels.base import Channel from parsl.jobs.states import JobStatus @@ -49,6 +49,9 @@ def __init__(self) -> None: self._mem_per_node: Optional[float] = None pass + # TODO: how about make this always return a job ID and must raise an exception on + # failure? Potentially it could make failure path handling simpler? because right now, you + # should catch an exception or look for None and handle them both the same? @abstractmethod def submit(self, command: str, tasks_per_node: int, job_name: str = "parsl.auto") -> object: ''' The submit method takes the command string to be executed upon @@ -69,6 +72,7 @@ def submit(self, command: str, tasks_per_node: int, job_name: str = "parsl.auto" Raises: - ExecutionProviderException or its subclasses + ^ is this true? I think we can raise anything... ''' pass @@ -92,7 +96,7 @@ def status(self, job_ids: List[object]) -> List[JobStatus]: pass @abstractmethod - def cancel(self, job_ids: List[object]) -> List[bool]: + def cancel(self, job_ids: Sequence[object]) -> Sequence[bool]: ''' Cancels the resources identified by the job_ids provided by the user. Args: @@ -122,8 +126,14 @@ def mem_per_node(self) -> Optional[float]: If this property is set, executors may use it to calculate how many tasks can run concurrently per node. + + This property, and cores_per_node, might become a HasCoresMem protocol, on + the way to detangling what is optional? """ - return self._mem_per_node + if hasattr(self, "_mem_per_node"): + return self._mem_per_node + else: + return None @mem_per_node.setter def mem_per_node(self, value: float) -> None: @@ -140,7 +150,10 @@ def cores_per_node(self) -> Optional[int]: If this property is set, executors may use it to calculate how many tasks can run concurrently per node. """ - return self._cores_per_node + if hasattr(self, "_cores_per_node"): + return self._cores_per_node + else: + return None @cores_per_node.setter def cores_per_node(self, value: int) -> None: diff --git a/parsl/providers/cluster_provider.py b/parsl/providers/cluster_provider.py index b7d48e39ba..807ac95f2b 100644 --- a/parsl/providers/cluster_provider.py +++ b/parsl/providers/cluster_provider.py @@ -2,15 +2,19 @@ from abc import abstractmethod from string import Template +from parsl.jobs.states import JobStatus from parsl.providers.errors import SchedulerMissingArgs, ScriptPathError from parsl.launchers.base import Launcher from parsl.launchers.errors import BadLauncher -from parsl.providers.base import ExecutionProvider +from parsl.providers.base import ExecutionProvider, Channeled logger = logging.getLogger(__name__) +from typing import Any, Dict, List, Tuple +from parsl.channels.base import Channel -class ClusterProvider(ExecutionProvider): + +class ClusterProvider(ExecutionProvider, Channeled): """ This class defines behavior common to all cluster/supercompute-style scheduler systems. Parameters @@ -46,16 +50,16 @@ class ClusterProvider(ExecutionProvider): """ def __init__(self, - label, - channel, - nodes_per_block, - init_blocks, - min_blocks, - max_blocks, - parallelism, - walltime, - launcher, - cmd_timeout=10): + label: str, + channel: Channel, + nodes_per_block: int, + init_blocks: int, + min_blocks: int, + max_blocks: int, + parallelism: float, # nb. the member field for this is used by strategy, so maybe this should be exposed at the layer above as a property? + walltime: str, + launcher: Launcher, + cmd_timeout: int = 10) -> None: self._label = label self.channel = channel @@ -73,9 +77,18 @@ def __init__(self, self.script_dir = None # Dictionary that keeps track of jobs, keyed on job_id + self.resources: Dict[Any, Any] self.resources = {} - def execute_wait(self, cmd, timeout=None): + # This annotation breaks slurm: + # parsl/providers/slurm/slurm.py:201: error: Item "None" of "Optional[str]" has no attribute "split" + # parsl/providers/slurm/slurm.py:207: error: Item "None" of "Optional[str]" has no attribute "strip" + # Theres a dependent type at work here which I can't describe in the type system: + # the optional strs are None when int != 0, for some providers. + # and when int == 0, the optional strs are strs + + def execute_wait(self, cmd, timeout=None) -> Tuple[int, str, str]: + t = self.cmd_timeout if timeout is not None: t = timeout @@ -123,7 +136,7 @@ def _write_submit_script(self, template, script_filename, job_name, configs): def _status(self): pass - def status(self, job_ids): + def status(self, job_ids: List[Any]) -> List[JobStatus]: """ Get the status of a list of jobs identified by the job identifiers returned from the submit request. diff --git a/parsl/providers/condor/condor.py b/parsl/providers/condor/condor.py index af875db457..63f8b5a73e 100644 --- a/parsl/providers/condor/condor.py +++ b/parsl/providers/condor/condor.py @@ -15,7 +15,7 @@ logger = logging.getLogger(__name__) -from typing import Dict, List, Optional +from typing import Any, Dict, List, Optional from parsl.channels.base import Channel # See http://pages.cs.wisc.edu/~adesmet/status.html @@ -155,7 +155,7 @@ def _status(self): if job_id in self.resources: self.resources[job_id]['status'] = JobStatus(state) - def status(self, job_ids): + def status(self, job_ids: List[Any]) -> List[JobStatus]: """Get the status of a list of jobs identified by their ids. Parameters @@ -261,7 +261,7 @@ def submit(self, command, tasks_per_node, job_name="parsl.condor"): except Exception as e: raise ScaleOutFailed(self.label, str(e)) - job_id = [] + job_id = [] # type: List[str] if retcode == 0: for line in stdout.split('\n'): diff --git a/parsl/providers/googlecloud/googlecloud.py b/parsl/providers/googlecloud/googlecloud.py index 74ebe44a82..9992a4aee2 100644 --- a/parsl/providers/googlecloud/googlecloud.py +++ b/parsl/providers/googlecloud/googlecloud.py @@ -1,6 +1,7 @@ import atexit import logging import os +from typing import List from parsl.launchers import SingleNodeLauncher from parsl.jobs.states import JobState, JobStatus @@ -118,7 +119,7 @@ def submit(self, command, tasks_per_node, job_name="parsl.gcs"): self.resources[name] = {"job_id": name, "status": JobStatus(translate_table[instance['status']])} return name - def status(self, job_ids): + def status(self, job_ids) -> List[JobStatus]: ''' Get the status of a list of jobs identified by the job identifiers returned from the submit request. diff --git a/parsl/providers/kubernetes/kube.py b/parsl/providers/kubernetes/kube.py index aa0e335679..6f2f0f0e1c 100644 --- a/parsl/providers/kubernetes/kube.py +++ b/parsl/providers/kubernetes/kube.py @@ -132,7 +132,7 @@ def __init__(self, self.resources: Dict[object, Dict[str, Any]] self.resources = {} - def submit(self, cmd_string, tasks_per_node, job_name="parsl"): + def submit(self, cmd_string, tasks_per_node, job_name="parsl") -> Optional[str]: """ Submit a job Args: - cmd_string :(String) - Name of the container to initiate @@ -168,7 +168,7 @@ def submit(self, cmd_string, tasks_per_node, job_name="parsl"): return pod_name - def status(self, job_ids): + def status(self, job_ids) -> List[JobStatus]: """ Get the status of a list of jobs identified by the job identifiers returned from the submit request. Args: @@ -182,7 +182,7 @@ def status(self, job_ids): self._status() return [self.resources[jid]['status'] for jid in job_ids] - def cancel(self, job_ids): + def cancel(self, job_ids) -> List[bool]: """ Cancels the jobs specified by a list of job ids Args: job_ids : [ ...] @@ -232,7 +232,7 @@ def _create_pod(self, job_name, port=80, cmd_string=None, - volumes=[]): + volumes=[]) -> None: """ Create a kubernetes pod for the job. Args: - image (string) : Docker image to launch @@ -301,7 +301,7 @@ def _create_pod(self, body=pod) logger.debug("Pod created. status='{0}'".format(str(api_response.status))) - def _delete_pod(self, pod_name): + def _delete_pod(self, pod_name) -> None: """Delete a pod""" api_response = self.kube_client.delete_namespaced_pod(name=pod_name, @@ -310,7 +310,7 @@ def _delete_pod(self, pod_name): logger.debug("Pod deleted. status='{0}'".format(str(api_response.status))) @property - def label(self): + def label(self) -> str: return "kubernetes" @property diff --git a/parsl/providers/local/local.py b/parsl/providers/local/local.py index d69e531bfd..ec6654b0f7 100644 --- a/parsl/providers/local/local.py +++ b/parsl/providers/local/local.py @@ -2,17 +2,21 @@ import os import time +from parsl.channels.base import Channel from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher -from parsl.providers.base import ExecutionProvider +from parsl.providers.base import Channeled, ExecutionProvider from parsl.providers.errors import SchedulerMissingArgs, ScriptPathError, SubmitException from parsl.utils import RepresentationMixin +from typing import Any, List + logger = logging.getLogger(__name__) -class LocalProvider(ExecutionProvider, RepresentationMixin): +class LocalProvider(ExecutionProvider, RepresentationMixin, Channeled): + """ Local Execution Provider This provider is used to provide execution resources from the localhost. @@ -36,7 +40,7 @@ class LocalProvider(ExecutionProvider, RepresentationMixin): """ def __init__(self, - channel=LocalChannel(), + channel: Channel = LocalChannel(), nodes_per_block=1, launcher=SingleNodeLauncher(), init_blocks=1, @@ -62,7 +66,7 @@ def __init__(self, # Dictionary that keeps track of jobs, keyed on job_id self.resources = {} - def status(self, job_ids): + def status(self, job_ids: List[Any]) -> List[JobStatus]: ''' Get the status of a list of jobs identified by their ids. Args: @@ -119,22 +123,32 @@ def status(self, job_ids): return [self.resources[jid]['status'] for jid in job_ids] - def _is_alive(self, job_dict): + def _is_alive(self, job_dict) -> bool: retcode, stdout, stderr = self.channel.execute_wait( 'ps -p {} > /dev/null 2> /dev/null; echo "STATUS:$?" '.format( job_dict['remote_pid']), self.cmd_timeout) - for line in stdout.split('\n'): - if line.startswith("STATUS:"): - status = line.split("STATUS:")[1].strip() - if status == "0": - return True - else: - return False + if stdout: + for line in stdout.split('\n'): + if line.startswith("STATUS:"): + status = line.split("STATUS:")[1].strip() + if status == "0": + return True + else: + return False + raise RuntimeError("Hit end of stdout scan without finding STATUS. Unclear what the correct default behaviour is here, so raising exception") + else: + raise RuntimeError("no stdout. Unclear what the correct default behaviour is here, so raising exception.") def _job_file_path(self, script_path: str, suffix: str) -> str: path = '{0}{1}'.format(script_path, suffix) if self._should_move_files(): - path = self.channel.pull_file(path, self.script_dir) + if not self.script_dir: + raise RuntimeError("want to pull_file but script_dir is not defined - unclear what the correct behaviour is so raising exception") + new_path = self.channel.pull_file(path, self.script_dir) + if path is None: + raise RuntimeError("pull_file returned None - unclear what the correct behaviour is so raising exception") + else: + path = new_path return path def _read_job_file(self, script_path: str, suffix: str) -> str: @@ -174,7 +188,7 @@ def _write_submit_script(self, script_string, script_filename): return True - def submit(self, command, tasks_per_node, job_name="parsl.localprovider"): + def submit(self, command: str, tasks_per_node: int, job_name: str = "parsl.localprovider") -> object: ''' Submits the command onto an Local Resource Manager job. Submit returns an ID that corresponds to the task that was just submitted. @@ -210,7 +224,7 @@ def submit(self, command, tasks_per_node, job_name="parsl.localprovider"): self._write_submit_script(wrap_command, script_path) - job_id = None + job_id = None # type: Any remote_pid = None if self._should_move_files(): logger.debug("Pushing start script") @@ -238,10 +252,13 @@ def submit(self, command, tasks_per_node, job_name="parsl.localprovider"): if retcode != 0: raise SubmitException(job_name, "Launch command exited with code {0}".format(retcode), stdout, stderr) - for line in stdout.split('\n'): - if line.startswith("PID:"): - remote_pid = line.split("PID:")[1].strip() - job_id = remote_pid + if stdout: + for line in stdout.split('\n'): + if line.startswith("PID:"): + remote_pid = line.split("PID:")[1].strip() + job_id = remote_pid + else: + logger.debug("no stdout, which would caused a runtime type error splitting stdout. Acting as if stdout has no lines.") if job_id is None: raise SubmitException(job_name, "Channel failed to start remote command/retrieve PID") diff --git a/parsl/providers/slurm/slurm.py b/parsl/providers/slurm/slurm.py index 61d8bdd2ca..42a1a2e61d 100644 --- a/parsl/providers/slurm/slurm.py +++ b/parsl/providers/slurm/slurm.py @@ -17,6 +17,8 @@ from parsl.providers.slurm.template import template_string from parsl.utils import RepresentationMixin, wtime_to_minutes +from typing import Any, Dict + logger = logging.getLogger(__name__) translate_table = { @@ -231,6 +233,7 @@ def submit(self, command: str, tasks_per_node: int, job_name="parsl.slurm") -> s logger.debug("Requesting one block with {} nodes".format(self.nodes_per_block)) + job_config: Dict[str, Any] job_config = {} job_config["submit_script_dir"] = self.channel.script_dir job_config["nodes"] = self.nodes_per_block diff --git a/parsl/providers/torque/torque.py b/parsl/providers/torque/torque.py index bf170c8d40..f0325e3011 100644 --- a/parsl/providers/torque/torque.py +++ b/parsl/providers/torque/torque.py @@ -11,6 +11,10 @@ logger = logging.getLogger(__name__) +from typing import Optional +from parsl.channels.base import Channel +from parsl.launchers.base import Launcher + # From the man pages for qstat for PBS/Torque systems translate_table = { 'B': JobState.RUNNING, # This state is returned for running array jobs @@ -68,19 +72,19 @@ class TorqueProvider(ClusterProvider, RepresentationMixin): """ def __init__(self, - channel=LocalChannel(), - account=None, - queue=None, - scheduler_options='', - worker_init='', - nodes_per_block=1, - init_blocks=1, - min_blocks=0, - max_blocks=1, - parallelism=1, - launcher=AprunLauncher(), - walltime="00:20:00", - cmd_timeout=120): + channel: Channel = LocalChannel(), + account: Optional[str] = None, + queue: Optional[str] = None, + scheduler_options: str = '', + worker_init: str = '', + nodes_per_block: int = 1, + init_blocks: int = 1, + min_blocks: int = 0, + max_blocks: int = 1, + parallelism: float = 1, + launcher: Launcher = AprunLauncher(), + walltime: str = "00:20:00", + cmd_timeout: int = 120) -> None: label = 'torque' super().__init__(label, channel, diff --git a/parsl/tests/manual_tests/test_worker_count.py b/parsl/tests/manual_tests/test_worker_count.py index cf3a89f262..1a71fc552e 100644 --- a/parsl/tests/manual_tests/test_worker_count.py +++ b/parsl/tests/manual_tests/test_worker_count.py @@ -14,8 +14,10 @@ from parsl.tests.manual_tests.htex_local import config from parsl.executors import HighThroughputExecutor +from parsl.providers.base import ExecutionProvider assert isinstance(config.executors[0], HighThroughputExecutor) config.executors[0].cores_per_worker = CORES_PER_WORKER +assert isinstance(config.executors[0].provider, ExecutionProvider) config.executors[0].provider.init_blocks = 1 # from htex_midway import config diff --git a/parsl/tests/test_checkpointing/test_periodic.py b/parsl/tests/test_checkpointing/test_periodic.py index 6fae6246a0..469ed5579e 100644 --- a/parsl/tests/test_checkpointing/test_periodic.py +++ b/parsl/tests/test_checkpointing/test_periodic.py @@ -7,6 +7,8 @@ from parsl.app.app import python_app from parsl.tests.configs.local_threads_checkpoint_periodic import config +dfk: parsl.DataFlowKernel + def local_setup(): global dfk @@ -36,6 +38,7 @@ def tstamp_to_seconds(line): def test_periodic(n=4): """Test checkpointing with task_periodic behavior """ + global dfk d = {} diff --git a/parsl/tests/test_python_apps/test_simple.py b/parsl/tests/test_python_apps/test_simple.py index 007db7b923..351d525146 100644 --- a/parsl/tests/test_python_apps/test_simple.py +++ b/parsl/tests/test_python_apps/test_simple.py @@ -1,5 +1,9 @@ from parsl.app.app import python_app +from concurrent.futures import Future + +from typing import Dict, Union + @python_app def increment(x): @@ -14,19 +18,22 @@ def slow_increment(x, dur): def test_increment(depth=5): - futs = {0: 0} + futs = {0: 0} # type: Dict[int, Union[int, Future]] for i in range(1, depth): futs[i] = increment(futs[i - 1]) - x = sum([futs[i].result() for i in futs if not isinstance(futs[i], int)]) + # this is a slightly awkward rearrangement: we need to bind f so that mypy + # can take the type property proved by isinstance and carry it over to + # reason about if f.result() valid. + x = sum([f.result() for i in futs for f in [futs[i]] if isinstance(f, Future)]) assert x == sum(range(1, depth)), "[TEST] increment [FAILED]" def test_slow_increment(depth=5): - futs = {0: 0} + futs = {0: 0} # type: Dict[int, Union[int, Future]] for i in range(1, depth): futs[i] = slow_increment(futs[i - 1], 0.01) - x = sum([futs[i].result() for i in futs if not isinstance(futs[i], int)]) + x = sum([f.result() for i in futs for f in [futs[i]] if isinstance(f, Future)]) assert x == sum(range(1, depth)), "[TEST] slow_increment [FAILED]" diff --git a/parsl/usage_tracking/usage.py b/parsl/usage_tracking/usage.py index c584855196..79bbd84d82 100644 --- a/parsl/usage_tracking/usage.py +++ b/parsl/usage_tracking/usage.py @@ -1,3 +1,4 @@ +from __future__ import annotations import uuid import time import os @@ -7,11 +8,15 @@ import sys import platform +from typing import List + +from parsl.multiprocessing import forkProcess, ForkProcess from parsl.utils import setproctitle -from parsl.multiprocessing import ForkProcess from parsl.dataflow.states import States from parsl.version import VERSION as PARSL_VERSION +import parsl.dataflow.dflow # can't import just the symbol for DataFlowKernel because of mutually-recursive imports + logger = logging.getLogger(__name__) from typing import Callable @@ -20,11 +25,15 @@ P = ParamSpec("P") -def async_process(fn: Callable[P, None]) -> Callable[P, None]: +# parsl/usage_tracking/usage.py:36: error: +# Incompatible return value type (got "Callable[[VarArg(Any), KwArg(Any)], ForkProcess]", +# expected "Callable[P, None]") [return-value] + +def async_process(fn: Callable[P, None]) -> Callable[P, ForkProcess]: """ Decorator function to launch a function as a separate process """ - def run(*args, **kwargs): - proc = ForkProcess(target=fn, args=args, kwargs=kwargs, name="Usage-Tracking") + def run(*args, **kwargs) -> ForkProcess: + proc = forkProcess(target=fn, args=args, kwargs=kwargs, name="Usage-Tracking") proc.start() return proc @@ -75,8 +84,10 @@ class UsageTracker: """ - def __init__(self, dfk, port=50077, - domain_name='tracking.parsl-project.org'): + def __init__(self, + dfk: parsl.dataflow.dflow.DataFlowKernel, + port: int = 50077, + domain_name: str = 'tracking.parsl-project.org') -> None: """Initialize usage tracking unless the user has opted-out. We will try to resolve the hostname specified in kwarg:domain_name @@ -99,7 +110,7 @@ def __init__(self, dfk, port=50077, # The sock timeout will only apply to UDP send and not domain resolution self.sock_timeout = 5 self.UDP_PORT = port - self.procs = [] + self.procs: List[ForkProcess] = [] self.dfk = dfk self.config = self.dfk.config self.uuid = str(uuid.uuid4()) diff --git a/parsl/utils.py b/parsl/utils.py index 2718a3983b..a9af9961e2 100644 --- a/parsl/utils.py +++ b/parsl/utils.py @@ -8,6 +8,7 @@ from contextlib import contextmanager from types import TracebackType from typing import Any, Callable, Iterator, List, Sequence, Tuple, Union, Generator, IO, Dict, Optional +from typing_extensions import Protocol, runtime_checkable import typeguard from typing_extensions import Type @@ -173,6 +174,11 @@ def wtime_to_minutes(time_string: str) -> int: return total_mins +@runtime_checkable +class IsWrapper(Protocol): + __wrapped__: Callable + + class RepresentationMixin: """A mixin class for adding a __repr__ method. @@ -203,6 +209,8 @@ def __init__(self, first, second, third='three', fourth='fourth'): def __repr__(self) -> str: init = self.__init__ # type: ignore[misc] + # init: Any # to override something I don't understand with myppy vs the init, iswrapper test below + # init = type(self).__init__ # does this change from self.__init__ work? # This test looks for a single layer of wrapping performed by # functools.update_wrapper, commonly used in decorators. This will @@ -212,7 +220,7 @@ def __repr__(self) -> str: # decorators, or cope with other decorators which do not use # functools.update_wrapper. - if hasattr(init, '__wrapped__'): + if isinstance(init, IsWrapper): init = init.__wrapped__ argspec = inspect.getfullargspec(init) @@ -304,7 +312,8 @@ class Timer: """ - def __init__(self, callback: Callable, *args: Any, interval: int = 5, name: Optional[str] = None) -> None: + # TODO: some kind of dependentish type here? eg Callable[X] and args has type X? + def __init__(self, callback: Callable, *args: Tuple[Any, ...], interval: float = 5, name: Optional[str] = None) -> None: """Initialize the Timer object. We start the timer thread here diff --git a/requirements.txt b/requirements.txt index c74b2b070a..e001e08e91 100644 --- a/requirements.txt +++ b/requirements.txt @@ -8,7 +8,7 @@ typeguard>=2.10,!=3.*,<5 typing-extensions>=4.6,<5 -globus-sdk +globus-sdk>=3,<4 dill tblib requests From 6f5b7a6a67c17af22087ca7c486f0ccb28d8bc21 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 10 May 2023 07:06:27 +0000 Subject: [PATCH 23/25] Add type annotations to common test configurations This should get more type checking to happen inside the test suite. It reveals a number of mypy failures that come from broken (disabled) tests and which need fixing. Perhaps it could drive actually fixing some of those tests? --- mypy.ini | 3 +++ 1 file changed, 3 insertions(+) diff --git a/mypy.ini b/mypy.ini index e083d97bb4..d2e1f99f39 100644 --- a/mypy.ini +++ b/mypy.ini @@ -144,6 +144,9 @@ disallow_untyped_defs = True [mypy-parsl.monitoring.visualization.*] ignore_errors = True +[mypy-parsl.tests] +check_untyped_defs = True + [mypy-parsl.tests.configs.local_user_opts] ignore_missing_imports = True From db1575b5c925477ef0605b60c73cafcd2b108fcb Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 20 Nov 2023 12:49:35 +0000 Subject: [PATCH 24/25] remove None-type from provider API This is awkward to typecheck, because "None" is still a valid return code: it is a job id TODO: this patch should become a documentation patch that changes the docstring only to say that None is deprecated, and cross-references the relevant github issue. --- parsl/providers/base.py | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/parsl/providers/base.py b/parsl/providers/base.py index edba6f49f2..2277336d08 100644 --- a/parsl/providers/base.py +++ b/parsl/providers/base.py @@ -49,15 +49,14 @@ def __init__(self) -> None: self._mem_per_node: Optional[float] = None pass - # TODO: how about make this always return a job ID and must raise an exception on - # failure? Potentially it could make failure path handling simpler? because right now, you - # should catch an exception or look for None and handle them both the same? @abstractmethod def submit(self, command: str, tasks_per_node: int, job_name: str = "parsl.auto") -> object: ''' The submit method takes the command string to be executed upon instantiation of a resource most often to start a pilot (such as for HighThroughputExecutor or WorkQueueExecutor). + If submission fails, an appropriate exception should be raised. + Args : - command (str) : The bash command string to be executed - tasks_per_node (int) : command invocations to be launched per node @@ -66,13 +65,7 @@ def submit(self, command: str, tasks_per_node: int, job_name: str = "parsl.auto" - job_name (str) : Human friendly name to be assigned to the job request Returns: - - A job identifier, this could be an integer, string etc - or None or any other object that evaluates to boolean false - if submission failed but an exception isn't thrown. - - Raises: - - ExecutionProviderException or its subclasses - ^ is this true? I think we can raise anything... + - A job identifier, of any type. ''' pass From 043421e1945e2ade69d648100fb055eccd34fa70 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Sat, 5 Aug 2023 10:33:26 +0000 Subject: [PATCH 25/25] marker for what i really expect to work