From 45d60a8b6913fcc79c4f9ed4b00f6aa5b8ccc2dc Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 11 Nov 2024 19:19:54 +0000 Subject: [PATCH 01/39] Remove some SSH channel related test infrastructure (#3693) SSH channels were removed in #3677 ## Type of change - Code maintenance/cleanup --- parsl/tests/conftest.py | 4 ---- test-requirements.txt | 2 -- 2 files changed, 6 deletions(-) diff --git a/parsl/tests/conftest.py b/parsl/tests/conftest.py index 63e5c45201..4bcdde0b7a 100644 --- a/parsl/tests/conftest.py +++ b/parsl/tests/conftest.py @@ -143,10 +143,6 @@ def pytest_configure(config): 'markers', 'staging_required: Marks tests that require a staging provider, when there is no sharedFS' ) - config.addinivalue_line( - 'markers', - 'sshd_required: Marks tests that require a SSHD' - ) config.addinivalue_line( 'markers', 'multiple_cores_required: Marks tests that require multiple cores, such as htex affinity' diff --git a/test-requirements.txt b/test-requirements.txt index 6abf727ccd..82ec5172c2 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -1,6 +1,5 @@ flake8==6.1.0 pandas -paramiko pytest>=7.4.0,<8 pytest-cov pytest-random-order @@ -10,7 +9,6 @@ mypy==1.5.1 types-mock types-python-dateutil types-requests -types-paramiko mpi4py # sqlalchemy is needed for typechecking, so it's here From 92ab47fa45c254d400af42ae1336d14208649996 Mon Sep 17 00:00:00 2001 From: Ryan Date: Wed, 13 Nov 2024 11:44:59 +1300 Subject: [PATCH 02/39] Update quickstart example config (#3695) # Description Minor modifications to the Quickstart documentation. Fixes a typo in `retries`, adds the relevant import statements to the Polaris config, and changes the config to use the debug queue rather than prod. ## Type of change Choose which options apply, and delete the ones which do not apply. - Update to human readable text: Documentation/error messages/comments --- docs/quickstart.rst | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/quickstart.rst b/docs/quickstart.rst index e1c3821466..d54763ee58 100644 --- a/docs/quickstart.rst +++ b/docs/quickstart.rst @@ -176,12 +176,17 @@ This script runs on a system that must stay on-line until all of your tasks comp much computing power, such as the login node for a supercomputer. The :class:`~parsl.config.Config` object holds definitions of Executors and the Providers and Launchers they rely on. -An example which launches 512 workers on 128 nodes of the Polaris supercomputer looks like +An example which launches 4 workers on 1 node of the Polaris supercomputer looks like .. code-block:: python + from parsl import Config + from parsl.executors import HighThroughputExecutor + from parsl.providers import PBSProProvider + from parsl.launchers import MpiExecLauncher + config = Config( - retires=1, # Restart task if they fail once + retries=1, # Restart task if they fail once executors=[ HighThroughputExecutor( available_accelerators=4, # Maps one worker per GPU @@ -191,13 +196,13 @@ An example which launches 512 workers on 128 nodes of the Polaris supercomputer account="example", worker_init="module load conda; conda activate parsl", walltime="1:00:00", - queue="prod", + queue="debug", scheduler_options="#PBS -l filesystems=home:eagle", # Change if data on other filesystem launcher=MpiExecLauncher( bind_cmd="--cpu-bind", overrides="--depth=64 --ppn 1" ), # Ensures 1 manger per node and allows it to divide work to all 64 cores select_options="ngpus=4", - nodes_per_block=128, + nodes_per_block=1, cpus_per_node=64, ), ), From 5cb58d153e402b2dcb972ecfdd849ebe5d94d23e Mon Sep 17 00:00:00 2001 From: Nick Tyler Date: Thu, 14 Nov 2024 04:41:45 -0800 Subject: [PATCH 03/39] Adds clusters option for slurm (#3694) # Description Adds the `clusters` option from slurm for sites with [multi-cluster slurm](https://slurm.schedmd.com/multi_cluster.html) setups. # Changed Behaviour Users on federated Slurm clusters will be able to schedule tasks between different clusters within the same slurm instance. # Fixes Fixes #3675 ## Type of change - New feature --------- Co-authored-by: Nicholas Tyler --- parsl/providers/slurm/slurm.py | 28 +++++++++++++++++++++++++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/parsl/providers/slurm/slurm.py b/parsl/providers/slurm/slurm.py index 2aa855bb48..d201c2f745 100644 --- a/parsl/providers/slurm/slurm.py +++ b/parsl/providers/slurm/slurm.py @@ -70,6 +70,9 @@ class SlurmProvider(ClusterProvider, RepresentationMixin): Slurm queue to place job in. If unspecified or ``None``, no queue slurm directive will be specified. constraint : str Slurm job constraint, often used to choose cpu or gpu type. If unspecified or ``None``, no constraint slurm directive will be added. + clusters : str + Slurm cluster name, or comma seperated cluster list, used to choose between different clusters in a federated Slurm instance. + If unspecified or ``None``, no slurm directive for clusters will be added. channel : Channel Channel for accessing this provider. nodes_per_block : int @@ -116,6 +119,7 @@ def __init__(self, account: Optional[str] = None, qos: Optional[str] = None, constraint: Optional[str] = None, + clusters: Optional[str] = None, channel: Channel = LocalChannel(), nodes_per_block: int = 1, cores_per_node: Optional[int] = None, @@ -152,6 +156,7 @@ def __init__(self, self.account = account self.qos = qos self.constraint = constraint + self.clusters = clusters self.scheduler_options = scheduler_options + '\n' if exclusive: self.scheduler_options += "#SBATCH --exclusive\n" @@ -163,6 +168,8 @@ def __init__(self, self.scheduler_options += "#SBATCH --qos={}\n".format(qos) if constraint: self.scheduler_options += "#SBATCH --constraint={}\n".format(constraint) + if clusters: + self.scheduler_options += "#SBATCH --clusters={}\n".format(clusters) self.regex_job_id = regex_job_id self.worker_init = worker_init + '\n' @@ -174,14 +181,22 @@ def __init__(self, logger.debug(f"sacct returned retcode={retcode} stderr={stderr}") if retcode == 0: logger.debug("using sacct to get job status") + _cmd = "sacct" + # Add clusters option to sacct if provided + if self.clusters: + _cmd += f" --clusters={self.clusters}" # Using state%20 to get enough characters to not truncate output # of the state. Without output can look like " CANCELLED+" - self._cmd = "sacct -X --noheader --format=jobid,state%20 --job '{0}'" + self._cmd = _cmd + " -X --noheader --format=jobid,state%20 --job '{0}'" self._translate_table = sacct_translate_table else: logger.debug(f"sacct failed with retcode={retcode}") logger.debug("falling back to using squeue to get job status") - self._cmd = "squeue --noheader --format='%i %t' --job '{0}'" + _cmd = "squeue" + # Add clusters option to squeue if provided + if self.clusters: + _cmd += f" --clusters={self.clusters}" + self._cmd = _cmd + " --noheader --format='%i %t' --job '{0}'" self._translate_table = squeue_translate_table def _status(self): @@ -344,7 +359,14 @@ def cancel(self, job_ids): ''' job_id_list = ' '.join(job_ids) - retcode, stdout, stderr = self.execute_wait("scancel {0}".format(job_id_list)) + + # Make the command to cancel jobs + _cmd = "scancel" + if self.clusters: + _cmd += f" --clusters={self.clusters}" + _cmd += " {0}" + + retcode, stdout, stderr = self.execute_wait(_cmd.format(job_id_list)) rets = None if retcode == 0: for jid in job_ids: From 3eb7e932f3054ce911920866746db55599441e22 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 14 Nov 2024 14:51:11 +0000 Subject: [PATCH 04/39] Make ZMQ, UDP and filesystem monitoring routers send via radios (#3700) This PR is intended to consolidate monitoring message sending in the monitoring radio code. This is a step towards removing Python multiprocessing from the monitoring code base (see issue #2343) by making it clearer how to change to a different message send implementation (by swapping out the radio implementation and configuration) Compare to how the interchange forwards HTEXRadio messages onwards via some other radio (which right now is always the ZMQRadioSender) -- rather than having its own ZMQ code. # Changed Behaviour none ## Type of change - Code maintenance/cleanup --- parsl/monitoring/monitoring.py | 4 +++- parsl/monitoring/router.py | 10 +++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/parsl/monitoring/monitoring.py b/parsl/monitoring/monitoring.py index a1b20f2705..08d771036a 100644 --- a/parsl/monitoring/monitoring.py +++ b/parsl/monitoring/monitoring.py @@ -270,6 +270,8 @@ def filesystem_receiver(logdir: str, q: Queue[TaggedMonitoringMessage], run_dir: new_dir = f"{base_path}/new/" logger.debug("Creating new and tmp paths under %s", base_path) + target_radio = MultiprocessingQueueRadioSender(q) + os.makedirs(tmp_dir, exist_ok=True) os.makedirs(new_dir, exist_ok=True) @@ -285,7 +287,7 @@ def filesystem_receiver(logdir: str, q: Queue[TaggedMonitoringMessage], run_dir: message = pickle.load(f) logger.debug("Message received is: %s", message) assert isinstance(message, tuple) - q.put(cast(TaggedMonitoringMessage, message)) + target_radio.send(cast(TaggedMonitoringMessage, message)) os.remove(full_path_filename) except Exception: logger.exception("Exception processing %s - probably will be retried next iteration", filename) diff --git a/parsl/monitoring/router.py b/parsl/monitoring/router.py index 1d4b522e82..a45500fc23 100644 --- a/parsl/monitoring/router.py +++ b/parsl/monitoring/router.py @@ -14,6 +14,7 @@ import zmq from parsl.log_utils import set_file_logger +from parsl.monitoring.radios import MultiprocessingQueueRadioSender from parsl.monitoring.types import TaggedMonitoringMessage from parsl.process_loggers import wrap_with_logs from parsl.utils import setproctitle @@ -55,7 +56,6 @@ def __init__(self, The amount of time in seconds to terminate the hub without receiving any messages, after the last dfk workflow message is received. resource_msgs : multiprocessing.Queue A multiprocessing queue to receive messages to be routed onwards to the database process - exit_event : Event An event that the main Parsl process will set to signal that the monitoring router should shut down. """ @@ -98,7 +98,7 @@ def __init__(self, min_port=zmq_port_range[0], max_port=zmq_port_range[1]) - self.resource_msgs = resource_msgs + self.target_radio = MultiprocessingQueueRadioSender(resource_msgs) self.exit_event = exit_event @wrap_with_logs(target="monitoring_router") @@ -125,7 +125,7 @@ def start_udp_listener(self) -> None: data, addr = self.udp_sock.recvfrom(2048) resource_msg = pickle.loads(data) self.logger.debug("Got UDP Message from {}: {}".format(addr, resource_msg)) - self.resource_msgs.put(resource_msg) + self.target_radio.send(resource_msg) except socket.timeout: pass @@ -136,7 +136,7 @@ def start_udp_listener(self) -> None: data, addr = self.udp_sock.recvfrom(2048) msg = pickle.loads(data) self.logger.debug("Got UDP Message from {}: {}".format(addr, msg)) - self.resource_msgs.put(msg) + self.target_radio.send(msg) last_msg_received_time = time.time() except socket.timeout: pass @@ -160,7 +160,7 @@ def start_zmq_listener(self) -> None: assert len(msg) >= 1, "ZMQ Receiver expects tuples of length at least 1, got {}".format(msg) assert len(msg) == 2, "ZMQ Receiver expects message tuples of exactly length 2, got {}".format(msg) - self.resource_msgs.put(msg) + self.target_radio.send(msg) except zmq.Again: pass except Exception: From 9fb5269099bec16a66df12f17ad87d9aabccd69d Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 14 Nov 2024 15:39:59 +0000 Subject: [PATCH 05/39] Remove ability to set monitoring log directory separate from rundir (#3699) Prior to this PR, MonitoringHub had a logdir parameter which let the log directory be set separately from the DFK-level run directory. Other Parsl components generally don't let the user set this unless there is a specific reason. So this PR removes that feature, reducing the amount of state to be threaded around. When reading this patch, note that what the DFK calls the rundir is a different directory vs what a Config object calls the rundir. # Changed Behaviour This removes a parameter from user facing MonitoringHub configuration. ## Type of change - Code maintenance/cleanup --- parsl/dataflow/dflow.py | 2 -- parsl/monitoring/db_manager.py | 12 ++++++------ parsl/monitoring/monitoring.py | 19 ++++++------------- parsl/monitoring/router.py | 12 ++++++------ 4 files changed, 18 insertions(+), 27 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index 83ea2e31cf..6cec168b5d 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -111,8 +111,6 @@ def __init__(self, config: Config) -> None: self.monitoring = config.monitoring if self.monitoring: - if self.monitoring.logdir is None: - self.monitoring.logdir = self.run_dir self.monitoring.start(self.run_dir, self.config.run_dir) self.time_began = datetime.datetime.now() diff --git a/parsl/monitoring/db_manager.py b/parsl/monitoring/db_manager.py index abdb038e79..8c1d76dbc6 100644 --- a/parsl/monitoring/db_manager.py +++ b/parsl/monitoring/db_manager.py @@ -279,7 +279,7 @@ class Resource(Base): class DatabaseManager: def __init__(self, db_url: str = 'sqlite:///runinfo/monitoring.db', - logdir: str = '.', + run_dir: str = '.', logging_level: int = logging.INFO, batching_interval: float = 1, batching_threshold: float = 99999, @@ -287,12 +287,12 @@ def __init__(self, self.workflow_end = False self.workflow_start_message: Optional[MonitoringMessage] = None - self.logdir = logdir - os.makedirs(self.logdir, exist_ok=True) + self.run_dir = run_dir + os.makedirs(self.run_dir, exist_ok=True) logger.propagate = False - set_file_logger("{}/database_manager.log".format(self.logdir), level=logging_level, + set_file_logger(f"{self.run_dir}/database_manager.log", level=logging_level, format_string="%(asctime)s.%(msecs)03d %(name)s:%(lineno)d [%(levelname)s] [%(threadName)s %(thread)d] %(message)s", name="database_manager") @@ -681,7 +681,7 @@ def close(self) -> None: def dbm_starter(exception_q: mpq.Queue, resource_msgs: mpq.Queue, db_url: str, - logdir: str, + run_dir: str, logging_level: int) -> None: """Start the database manager process @@ -692,7 +692,7 @@ def dbm_starter(exception_q: mpq.Queue, try: dbm = DatabaseManager(db_url=db_url, - logdir=logdir, + run_dir=run_dir, logging_level=logging_level) logger.info("Starting dbm in dbm starter") dbm.start(resource_msgs) diff --git a/parsl/monitoring/monitoring.py b/parsl/monitoring/monitoring.py index 08d771036a..e82c8fb688 100644 --- a/parsl/monitoring/monitoring.py +++ b/parsl/monitoring/monitoring.py @@ -44,7 +44,6 @@ def __init__(self, workflow_name: Optional[str] = None, workflow_version: Optional[str] = None, logging_endpoint: Optional[str] = None, - logdir: Optional[str] = None, monitoring_debug: bool = False, resource_monitoring_enabled: bool = True, resource_monitoring_interval: float = 30): # in seconds @@ -73,8 +72,6 @@ def __init__(self, The database connection url for monitoring to log the information. These URLs follow RFC-1738, and can include username, password, hostname, database name. Default: sqlite, in the configured run_dir. - logdir : str - Parsl log directory paths. Logs and temp files go here. Default: '.' monitoring_debug : Bool Enable monitoring debug logging. Default: False resource_monitoring_enabled : boolean @@ -96,7 +93,6 @@ def __init__(self, self.hub_port_range = hub_port_range self.logging_endpoint = logging_endpoint - self.logdir = logdir self.monitoring_debug = monitoring_debug self.workflow_name = workflow_name @@ -109,13 +105,10 @@ def start(self, dfk_run_dir: str, config_run_dir: Union[str, os.PathLike]) -> No logger.debug("Starting MonitoringHub") - if self.logdir is None: - self.logdir = "." - if self.logging_endpoint is None: self.logging_endpoint = f"sqlite:///{os.fspath(config_run_dir)}/monitoring.db" - os.makedirs(self.logdir, exist_ok=True) + os.makedirs(dfk_run_dir, exist_ok=True) self.monitoring_hub_active = True @@ -151,7 +144,7 @@ def start(self, dfk_run_dir: str, config_run_dir: Union[str, os.PathLike]) -> No "hub_address": self.hub_address, "udp_port": self.hub_port, "zmq_port_range": self.hub_port_range, - "logdir": self.logdir, + "run_dir": dfk_run_dir, "logging_level": logging.DEBUG if self.monitoring_debug else logging.INFO, }, name="Monitoring-Router-Process", @@ -161,7 +154,7 @@ def start(self, dfk_run_dir: str, config_run_dir: Union[str, os.PathLike]) -> No self.dbm_proc = ForkProcess(target=dbm_starter, args=(self.exception_q, self.resource_msgs,), - kwargs={"logdir": self.logdir, + kwargs={"run_dir": dfk_run_dir, "logging_level": logging.DEBUG if self.monitoring_debug else logging.INFO, "db_url": self.logging_endpoint, }, @@ -172,7 +165,7 @@ def start(self, dfk_run_dir: str, config_run_dir: Union[str, os.PathLike]) -> No logger.info("Started the router process %s and DBM process %s", self.router_proc.pid, self.dbm_proc.pid) self.filesystem_proc = ForkProcess(target=filesystem_receiver, - args=(self.logdir, self.resource_msgs, dfk_run_dir), + args=(self.resource_msgs, dfk_run_dir), name="Monitoring-Filesystem-Process", daemon=True ) @@ -258,8 +251,8 @@ def close(self) -> None: @wrap_with_logs -def filesystem_receiver(logdir: str, q: Queue[TaggedMonitoringMessage], run_dir: str) -> None: - logger = set_file_logger("{}/monitoring_filesystem_radio.log".format(logdir), +def filesystem_receiver(q: Queue[TaggedMonitoringMessage], run_dir: str) -> None: + logger = set_file_logger(f"{run_dir}/monitoring_filesystem_radio.log", name="monitoring_filesystem_radio", level=logging.INFO) diff --git a/parsl/monitoring/router.py b/parsl/monitoring/router.py index a45500fc23..04e7480a7a 100644 --- a/parsl/monitoring/router.py +++ b/parsl/monitoring/router.py @@ -31,7 +31,7 @@ def __init__(self, zmq_port_range: Tuple[int, int] = (55050, 56000), monitoring_hub_address: str = "127.0.0.1", - logdir: str = ".", + run_dir: str = ".", logging_level: int = logging.INFO, atexit_timeout: int = 3, # in seconds resource_msgs: mpq.Queue, @@ -48,7 +48,7 @@ def __init__(self, zmq_port_range : tuple(int, int) The MonitoringHub picks ports at random from the range which will be used by Hub. Default: (55050, 56000) - logdir : str + run_dir : str Parsl log directory paths. Logs and temp files go here. Default: '.' logging_level : int Logging level as defined in the logging module. Default: logging.INFO @@ -59,8 +59,8 @@ def __init__(self, exit_event : Event An event that the main Parsl process will set to signal that the monitoring router should shut down. """ - os.makedirs(logdir, exist_ok=True) - self.logger = set_file_logger("{}/monitoring_router.log".format(logdir), + os.makedirs(run_dir, exist_ok=True) + self.logger = set_file_logger(f"{run_dir}/monitoring_router.log", name="monitoring_router", level=logging_level) self.logger.debug("Monitoring router starting") @@ -187,14 +187,14 @@ def router_starter(*, udp_port: Optional[int], zmq_port_range: Tuple[int, int], - logdir: str, + run_dir: str, logging_level: int) -> None: setproctitle("parsl: monitoring router") try: router = MonitoringRouter(hub_address=hub_address, udp_port=udp_port, zmq_port_range=zmq_port_range, - logdir=logdir, + run_dir=run_dir, logging_level=logging_level, resource_msgs=resource_msgs, exit_event=exit_event) From 91146c1b6c70fef57a537c8385a244994ef0c4e5 Mon Sep 17 00:00:00 2001 From: rjmello <30907815+rjmello@users.noreply.github.com> Date: Tue, 19 Nov 2024 11:26:23 -0500 Subject: [PATCH 06/39] Extract MPI code from `execute_task()` (#3702) # Description The `execute_task()` function is used by multiple executors, but the MPI code is specific to HTEX. ## Type of change - Code maintenance/cleanup --- .../mpi_resource_management.py | 1 + .../high_throughput/process_worker_pool.py | 44 ++++++++++--------- 2 files changed, 25 insertions(+), 20 deletions(-) diff --git a/parsl/executors/high_throughput/mpi_resource_management.py b/parsl/executors/high_throughput/mpi_resource_management.py index 3f3fc33ea4..09745421d1 100644 --- a/parsl/executors/high_throughput/mpi_resource_management.py +++ b/parsl/executors/high_throughput/mpi_resource_management.py @@ -177,6 +177,7 @@ def put_task(self, task_package: dict): self._map_tasks_to_nodes[task_package["task_id"]] = allocated_nodes buffer = pack_res_spec_apply_message(_f, _args, _kwargs, resource_spec) task_package["buffer"] = buffer + task_package["resource_spec"] = resource_spec self.pending_task_q.put(task_package) diff --git a/parsl/executors/high_throughput/process_worker_pool.py b/parsl/executors/high_throughput/process_worker_pool.py index e75af86743..957d670188 100755 --- a/parsl/executors/high_throughput/process_worker_pool.py +++ b/parsl/executors/high_throughput/process_worker_pool.py @@ -590,28 +590,25 @@ def update_resource_spec_env_vars(mpi_launcher: str, resource_spec: Dict, node_i os.environ[key] = prefix_table[key] -def execute_task(bufs, mpi_launcher: Optional[str] = None): - """Deserialize the buffer and execute the task. +def _init_mpi_env(mpi_launcher: str, resource_spec: Dict): + node_list = resource_spec.get("MPI_NODELIST") + if node_list is None: + return + nodes_for_task = node_list.split(',') + logger.info(f"Launching task on provisioned nodes: {nodes_for_task}") + update_resource_spec_env_vars(mpi_launcher=mpi_launcher, resource_spec=resource_spec, node_info=nodes_for_task) + +def execute_task(bufs: bytes): + """Deserialize the buffer and execute the task. Returns the result or throws exception. """ - user_ns = locals() - user_ns.update({'__builtins__': __builtins__}) - - f, args, kwargs, resource_spec = unpack_res_spec_apply_message(bufs, user_ns, copy=False) + f, args, kwargs, resource_spec = unpack_res_spec_apply_message(bufs, copy=False) for varname in resource_spec: envname = "PARSL_" + str(varname).upper() os.environ[envname] = str(resource_spec[varname]) - if resource_spec.get("MPI_NODELIST"): - worker_id = os.environ['PARSL_WORKER_RANK'] - nodes_for_task = resource_spec["MPI_NODELIST"].split(',') - logger.info(f"Launching task on provisioned nodes: {nodes_for_task}") - assert mpi_launcher - update_resource_spec_env_vars(mpi_launcher, - resource_spec=resource_spec, - node_info=nodes_for_task) # We might need to look into callability of the function from itself # since we change it's name in the new namespace prefix = "parsl_" @@ -620,13 +617,18 @@ def execute_task(bufs, mpi_launcher: Optional[str] = None): kwargname = prefix + "kwargs" resultname = prefix + "result" - user_ns.update({fname: f, - argname: args, - kwargname: kwargs, - resultname: resultname}) - code = "{0} = {1}(*{2}, **{3})".format(resultname, fname, argname, kwargname) + + user_ns = locals() + user_ns.update({ + '__builtins__': __builtins__, + fname: f, + argname: args, + kwargname: kwargs, + resultname: resultname + }) + exec(code, user_ns, user_ns) return user_ns.get(resultname) @@ -786,8 +788,10 @@ def manager_is_alive(): ready_worker_count.value -= 1 worker_enqueued = False + _init_mpi_env(mpi_launcher=mpi_launcher, resource_spec=req["resource_spec"]) + try: - result = execute_task(req['buffer'], mpi_launcher=mpi_launcher) + result = execute_task(req['buffer']) serialized_result = serialize(result, buffer_threshold=1000000) except Exception as e: logger.info('Caught an exception: {}'.format(e)) From 8f0c5a1509288c913edf50559394d649047759ec Mon Sep 17 00:00:00 2001 From: rjmello <30907815+rjmello@users.noreply.github.com> Date: Tue, 19 Nov 2024 14:06:25 -0500 Subject: [PATCH 07/39] Move `execute_task()` to a dedicated module (#3701) # Description Multiple executors use the `execute_task()` function, so moving it to its own module improves code organization and reusability. Also removed MPI-related code from `execute_task()`, as it's specific to the HTEX. ## Type of change - Code maintenance/cleanup --- parsl/executors/execute_task.py | 37 +++++++++++++++++++ parsl/executors/flux/execute_parsl_task.py | 2 +- .../high_throughput/process_worker_pool.py | 37 +------------------ parsl/executors/radical/rpex_worker.py | 2 +- parsl/tests/test_execute_task.py | 29 +++++++++++++++ 5 files changed, 70 insertions(+), 37 deletions(-) create mode 100644 parsl/executors/execute_task.py create mode 100644 parsl/tests/test_execute_task.py diff --git a/parsl/executors/execute_task.py b/parsl/executors/execute_task.py new file mode 100644 index 0000000000..5bcd79dda6 --- /dev/null +++ b/parsl/executors/execute_task.py @@ -0,0 +1,37 @@ +import os + +from parsl.serialize import unpack_res_spec_apply_message + + +def execute_task(bufs: bytes): + """Deserialize the buffer and execute the task. + Returns the result or throws exception. + """ + f, args, kwargs, resource_spec = unpack_res_spec_apply_message(bufs, copy=False) + + for varname in resource_spec: + envname = "PARSL_" + str(varname).upper() + os.environ[envname] = str(resource_spec[varname]) + + # We might need to look into callability of the function from itself + # since we change it's name in the new namespace + prefix = "parsl_" + fname = prefix + "f" + argname = prefix + "args" + kwargname = prefix + "kwargs" + resultname = prefix + "result" + + code = "{0} = {1}(*{2}, **{3})".format(resultname, fname, + argname, kwargname) + + user_ns = locals() + user_ns.update({ + '__builtins__': __builtins__, + fname: f, + argname: args, + kwargname: kwargs, + resultname: resultname + }) + + exec(code, user_ns, user_ns) + return user_ns.get(resultname) diff --git a/parsl/executors/flux/execute_parsl_task.py b/parsl/executors/flux/execute_parsl_task.py index ddf3c67e14..4372578e26 100644 --- a/parsl/executors/flux/execute_parsl_task.py +++ b/parsl/executors/flux/execute_parsl_task.py @@ -4,8 +4,8 @@ import logging import os +from parsl.executors.execute_task import execute_task from parsl.executors.flux import TaskResult -from parsl.executors.high_throughput.process_worker_pool import execute_task from parsl.serialize import serialize diff --git a/parsl/executors/high_throughput/process_worker_pool.py b/parsl/executors/high_throughput/process_worker_pool.py index 957d670188..a8bbaa9be8 100755 --- a/parsl/executors/high_throughput/process_worker_pool.py +++ b/parsl/executors/high_throughput/process_worker_pool.py @@ -23,6 +23,7 @@ from parsl import curvezmq from parsl.app.errors import RemoteExceptionWrapper +from parsl.executors.execute_task import execute_task from parsl.executors.high_throughput.errors import WorkerLost from parsl.executors.high_throughput.mpi_prefix_composer import ( VALID_LAUNCHERS, @@ -35,7 +36,7 @@ from parsl.executors.high_throughput.probe import probe_addresses from parsl.multiprocessing import SpawnContext from parsl.process_loggers import wrap_with_logs -from parsl.serialize import serialize, unpack_res_spec_apply_message +from parsl.serialize import serialize from parsl.version import VERSION as PARSL_VERSION HEARTBEAT_CODE = (2 ** 32) - 1 @@ -599,40 +600,6 @@ def _init_mpi_env(mpi_launcher: str, resource_spec: Dict): update_resource_spec_env_vars(mpi_launcher=mpi_launcher, resource_spec=resource_spec, node_info=nodes_for_task) -def execute_task(bufs: bytes): - """Deserialize the buffer and execute the task. - Returns the result or throws exception. - """ - f, args, kwargs, resource_spec = unpack_res_spec_apply_message(bufs, copy=False) - - for varname in resource_spec: - envname = "PARSL_" + str(varname).upper() - os.environ[envname] = str(resource_spec[varname]) - - # We might need to look into callability of the function from itself - # since we change it's name in the new namespace - prefix = "parsl_" - fname = prefix + "f" - argname = prefix + "args" - kwargname = prefix + "kwargs" - resultname = prefix + "result" - - code = "{0} = {1}(*{2}, **{3})".format(resultname, fname, - argname, kwargname) - - user_ns = locals() - user_ns.update({ - '__builtins__': __builtins__, - fname: f, - argname: args, - kwargname: kwargs, - resultname: resultname - }) - - exec(code, user_ns, user_ns) - return user_ns.get(resultname) - - @wrap_with_logs(target="worker_log") def worker( worker_id: int, diff --git a/parsl/executors/radical/rpex_worker.py b/parsl/executors/radical/rpex_worker.py index db1b7d2bea..09482d8d01 100644 --- a/parsl/executors/radical/rpex_worker.py +++ b/parsl/executors/radical/rpex_worker.py @@ -4,7 +4,7 @@ import parsl.app.errors as pe from parsl.app.bash import remote_side_bash_executor -from parsl.executors.high_throughput.process_worker_pool import execute_task +from parsl.executors.execute_task import execute_task from parsl.serialize import serialize, unpack_res_spec_apply_message diff --git a/parsl/tests/test_execute_task.py b/parsl/tests/test_execute_task.py new file mode 100644 index 0000000000..42fb59c5c1 --- /dev/null +++ b/parsl/tests/test_execute_task.py @@ -0,0 +1,29 @@ +import os + +import pytest + +from parsl.executors.execute_task import execute_task +from parsl.serialize.facade import pack_res_spec_apply_message + + +def addemup(*args: int, name: str = "apples"): + total = sum(args) + return f"{total} {name}" + + +@pytest.mark.local +def test_execute_task(): + args = (1, 2, 3) + kwargs = {"name": "boots"} + buff = pack_res_spec_apply_message(addemup, args, kwargs, {}) + res = execute_task(buff) + assert res == addemup(*args, **kwargs) + + +@pytest.mark.local +def test_execute_task_resource_spec(): + resource_spec = {"num_nodes": 2, "ranks_per_node": 2, "num_ranks": 4} + buff = pack_res_spec_apply_message(addemup, (1, 2), {}, resource_spec) + execute_task(buff) + for key, val in resource_spec.items(): + assert os.environ[f"PARSL_{key.upper()}"] == str(val) From 07dfb42d94e39feae7ca2f1b4c49d881a59c11f7 Mon Sep 17 00:00:00 2001 From: rjmello <30907815+rjmello@users.noreply.github.com> Date: Wed, 20 Nov 2024 14:05:17 -0500 Subject: [PATCH 08/39] Drop unused args from unpack functions (#3703) # Description The `user_ns` and `copy` arguments from `unpack_apply_message()` and `unpack_res_spec_apply_message()` are not used. ## Type of change - Code maintenance/cleanup --- parsl/executors/execute_task.py | 2 +- parsl/executors/high_throughput/mpi_resource_management.py | 4 +--- parsl/executors/radical/rpex_worker.py | 2 +- parsl/executors/workqueue/exec_parsl_function.py | 2 +- parsl/serialize/facade.py | 6 +++--- 5 files changed, 7 insertions(+), 9 deletions(-) diff --git a/parsl/executors/execute_task.py b/parsl/executors/execute_task.py index 5bcd79dda6..41c2f3cc9b 100644 --- a/parsl/executors/execute_task.py +++ b/parsl/executors/execute_task.py @@ -7,7 +7,7 @@ def execute_task(bufs: bytes): """Deserialize the buffer and execute the task. Returns the result or throws exception. """ - f, args, kwargs, resource_spec = unpack_res_spec_apply_message(bufs, copy=False) + f, args, kwargs, resource_spec = unpack_res_spec_apply_message(bufs) for varname in resource_spec: envname = "PARSL_" + str(varname).upper() diff --git a/parsl/executors/high_throughput/mpi_resource_management.py b/parsl/executors/high_throughput/mpi_resource_management.py index 09745421d1..ac02e0c419 100644 --- a/parsl/executors/high_throughput/mpi_resource_management.py +++ b/parsl/executors/high_throughput/mpi_resource_management.py @@ -160,9 +160,7 @@ def put_task(self, task_package: dict): """Schedule task if resources are available otherwise backlog the task""" user_ns = locals() user_ns.update({"__builtins__": __builtins__}) - _f, _args, _kwargs, resource_spec = unpack_res_spec_apply_message( - task_package["buffer"], user_ns, copy=False - ) + _f, _args, _kwargs, resource_spec = unpack_res_spec_apply_message(task_package["buffer"]) nodes_needed = resource_spec.get("num_nodes") if nodes_needed: diff --git a/parsl/executors/radical/rpex_worker.py b/parsl/executors/radical/rpex_worker.py index 09482d8d01..90243b558a 100644 --- a/parsl/executors/radical/rpex_worker.py +++ b/parsl/executors/radical/rpex_worker.py @@ -33,7 +33,7 @@ def _dispatch_proc(self, task): try: buffer = rp.utils.deserialize_bson(task['description']['executable']) - func, args, kwargs, _resource_spec = unpack_res_spec_apply_message(buffer, {}, copy=False) + func, args, kwargs, _resource_spec = unpack_res_spec_apply_message(buffer) ret = remote_side_bash_executor(func, *args, **kwargs) exc = (None, None) val = None diff --git a/parsl/executors/workqueue/exec_parsl_function.py b/parsl/executors/workqueue/exec_parsl_function.py index 06a86a5c8d..d19d92efe6 100644 --- a/parsl/executors/workqueue/exec_parsl_function.py +++ b/parsl/executors/workqueue/exec_parsl_function.py @@ -94,7 +94,7 @@ def unpack_source_code_function(function_info, user_namespace): def unpack_byte_code_function(function_info, user_namespace): from parsl.serialize import unpack_apply_message - func, args, kwargs = unpack_apply_message(function_info["byte code"], user_namespace, copy=False) + func, args, kwargs = unpack_apply_message(function_info["byte code"]) return (func, 'parsl_function_name', args, kwargs) diff --git a/parsl/serialize/facade.py b/parsl/serialize/facade.py index f8e76f174b..2e02e2b983 100644 --- a/parsl/serialize/facade.py +++ b/parsl/serialize/facade.py @@ -87,16 +87,16 @@ def pack_res_spec_apply_message(func: Any, args: Any, kwargs: Any, resource_spec return pack_apply_message(func, args, (kwargs, resource_specification), buffer_threshold=buffer_threshold) -def unpack_apply_message(packed_buffer: bytes, user_ns: Any = None, copy: Any = False) -> List[Any]: +def unpack_apply_message(packed_buffer: bytes) -> List[Any]: """ Unpack and deserialize function and parameters """ return [deserialize(buf) for buf in unpack_buffers(packed_buffer)] -def unpack_res_spec_apply_message(packed_buffer: bytes, user_ns: Any = None, copy: Any = False) -> List[Any]: +def unpack_res_spec_apply_message(packed_buffer: bytes) -> List[Any]: """ Unpack and deserialize function, parameters, and resource_specification """ - func, args, (kwargs, resource_spec) = unpack_apply_message(packed_buffer, user_ns=user_ns, copy=copy) + func, args, (kwargs, resource_spec) = unpack_apply_message(packed_buffer) return [func, args, kwargs, resource_spec] From 1c3e5096562650bca6542bf99805fa7bc230a8b8 Mon Sep 17 00:00:00 2001 From: Yadu Nand Babuji Date: Mon, 25 Nov 2024 13:38:06 -0600 Subject: [PATCH 09/39] Adding support for IPV6 addresses (#3704) # Description This PR adds a new option: `HighThoughputExecutor(loopback_address: str = "127.0.0.1")` which can be used to specify the internal address used by HTEX for communication between the executor and the interchange. In addition, all ZMQ sockets are now are set to having IPv6 enabled. The test config `htex_local` has been updated to use `loopback_address="::1"` for testing. # Changed Behaviour * IPv6 support is enabled on all HTEX ZMQ components. * HTEX now supports a `loopback_address` which allows configuring the address used for internal communication Fixes # (issue) ## Type of change Choose which options apply, and delete the ones which do not apply. - New feature - Update to human readable text: Documentation/error messages/comments --- parsl/addresses.py | 20 ++++++++++++++++++- parsl/curvezmq.py | 4 ++++ parsl/executors/high_throughput/executor.py | 19 +++++++++++++----- .../executors/high_throughput/interchange.py | 15 +++++++------- .../executors/high_throughput/mpi_executor.py | 2 ++ parsl/executors/high_throughput/probe.py | 8 ++++---- .../high_throughput/process_worker_pool.py | 5 +++-- parsl/executors/high_throughput/zmq_pipes.py | 9 +++++---- parsl/tests/configs/htex_local.py | 1 + parsl/tests/test_htex/test_zmq_binding.py | 5 +++-- parsl/tests/unit/test_address.py | 20 +++++++++++++++++++ 11 files changed, 83 insertions(+), 25 deletions(-) create mode 100644 parsl/tests/unit/test_address.py diff --git a/parsl/addresses.py b/parsl/addresses.py index f45e6dfbd1..df52d1047c 100644 --- a/parsl/addresses.py +++ b/parsl/addresses.py @@ -6,6 +6,7 @@ so some experimentation will probably be needed to choose the correct one. """ +import ipaddress import logging import platform import socket @@ -17,7 +18,7 @@ except ImportError: fcntl = None # type: ignore[assignment] import struct -from typing import Callable, List, Set +from typing import Callable, List, Set, Union import psutil import typeguard @@ -156,3 +157,20 @@ def get_any_address() -> str: if addr == '': raise Exception('Cannot find address of the local machine.') return addr + + +def tcp_url(address: str, port: Union[str, int, None] = None) -> str: + """Construct a tcp url safe for IPv4 and IPv6""" + if address == "*": + return "tcp://*" + + ip_addr = ipaddress.ip_address(address) + + port_suffix = f":{port}" if port else "" + + if ip_addr.version == 6 and port_suffix: + url = f"tcp://[{address}]{port_suffix}" + else: + url = f"tcp://{address}{port_suffix}" + + return url diff --git a/parsl/curvezmq.py b/parsl/curvezmq.py index 3e0ad9409f..e90e13a5bd 100644 --- a/parsl/curvezmq.py +++ b/parsl/curvezmq.py @@ -160,6 +160,9 @@ def socket(self, socket_type: int, *args, **kwargs) -> zmq.Socket: except zmq.ZMQError as e: raise ValueError("Invalid CurveZMQ key format") from e sock.setsockopt(zmq.CURVE_SERVER, True) # Must come before bind + + # This flag enables IPV6 in addition to IPV4 + sock.setsockopt(zmq.IPV6, True) return sock def term(self): @@ -202,4 +205,5 @@ def socket(self, socket_type: int, *args, **kwargs) -> zmq.Socket: sock.setsockopt(zmq.CURVE_SERVERKEY, server_public_key) except zmq.ZMQError as e: raise ValueError("Invalid CurveZMQ key format") from e + sock.setsockopt(zmq.IPV6, True) return sock diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index a1def0466a..882776cf5c 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -86,7 +86,7 @@ address : string An address to connect to the main Parsl process which is reachable from the network in which - workers will be running. This field expects an IPv4 address (xxx.xxx.xxx.xxx). + workers will be running. This field expects an IPv4 or IPv6 address. Most login nodes on clusters have several network interfaces available, only some of which can be reached from the compute nodes. This field can be used to limit the executor to listen only on a specific interface, and limiting connections to the internal network. @@ -94,6 +94,11 @@ Setting an address here overrides the default behavior. default=None + loopback_address: string + Specify address used for internal communication between executor and interchange. + Supports IPv4 and IPv6 addresses + default=127.0.0.1 + worker_ports : (int, int) Specify the ports to be used by workers to connect to Parsl. If this option is specified, worker_port_range will not be honored. @@ -224,6 +229,7 @@ class HighThroughputExecutor(BlockProviderExecutor, RepresentationMixin, UsageIn Parsl will create names as integers starting with 0. default: empty list + """ @typeguard.typechecked @@ -233,6 +239,7 @@ def __init__(self, launch_cmd: Optional[str] = None, interchange_launch_cmd: Optional[Sequence[str]] = None, address: Optional[str] = None, + loopback_address: str = "127.0.0.1", 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), @@ -268,6 +275,8 @@ def __init__(self, self.address = address self.address_probe_timeout = address_probe_timeout self.manager_selector = manager_selector + self.loopback_address = loopback_address + if self.address: self.all_addresses = address else: @@ -408,13 +417,13 @@ def start(self): ) self.outgoing_q = zmq_pipes.TasksOutgoing( - "127.0.0.1", self.interchange_port_range, self.cert_dir + self.loopback_address, self.interchange_port_range, self.cert_dir ) self.incoming_q = zmq_pipes.ResultsIncoming( - "127.0.0.1", self.interchange_port_range, self.cert_dir + self.loopback_address, self.interchange_port_range, self.cert_dir ) self.command_client = zmq_pipes.CommandClient( - "127.0.0.1", self.interchange_port_range, self.cert_dir + self.loopback_address, self.interchange_port_range, self.cert_dir ) self._result_queue_thread = None @@ -515,7 +524,7 @@ def _start_local_interchange_process(self) -> None: get the worker task and result ports that the interchange has bound to. """ - interchange_config = {"client_address": "127.0.0.1", + interchange_config = {"client_address": self.loopback_address, "client_ports": (self.outgoing_q.port, self.incoming_q.port, self.command_client.port), diff --git a/parsl/executors/high_throughput/interchange.py b/parsl/executors/high_throughput/interchange.py index be38ccf168..88bb6c7156 100644 --- a/parsl/executors/high_throughput/interchange.py +++ b/parsl/executors/high_throughput/interchange.py @@ -14,6 +14,7 @@ import zmq from parsl import curvezmq +from parsl.addresses import tcp_url from parsl.app.errors import RemoteExceptionWrapper from parsl.executors.high_throughput.errors import ManagerLost, VersionMismatch from parsl.executors.high_throughput.manager_record import ManagerRecord @@ -115,13 +116,13 @@ def __init__(self, self.zmq_context = curvezmq.ServerContext(self.cert_dir) self.task_incoming = self.zmq_context.socket(zmq.DEALER) self.task_incoming.set_hwm(0) - self.task_incoming.connect("tcp://{}:{}".format(client_address, client_ports[0])) + self.task_incoming.connect(tcp_url(client_address, client_ports[0])) self.results_outgoing = self.zmq_context.socket(zmq.DEALER) self.results_outgoing.set_hwm(0) - self.results_outgoing.connect("tcp://{}:{}".format(client_address, client_ports[1])) + self.results_outgoing.connect(tcp_url(client_address, client_ports[1])) self.command_channel = self.zmq_context.socket(zmq.REP) - self.command_channel.connect("tcp://{}:{}".format(client_address, client_ports[2])) + self.command_channel.connect(tcp_url(client_address, client_ports[2])) logger.info("Connected to client") self.run_id = run_id @@ -144,14 +145,14 @@ def __init__(self, self.worker_task_port = self.worker_ports[0] self.worker_result_port = self.worker_ports[1] - self.task_outgoing.bind(f"tcp://{self.interchange_address}:{self.worker_task_port}") - self.results_incoming.bind(f"tcp://{self.interchange_address}:{self.worker_result_port}") + self.task_outgoing.bind(tcp_url(self.interchange_address, self.worker_task_port)) + self.results_incoming.bind(tcp_url(self.interchange_address, self.worker_result_port)) else: - self.worker_task_port = self.task_outgoing.bind_to_random_port(f"tcp://{self.interchange_address}", + self.worker_task_port = self.task_outgoing.bind_to_random_port(tcp_url(self.interchange_address), min_port=worker_port_range[0], max_port=worker_port_range[1], max_tries=100) - self.worker_result_port = self.results_incoming.bind_to_random_port(f"tcp://{self.interchange_address}", + self.worker_result_port = self.results_incoming.bind_to_random_port(tcp_url(self.interchange_address), min_port=worker_port_range[0], max_port=worker_port_range[1], max_tries=100) diff --git a/parsl/executors/high_throughput/mpi_executor.py b/parsl/executors/high_throughput/mpi_executor.py index 04b8cf5197..ba2a8ad099 100644 --- a/parsl/executors/high_throughput/mpi_executor.py +++ b/parsl/executors/high_throughput/mpi_executor.py @@ -50,6 +50,7 @@ def __init__(self, launch_cmd: Optional[str] = None, interchange_launch_cmd: Optional[str] = None, address: Optional[str] = None, + loopback_address: str = "127.0.0.1", 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), @@ -78,6 +79,7 @@ def __init__(self, launch_cmd=launch_cmd, interchange_launch_cmd=interchange_launch_cmd, address=address, + loopback_address=loopback_address, worker_ports=worker_ports, worker_port_range=worker_port_range, interchange_port_range=interchange_port_range, diff --git a/parsl/executors/high_throughput/probe.py b/parsl/executors/high_throughput/probe.py index 6b8defe601..426bf7a064 100644 --- a/parsl/executors/high_throughput/probe.py +++ b/parsl/executors/high_throughput/probe.py @@ -6,7 +6,7 @@ import zmq from zmq.utils.monitor import recv_monitor_message -from parsl.addresses import get_all_addresses +from parsl.addresses import get_all_addresses, tcp_url logger = logging.getLogger(__name__) @@ -32,7 +32,8 @@ def probe_addresses(addresses, task_port, timeout=120): for addr in addresses: socket = context.socket(zmq.DEALER) socket.setsockopt(zmq.LINGER, 0) - url = "tcp://{}:{}".format(addr, task_port) + socket.setsockopt(zmq.IPV6, True) + url = tcp_url(addr, task_port) logger.debug("Trying to connect back on {}".format(url)) socket.connect(url) addr_map[addr] = {'sock': socket, @@ -71,8 +72,7 @@ def __init__(self, addresses, port): address = probe_addresses(addresses, port) print("Viable address :", address) - self.task_incoming.connect("tcp://{}:{}".format(address, port)) - print("Here") + self.task_incoming.connect(tcp_url(address, port)) def heartbeat(self): """ Send heartbeat to the incoming task queue diff --git a/parsl/executors/high_throughput/process_worker_pool.py b/parsl/executors/high_throughput/process_worker_pool.py index a8bbaa9be8..09152de01f 100755 --- a/parsl/executors/high_throughput/process_worker_pool.py +++ b/parsl/executors/high_throughput/process_worker_pool.py @@ -22,6 +22,7 @@ import zmq from parsl import curvezmq +from parsl.addresses import tcp_url from parsl.app.errors import RemoteExceptionWrapper from parsl.executors.execute_task import execute_task from parsl.executors.high_throughput.errors import WorkerLost @@ -159,8 +160,8 @@ def __init__(self, *, raise Exception("No viable address found") else: logger.info("Connection to Interchange successful on {}".format(ix_address)) - task_q_url = "tcp://{}:{}".format(ix_address, task_port) - result_q_url = "tcp://{}:{}".format(ix_address, result_port) + task_q_url = tcp_url(ix_address, task_port) + result_q_url = tcp_url(ix_address, result_port) logger.info("Task url : {}".format(task_q_url)) logger.info("Result url : {}".format(result_q_url)) except Exception: diff --git a/parsl/executors/high_throughput/zmq_pipes.py b/parsl/executors/high_throughput/zmq_pipes.py index eb0e0f515b..54ed8c1da9 100644 --- a/parsl/executors/high_throughput/zmq_pipes.py +++ b/parsl/executors/high_throughput/zmq_pipes.py @@ -8,6 +8,7 @@ import zmq from parsl import curvezmq +from parsl.addresses import tcp_url from parsl.errors import InternalConsistencyError from parsl.executors.high_throughput.errors import ( CommandClientBadError, @@ -52,11 +53,11 @@ def create_socket_and_bind(self): self.zmq_socket = self.zmq_context.socket(zmq.REQ) self.zmq_socket.setsockopt(zmq.LINGER, 0) if self.port is None: - self.port = self.zmq_socket.bind_to_random_port("tcp://{}".format(self.ip_address), + self.port = self.zmq_socket.bind_to_random_port(tcp_url(self.ip_address), min_port=self.port_range[0], max_port=self.port_range[1]) else: - self.zmq_socket.bind("tcp://{}:{}".format(self.ip_address, self.port)) + self.zmq_socket.bind(tcp_url(self.ip_address, self.port)) def run(self, message, max_retries=3, timeout_s=None): """ This function needs to be fast at the same time aware of the possibility of @@ -146,7 +147,7 @@ def __init__(self, ip_address, port_range, cert_dir: Optional[str] = None): self.zmq_context = curvezmq.ClientContext(cert_dir) self.zmq_socket = self.zmq_context.socket(zmq.DEALER) self.zmq_socket.set_hwm(0) - self.port = self.zmq_socket.bind_to_random_port("tcp://{}".format(ip_address), + self.port = self.zmq_socket.bind_to_random_port(tcp_url(ip_address), min_port=port_range[0], max_port=port_range[1]) self.poller = zmq.Poller() @@ -202,7 +203,7 @@ def __init__(self, ip_address, port_range, cert_dir: Optional[str] = None): self.zmq_context = curvezmq.ClientContext(cert_dir) self.results_receiver = self.zmq_context.socket(zmq.DEALER) self.results_receiver.set_hwm(0) - self.port = self.results_receiver.bind_to_random_port("tcp://{}".format(ip_address), + self.port = self.results_receiver.bind_to_random_port(tcp_url(ip_address), min_port=port_range[0], max_port=port_range[1]) diff --git a/parsl/tests/configs/htex_local.py b/parsl/tests/configs/htex_local.py index 5553a7f70a..3dae57b758 100644 --- a/parsl/tests/configs/htex_local.py +++ b/parsl/tests/configs/htex_local.py @@ -10,6 +10,7 @@ def fresh_config(): executors=[ HighThroughputExecutor( label="htex_local", + loopback_address="::1", worker_debug=True, cores_per_worker=1, encrypted=True, diff --git a/parsl/tests/test_htex/test_zmq_binding.py b/parsl/tests/test_htex/test_zmq_binding.py index e21c065d0d..c189e778b2 100644 --- a/parsl/tests/test_htex/test_zmq_binding.py +++ b/parsl/tests/test_htex/test_zmq_binding.py @@ -87,7 +87,7 @@ def test_interchange_binding_with_non_ipv4_address(cert_dir: Optional[str]): def test_interchange_binding_bad_address(cert_dir: Optional[str]): """Confirm that we raise a ZMQError when a bad address is supplied""" address = "550.0.0.0" - with pytest.raises(zmq.error.ZMQError): + with pytest.raises(ValueError): make_interchange(interchange_address=address, cert_dir=cert_dir) @@ -103,4 +103,5 @@ def test_limited_interface_binding(cert_dir: Optional[str]): matched_conns = [conn for conn in conns if conn.laddr.port == ix.worker_result_port] assert len(matched_conns) == 1 - assert matched_conns[0].laddr.ip == address + # laddr.ip can return ::ffff:127.0.0.1 when using IPv6 + assert address in matched_conns[0].laddr.ip diff --git a/parsl/tests/unit/test_address.py b/parsl/tests/unit/test_address.py new file mode 100644 index 0000000000..edec854e5e --- /dev/null +++ b/parsl/tests/unit/test_address.py @@ -0,0 +1,20 @@ +import pytest + +from parsl.addresses import tcp_url + + +@pytest.mark.local +@pytest.mark.parametrize("address, port,expected", [ + ("127.0.0.1", 55001, "tcp://127.0.0.1:55001"), + ("127.0.0.1", "55001", "tcp://127.0.0.1:55001"), + ("127.0.0.1", None, "tcp://127.0.0.1"), + ("::1", "55001", "tcp://[::1]:55001"), + ("::ffff:127.0.0.1", 55001, "tcp://[::ffff:127.0.0.1]:55001"), + ("::ffff:127.0.0.1", None, "tcp://::ffff:127.0.0.1"), + ("::ffff:127.0.0.1", None, "tcp://::ffff:127.0.0.1"), + ("*", None, "tcp://*"), +]) +def test_tcp_url(address, port, expected): + """Confirm valid address generation""" + result = tcp_url(address, port) + assert result == expected From 7c2646e9d65a07b7933b3c73aa8d0e275848726c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 26 Nov 2024 18:07:28 +0000 Subject: [PATCH 10/39] Remove provider/channel push/pull of files (#3690) This removes push/pull support from channels. All the HPC providers used channel.push_file to push their batch scripts to the (no-longer) remote system. This has been a basically dead code path since removal of non-remote channels: The channel is always a LocalChannel now (PR #3677) The "remote" script directory is always the local script directory (PR #3688) and so `LocalChannel.push_file` always skips making a copy and returns the path it was given without further action. So all the removed code is a no-op, and this PR simplifies that into nothing. # Changed Behaviour Some providers had an option to let users decide if scripts (and other files) would be pushed/pulled to the remote system. Those options are removed by this PR. ## Type of change - New feature - Code maintenance/cleanup --- parsl/channels/base.py | 28 ----------------- parsl/channels/local/local.py | 36 ---------------------- parsl/providers/condor/condor.py | 8 ++--- parsl/providers/grid_engine/grid_engine.py | 5 ++- parsl/providers/local/local.py | 16 +--------- parsl/providers/lsf/lsf.py | 14 ++------- parsl/providers/pbspro/pbspro.py | 4 +-- parsl/providers/slurm/slurm.py | 12 +------- parsl/providers/torque/torque.py | 4 +-- 9 files changed, 11 insertions(+), 116 deletions(-) diff --git a/parsl/channels/base.py b/parsl/channels/base.py index ee0097f0d0..05241b878d 100644 --- a/parsl/channels/base.py +++ b/parsl/channels/base.py @@ -52,31 +52,3 @@ def script_dir(self) -> str: @script_dir.setter def script_dir(self, value: str) -> None: pass - - @abstractmethod - def push_file(self, source: str, dest_dir: str) -> str: - ''' Channel will take care of moving the file from source to the destination - directory - - Args: - source (string) : Full filepath of the file to be moved - dest_dir (string) : Absolute path of the directory to move to - - Returns: - destination_path (string) - ''' - pass - - @abstractmethod - def pull_file(self, remote_source: str, local_dir: str) -> str: - ''' Transport file on the remote side to a local directory - - Args: - remote_source (string): remote_source - local_dir (string): Local directory to copy to - - - Returns: - destination_path (string) - ''' - pass diff --git a/parsl/channels/local/local.py b/parsl/channels/local/local.py index 1d7a15dfdf..6ef014ac19 100644 --- a/parsl/channels/local/local.py +++ b/parsl/channels/local/local.py @@ -1,10 +1,8 @@ import logging import os -import shutil import subprocess from parsl.channels.base import Channel -from parsl.channels.errors import FileCopyException from parsl.utils import RepresentationMixin logger = logging.getLogger(__name__) @@ -57,40 +55,6 @@ def execute_wait(self, cmd, walltime=None): return (retcode, stdout.decode("utf-8"), stderr.decode("utf-8")) - def push_file(self, source, dest_dir): - ''' 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. - - Args: - - source (string) : Path to the source file - - dest_dir (string) : Path to the directory to which the files is to be copied - - Returns: - - destination_path (String) : Absolute path of the destination file - - Raises: - - FileCopyException : If file copy failed. - ''' - - local_dest = os.path.join(dest_dir, os.path.basename(source)) - - # Only attempt to copy if the target dir and source dir are different - if os.path.dirname(source) != dest_dir: - try: - shutil.copyfile(source, local_dest) - os.chmod(local_dest, 0o700) - - except OSError as e: - raise FileCopyException(e, "localhost") - - else: - os.chmod(local_dest, 0o700) - - return local_dest - - def pull_file(self, remote_source, local_dir): - return self.push_file(remote_source, local_dir) - @property def script_dir(self): return self._script_dir diff --git a/parsl/providers/condor/condor.py b/parsl/providers/condor/condor.py index a736386d38..c8142c4026 100644 --- a/parsl/providers/condor/condor.py +++ b/parsl/providers/condor/condor.py @@ -245,16 +245,14 @@ def submit(self, command, tasks_per_node, job_name="parsl.condor"): with open(userscript_path, 'w') as f: f.write(job_config["worker_init"] + '\n' + wrapped_command) - user_script_path = self.channel.push_file(userscript_path, self.channel.script_dir) - the_input_files = [user_script_path] + self.transfer_input_files + the_input_files = [userscript_path] + self.transfer_input_files job_config["input_files"] = ','.join(the_input_files) - job_config["job_script"] = os.path.basename(user_script_path) + job_config["job_script"] = os.path.basename(userscript_path) # Construct and move the submit script self._write_submit_script(template_string, script_path, job_name, job_config) - channel_script_path = self.channel.push_file(script_path, self.channel.script_dir) - cmd = "condor_submit {0}".format(channel_script_path) + cmd = "condor_submit {0}".format(script_path) try: retcode, stdout, stderr = self.execute_wait(cmd) except Exception as e: diff --git a/parsl/providers/grid_engine/grid_engine.py b/parsl/providers/grid_engine/grid_engine.py index e7db987022..ddedcaa3e8 100644 --- a/parsl/providers/grid_engine/grid_engine.py +++ b/parsl/providers/grid_engine/grid_engine.py @@ -142,11 +142,10 @@ def submit(self, command, tasks_per_node, job_name="parsl.sge"): logger.debug("Writing submit script") self._write_submit_script(template_string, script_path, job_name, job_config) - channel_script_path = self.channel.push_file(script_path, self.channel.script_dir) if self.queue is not None: - cmd = "qsub -q {0} -terse {1}".format(self.queue, channel_script_path) + cmd = "qsub -q {0} -terse {1}".format(self.queue, script_path) else: - cmd = "qsub -terse {0}".format(channel_script_path) + cmd = "qsub -terse {0}".format(script_path) retcode, stdout, stderr = self.execute_wait(cmd) if retcode == 0: diff --git a/parsl/providers/local/local.py b/parsl/providers/local/local.py index f13521466a..5ecf174df2 100644 --- a/parsl/providers/local/local.py +++ b/parsl/providers/local/local.py @@ -32,9 +32,6 @@ class LocalProvider(ExecutionProvider, RepresentationMixin): Ratio of provisioned task slots to active tasks. A parallelism value of 1 represents aggressive scaling where as many resources as possible are used; parallelism close to 0 represents the opposite situation in which as few resources as possible (i.e., min_blocks) are used. - move_files : Optional[Bool] - Should files be moved? By default, Parsl will try to figure this out itself (= None). - If True, then will always move. If False, will never move. worker_init : str Command to be run before starting a worker, such as 'module load Anaconda; source activate env'. """ @@ -48,8 +45,7 @@ def __init__(self, max_blocks=1, worker_init='', cmd_timeout=30, - parallelism=1, - move_files=None): + parallelism=1): self.channel = channel self._label = 'local' self.nodes_per_block = nodes_per_block @@ -61,7 +57,6 @@ def __init__(self, self.parallelism = parallelism self.script_dir = None self.cmd_timeout = cmd_timeout - self.move_files = move_files # Dictionary that keeps track of jobs, keyed on job_id self.resources = {} @@ -83,7 +78,6 @@ def status(self, job_ids): if job_dict['status'] and job_dict['status'].terminal: # We already checked this and it can't change after that continue - # Script path should point to remote path if _should_move_files() is True script_path = job_dict['script_path'] alive = self._is_alive(job_dict) @@ -137,8 +131,6 @@ def _is_alive(self, job_dict): 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) return path def _read_job_file(self, script_path: str, suffix: str) -> str: @@ -216,9 +208,6 @@ def submit(self, command, tasks_per_node, job_name="parsl.localprovider"): job_id = None remote_pid = None - if self._should_move_files(): - logger.debug("Pushing start script") - script_path = self.channel.push_file(script_path, self.channel.script_dir) logger.debug("Launching") # We need to capture the exit code and the streams, so we put them in files. We also write @@ -254,9 +243,6 @@ def submit(self, command, tasks_per_node, job_name="parsl.localprovider"): return job_id - def _should_move_files(self): - return (self.move_files is None and not isinstance(self.channel, LocalChannel)) or (self.move_files) - def cancel(self, job_ids): ''' Cancels the jobs specified by a list of job ids diff --git a/parsl/providers/lsf/lsf.py b/parsl/providers/lsf/lsf.py index 8f18f5c879..b446b063a4 100644 --- a/parsl/providers/lsf/lsf.py +++ b/parsl/providers/lsf/lsf.py @@ -68,7 +68,6 @@ class LSFProvider(ClusterProvider, RepresentationMixin): :class:`~parsl.launchers.SingleNodeLauncher` (the default), :class:`~parsl.launchers.SrunLauncher`, or :class:`~parsl.launchers.AprunLauncher` - move_files : Optional[Bool]: should files be moved? by default, Parsl will try to move files. bsub_redirection: Bool Should a redirection symbol "<" be included when submitting jobs, i.e., Bsub < job_script. request_by_nodes: Bool @@ -92,7 +91,6 @@ def __init__(self, project=None, queue=None, cmd_timeout=120, - move_files=True, bsub_redirection=False, request_by_nodes=True, launcher=SingleNodeLauncher()): @@ -112,7 +110,6 @@ def __init__(self, self.queue = queue self.cores_per_block = cores_per_block self.cores_per_node = cores_per_node - self.move_files = move_files self.bsub_redirection = bsub_redirection self.request_by_nodes = request_by_nodes @@ -230,17 +227,10 @@ def submit(self, command, tasks_per_node, job_name="parsl.lsf"): logger.debug("Writing submit script") self._write_submit_script(template_string, script_path, job_name, job_config) - if self.move_files: - logger.debug("moving files") - channel_script_path = self.channel.push_file(script_path, self.channel.script_dir) - else: - logger.debug("not moving files") - channel_script_path = script_path - if self.bsub_redirection: - cmd = "bsub < {0}".format(channel_script_path) + cmd = "bsub < {0}".format(script_path) else: - cmd = "bsub {0}".format(channel_script_path) + cmd = "bsub {0}".format(script_path) retcode, stdout, stderr = super().execute_wait(cmd) job_id = None diff --git a/parsl/providers/pbspro/pbspro.py b/parsl/providers/pbspro/pbspro.py index 752f504334..71c958f000 100644 --- a/parsl/providers/pbspro/pbspro.py +++ b/parsl/providers/pbspro/pbspro.py @@ -183,15 +183,13 @@ def submit(self, command, tasks_per_node, job_name="parsl"): logger.debug("Writing submit script") self._write_submit_script(self.template_string, script_path, job_name, job_config) - channel_script_path = self.channel.push_file(script_path, self.channel.script_dir) - submit_options = '' if self.queue is not None: submit_options = '{0} -q {1}'.format(submit_options, self.queue) if self.account is not None: submit_options = '{0} -A {1}'.format(submit_options, self.account) - launch_cmd = "qsub {0} {1}".format(submit_options, channel_script_path) + launch_cmd = "qsub {0} {1}".format(submit_options, script_path) retcode, stdout, stderr = self.execute_wait(launch_cmd) job_id = None diff --git a/parsl/providers/slurm/slurm.py b/parsl/providers/slurm/slurm.py index d201c2f745..9b6f38b9d9 100644 --- a/parsl/providers/slurm/slurm.py +++ b/parsl/providers/slurm/slurm.py @@ -110,7 +110,6 @@ class SlurmProvider(ClusterProvider, RepresentationMixin): :class:`~parsl.launchers.SingleNodeLauncher` (the default), :class:`~parsl.launchers.SrunLauncher`, or :class:`~parsl.launchers.AprunLauncher` - move_files : Optional[Bool]: should files be moved? by default, Parsl will try to move files. """ @typeguard.typechecked @@ -134,7 +133,6 @@ def __init__(self, worker_init: str = '', cmd_timeout: int = 10, exclusive: bool = True, - move_files: bool = True, launcher: Launcher = SingleNodeLauncher()): label = 'slurm' super().__init__(label, @@ -152,7 +150,6 @@ def __init__(self, self.cores_per_node = cores_per_node self.mem_per_node = mem_per_node self.exclusive = exclusive - self.move_files = move_files self.account = account self.qos = qos self.constraint = constraint @@ -308,14 +305,7 @@ def submit(self, command: str, tasks_per_node: int, job_name="parsl.slurm") -> s logger.debug("Writing submit script") self._write_submit_script(template_string, script_path, job_name, job_config) - if self.move_files: - logger.debug("moving files") - channel_script_path = self.channel.push_file(script_path, self.channel.script_dir) - else: - logger.debug("not moving files") - channel_script_path = script_path - - retcode, stdout, stderr = self.execute_wait("sbatch {0}".format(channel_script_path)) + retcode, stdout, stderr = self.execute_wait("sbatch {0}".format(script_path)) if retcode == 0: for line in stdout.split('\n'): diff --git a/parsl/providers/torque/torque.py b/parsl/providers/torque/torque.py index c15591706c..7992893abb 100644 --- a/parsl/providers/torque/torque.py +++ b/parsl/providers/torque/torque.py @@ -189,15 +189,13 @@ def submit(self, command, tasks_per_node, job_name="parsl.torque"): logger.debug("Writing submit script") self._write_submit_script(self.template_string, script_path, job_name, job_config) - channel_script_path = self.channel.push_file(script_path, self.channel.script_dir) - submit_options = '' if self.queue is not None: submit_options = '{0} -q {1}'.format(submit_options, self.queue) if self.account is not None: submit_options = '{0} -A {1}'.format(submit_options, self.account) - launch_cmd = "qsub {0} {1}".format(submit_options, channel_script_path) + launch_cmd = "qsub {0} {1}".format(submit_options, script_path) retcode, stdout, stderr = self.execute_wait(launch_cmd) job_id = None From 7622caada72429c63df3c73546c1d711cbb8e078 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 2 Dec 2024 18:43:16 +0000 Subject: [PATCH 11/39] Make result queue poll for shutdown, and tidy up at shutdown (#3709) This poll happens at the configured htex poll period, which defaults to 10ms. Under heavy result load, this shoudn't result in much additional load: the poll loop will already be looping a lot to process the results. Under lower result load, there is a slight observable increase in CPU usage: a 30second sleep task shows this before this PR: before: real 0m37.451s user 0m2.160s sys 0m0.376s run 2, user 2.160s run 3, user 2.116s and this after this PR: real 0m37.473s user 0m2.400s sys 0m0.557s Run 2, 2.457s Run 3, 2.452s At shutdown, the ZMQ socket for incoming results is closed. This reduces both the number of threads and number of file descriptors left behind by the `--config local` tests. For example: $ pytest parsl/tests/test_monitoring/ --config local Before this PR, at end of test: 32 threads, 451 fds open. After this PR, at end of test: 1 thread, 48 fds open. This is part of PR #3397 shutdown tidyup. # Description Please include a summary of the change and (optionally) which issue is fixed. Please also include relevant motivation and context. # Changed Behaviour nothing should be really visible to normal users. Increased CPU usage in the above documented situations. ## Type of change - New feature - Code maintenance/cleanup --- parsl/executors/high_throughput/executor.py | 17 +++++++++++++++-- parsl/executors/high_throughput/zmq_pipes.py | 17 +++++++++++++---- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index 882776cf5c..d3fa522619 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -331,6 +331,9 @@ def __init__(self, interchange_launch_cmd = DEFAULT_INTERCHANGE_LAUNCH_CMD self.interchange_launch_cmd = interchange_launch_cmd + self._result_queue_thread_exit = threading.Event() + self._result_queue_thread: Optional[threading.Thread] = None + radio_mode = "htex" enable_mpi_mode: bool = False mpi_launcher: str = "mpiexec" @@ -455,9 +458,11 @@ def _result_queue_worker(self): """ logger.debug("Result queue worker starting") - while not self.bad_state_is_set: + while not self.bad_state_is_set and not self._result_queue_thread_exit.is_set(): try: - msgs = self.incoming_q.get() + msgs = self.incoming_q.get(timeout_ms=self.poll_period) + if msgs is None: # timeout + continue except IOError as e: logger.exception("Caught broken queue with exception code {}: {}".format(e.errno, e)) @@ -515,6 +520,8 @@ def _result_queue_worker(self): else: raise BadMessage("Message received with unknown type {}".format(msg['type'])) + logger.info("Closing result ZMQ pipe") + self.incoming_q.close() logger.info("Result queue worker finished") def _start_local_interchange_process(self) -> None: @@ -817,6 +824,8 @@ def shutdown(self, timeout: float = 10.0): logger.info("Attempting HighThroughputExecutor shutdown") + logger.info("Terminating interchange and result queue thread") + self._result_queue_thread_exit.set() self.interchange_proc.terminate() try: self.interchange_proc.wait(timeout=timeout) @@ -841,6 +850,10 @@ def shutdown(self, timeout: float = 10.0): logger.info("Closing command client") self.command_client.close() + logger.info("Waiting for result queue thread exit") + if self._result_queue_thread: + self._result_queue_thread.join() + logger.info("Finished HighThroughputExecutor shutdown attempt") def get_usage_information(self): diff --git a/parsl/executors/high_throughput/zmq_pipes.py b/parsl/executors/high_throughput/zmq_pipes.py index 54ed8c1da9..a7278cf067 100644 --- a/parsl/executors/high_throughput/zmq_pipes.py +++ b/parsl/executors/high_throughput/zmq_pipes.py @@ -206,12 +206,21 @@ def __init__(self, ip_address, port_range, cert_dir: Optional[str] = None): self.port = self.results_receiver.bind_to_random_port(tcp_url(ip_address), min_port=port_range[0], max_port=port_range[1]) + self.poller = zmq.Poller() + self.poller.register(self.results_receiver, zmq.POLLIN) - def get(self): + def get(self, timeout_ms=None): + """Get a message from the queue, returning None if timeout expires + without a message. timeout is measured in milliseconds. + """ logger.debug("Waiting for ResultsIncoming message") - m = self.results_receiver.recv_multipart() - logger.debug("Received ResultsIncoming message") - return m + socks = dict(self.poller.poll(timeout=timeout_ms)) + if self.results_receiver in socks and socks[self.results_receiver] == zmq.POLLIN: + m = self.results_receiver.recv_multipart() + logger.debug("Received ResultsIncoming message") + return m + else: + return None def close(self): self.results_receiver.close() From 1f583af76370eebd9640eff059cfd674ea1d0325 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 2 Dec 2024 18:44:40 +0000 Subject: [PATCH 12/39] Remove monitoring UDP hub_port from example/test configs (#3710) This port will be chosen dynamically, and that is fine in test situations. This is probably also better in environments where users run multiple Parsl instances at once: they cannot all use the removed port 55055 and will interfere with each other. Ongoing work to refactor monitoring radios will remove this hub_port parameter, making it either unnecessary (when non-UDP monitoring radios are used) or specified as part of the radio selection. This PR removes hub_port from tests and examples as preparation for that change. # Changed Behaviour The monitoring UDP port will be chosen dynamically in tests and examples. ## Type of change - Code maintenance/cleanup --- docs/userguide/monitoring.rst | 1 - parsl/configs/ASPIRE1.py | 1 - parsl/tests/configs/htex_local_alternate.py | 1 - parsl/tests/configs/local_threads_monitoring.py | 1 - parsl/tests/manual_tests/test_udp_simple.py | 1 - .../tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py | 1 - parsl/tests/test_monitoring/test_stdouterr.py | 1 - 7 files changed, 7 deletions(-) diff --git a/docs/userguide/monitoring.rst b/docs/userguide/monitoring.rst index 3404862450..02b3177ca7 100644 --- a/docs/userguide/monitoring.rst +++ b/docs/userguide/monitoring.rst @@ -42,7 +42,6 @@ configuration. Here the `parsl.monitoring.MonitoringHub` is specified to use por ], monitoring=MonitoringHub( hub_address=address_by_hostname(), - hub_port=55055, monitoring_debug=False, resource_monitoring_interval=10, ), diff --git a/parsl/configs/ASPIRE1.py b/parsl/configs/ASPIRE1.py index 7792f15dba..017e1061d7 100644 --- a/parsl/configs/ASPIRE1.py +++ b/parsl/configs/ASPIRE1.py @@ -34,7 +34,6 @@ ], monitoring=MonitoringHub( hub_address=address_by_interface('ib0'), - hub_port=55055, resource_monitoring_interval=10, ), strategy='simple', diff --git a/parsl/tests/configs/htex_local_alternate.py b/parsl/tests/configs/htex_local_alternate.py index d84a07ad84..5667f3ff8c 100644 --- a/parsl/tests/configs/htex_local_alternate.py +++ b/parsl/tests/configs/htex_local_alternate.py @@ -62,7 +62,6 @@ def fresh_config(): retries=2, monitoring=MonitoringHub( hub_address="localhost", - hub_port=55055, monitoring_debug=False, resource_monitoring_interval=1, ), diff --git a/parsl/tests/configs/local_threads_monitoring.py b/parsl/tests/configs/local_threads_monitoring.py index 81b9095285..9f105af25d 100644 --- a/parsl/tests/configs/local_threads_monitoring.py +++ b/parsl/tests/configs/local_threads_monitoring.py @@ -5,7 +5,6 @@ config = Config(executors=[ThreadPoolExecutor(label='threads', max_threads=4)], monitoring=MonitoringHub( hub_address="localhost", - hub_port=55055, resource_monitoring_interval=3, ) ) diff --git a/parsl/tests/manual_tests/test_udp_simple.py b/parsl/tests/manual_tests/test_udp_simple.py index 8de257d8fa..847f8803bf 100644 --- a/parsl/tests/manual_tests/test_udp_simple.py +++ b/parsl/tests/manual_tests/test_udp_simple.py @@ -15,7 +15,6 @@ def local_setup(): ], monitoring=MonitoringHub( hub_address="127.0.0.1", - hub_port=55055, logging_level=logging.INFO, resource_monitoring_interval=10)) diff --git a/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py b/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py index ada972e747..c54486f011 100644 --- a/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py +++ b/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py @@ -37,7 +37,6 @@ def fresh_config(run_dir, strategy, db_url): strategy_period=0.1, monitoring=MonitoringHub( hub_address="localhost", - hub_port=55055, logging_endpoint=db_url ) ) diff --git a/parsl/tests/test_monitoring/test_stdouterr.py b/parsl/tests/test_monitoring/test_stdouterr.py index d1817164c0..8e1935045f 100644 --- a/parsl/tests/test_monitoring/test_stdouterr.py +++ b/parsl/tests/test_monitoring/test_stdouterr.py @@ -37,7 +37,6 @@ def fresh_config(run_dir): strategy_period=0.1, monitoring=MonitoringHub( hub_address="localhost", - hub_port=55055, ) ) From 90d4a8683527af24c8432debe02d85b7a6622e87 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 3 Dec 2024 08:03:09 +0000 Subject: [PATCH 13/39] Move execute_wait from vestigial LocalChannel into parsl.utils (#3705) # Description This moves the execute_wait functionality previously provided by LocalChannel into parsl.utils, as part of #3515. LocalChannel.execute_wait did not reference `self` so it already basically behaved as a function rather than a method. This leaves LocalChannel as solely a place for script_directory, which will be untangled in a subsequent PR. # Changed Behaviour none ## Type of change - Code maintenance/cleanup --- parsl/channels/base.py | 34 +----------------- parsl/channels/local/local.py | 35 ------------------- parsl/providers/cluster_provider.py | 3 +- parsl/providers/local/local.py | 10 +++--- parsl/tests/test_channels/__init__.py | 0 .../tests/test_channels/test_large_output.py | 22 ------------ .../tests/test_channels/test_local_channel.py | 19 ---------- parsl/tests/test_utils/test_execute_wait.py | 35 +++++++++++++++++++ parsl/utils.py | 35 +++++++++++++++++++ 9 files changed, 78 insertions(+), 115 deletions(-) delete mode 100644 parsl/tests/test_channels/__init__.py delete mode 100644 parsl/tests/test_channels/test_large_output.py delete mode 100644 parsl/tests/test_channels/test_local_channel.py create mode 100644 parsl/tests/test_utils/test_execute_wait.py diff --git a/parsl/channels/base.py b/parsl/channels/base.py index 05241b878d..0024d45a6b 100644 --- a/parsl/channels/base.py +++ b/parsl/channels/base.py @@ -1,39 +1,7 @@ -from abc import ABCMeta, abstractmethod, abstractproperty -from typing import Tuple +from abc import ABCMeta, abstractproperty class Channel(metaclass=ABCMeta): - """Channels are abstractions that enable ExecutionProviders to talk to - resource managers of remote compute facilities. - - For certain resources such as campus clusters or supercomputers at - research laboratories, resource requirements may require authentication. - - The only remaining Channel, *LocalChannel*, executes commands locally in a - shell. - - Channels provide the ability to execute commands remotely, using the - execute_wait method, and manipulate the remote file system using methods - such as push_file, pull_file and makedirs. - - Channels should ensure that each launched command runs in a new process - group, so that providers (such as LocalProvider) which terminate long - running commands using process groups can do so. - """ - - @abstractmethod - def execute_wait(self, cmd: str, walltime: int = 0) -> Tuple[int, str, str]: - ''' Executes the cmd, with a defined walltime. - - Args: - - cmd (string): Command string to execute over the channel - - walltime (int) : Timeout in seconds - - Returns: - - (exit_code, stdout, stderr) (int, string, string) - ''' - pass - @abstractproperty def script_dir(self) -> str: ''' This is a property. Returns the directory assigned for storing all internal scripts such as diff --git a/parsl/channels/local/local.py b/parsl/channels/local/local.py index 6ef014ac19..4c1712ef30 100644 --- a/parsl/channels/local/local.py +++ b/parsl/channels/local/local.py @@ -1,6 +1,5 @@ import logging import os -import subprocess from parsl.channels.base import Channel from parsl.utils import RepresentationMixin @@ -21,40 +20,6 @@ def __init__(self): ''' self.script_dir = None - def execute_wait(self, cmd, walltime=None): - ''' Synchronously execute a commandline string on the shell. - - Args: - - cmd (string) : Commandline string to execute - - walltime (int) : walltime in seconds - - Returns: - - retcode : Return code from the execution - - stdout : stdout string - - stderr : stderr string - ''' - try: - logger.debug("Creating process with command '%s'", cmd) - proc = subprocess.Popen( - cmd, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - shell=True, - preexec_fn=os.setpgrp - ) - logger.debug("Created process with pid %s. Performing communicate", proc.pid) - (stdout, stderr) = proc.communicate(timeout=walltime) - retcode = proc.returncode - logger.debug("Process %s returned %s", proc.pid, proc.returncode) - - except Exception: - logger.exception(f"Execution of command failed:\n{cmd}") - raise - else: - logger.debug("Execution of command in process %s completed normally", proc.pid) - - return (retcode, stdout.decode("utf-8"), stderr.decode("utf-8")) - @property def script_dir(self): return self._script_dir diff --git a/parsl/providers/cluster_provider.py b/parsl/providers/cluster_provider.py index 6bc76bdf22..f2feb2ddf3 100644 --- a/parsl/providers/cluster_provider.py +++ b/parsl/providers/cluster_provider.py @@ -6,6 +6,7 @@ from parsl.launchers.errors import BadLauncher from parsl.providers.base import ExecutionProvider from parsl.providers.errors import SchedulerMissingArgs, ScriptPathError +from parsl.utils import execute_wait logger = logging.getLogger(__name__) @@ -76,7 +77,7 @@ def execute_wait(self, cmd, timeout=None): t = self.cmd_timeout if timeout is not None: t = timeout - return self.channel.execute_wait(cmd, t) + return execute_wait(cmd, t) def _write_submit_script(self, template, script_filename, job_name, configs): """Generate submit script and write it to a file. diff --git a/parsl/providers/local/local.py b/parsl/providers/local/local.py index 5ecf174df2..6357c85cba 100644 --- a/parsl/providers/local/local.py +++ b/parsl/providers/local/local.py @@ -11,7 +11,7 @@ ScriptPathError, SubmitException, ) -from parsl.utils import RepresentationMixin +from parsl.utils import RepresentationMixin, execute_wait logger = logging.getLogger(__name__) @@ -118,7 +118,7 @@ def status(self, job_ids): return [self.resources[jid]['status'] for jid in job_ids] def _is_alive(self, job_dict): - retcode, stdout, stderr = self.channel.execute_wait( + retcode, stdout, stderr = 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'): @@ -223,11 +223,11 @@ def submit(self, command, tasks_per_node, job_name="parsl.localprovider"): # cancel the task later. # # We need to do the >/dev/null 2>&1 so that bash closes stdout, otherwise - # channel.execute_wait hangs reading the process stdout until all the + # execute_wait hangs reading the process stdout until all the # background commands complete. cmd = '/bin/bash -c \'echo - >{0}.ec && {{ {{ bash {0} 1>{0}.out 2>{0}.err ; ' \ 'echo $? > {0}.ec ; }} >/dev/null 2>&1 & echo "PID:$!" ; }}\''.format(script_path) - retcode, stdout, stderr = self.channel.execute_wait(cmd, self.cmd_timeout) + retcode, stdout, stderr = execute_wait(cmd, self.cmd_timeout) if retcode != 0: raise SubmitException(job_name, "Launch command exited with code {0}".format(retcode), stdout, stderr) @@ -258,7 +258,7 @@ def cancel(self, job_ids): job_dict['cancelled'] = True logger.debug("Terminating job/process ID: {0}".format(job)) cmd = "kill -- -$(ps -o pgid= {} | grep -o '[0-9]*')".format(job_dict['remote_pid']) - retcode, stdout, stderr = self.channel.execute_wait(cmd, self.cmd_timeout) + retcode, stdout, stderr = execute_wait(cmd, self.cmd_timeout) if retcode != 0: logger.warning("Failed to kill PID: {} and child processes on {}".format(job_dict['remote_pid'], self.label)) diff --git a/parsl/tests/test_channels/__init__.py b/parsl/tests/test_channels/__init__.py deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/parsl/tests/test_channels/test_large_output.py b/parsl/tests/test_channels/test_large_output.py deleted file mode 100644 index bfc96f38bc..0000000000 --- a/parsl/tests/test_channels/test_large_output.py +++ /dev/null @@ -1,22 +0,0 @@ -import pytest - -from parsl.channels.local.local import LocalChannel - - -@pytest.mark.local -def test_local_large_output_2210(): - """Regression test for #2210. - The local channel was hanging if the specified command gave too - much output, due to a race condition between process exiting and - pipes filling up. - """ - - c = LocalChannel() - - # this will output 128kb of stdout - c.execute_wait("yes | dd count=128 bs=1024", walltime=60) - - # if this test fails, execute_wait should raise a timeout - # exception. - - # The contents out the output is not verified by this test diff --git a/parsl/tests/test_channels/test_local_channel.py b/parsl/tests/test_channels/test_local_channel.py deleted file mode 100644 index a3f55d096d..0000000000 --- a/parsl/tests/test_channels/test_local_channel.py +++ /dev/null @@ -1,19 +0,0 @@ -import pytest - -from parsl.channels.local.local import LocalChannel - - -@pytest.mark.local -def test_env(): - ''' Regression testing for issue #27 - ''' - - lc = LocalChannel() - rc, stdout, stderr = lc.execute_wait("env", 1) - - stdout = stdout.split('\n') - x = [s for s in stdout if s.startswith("PATH=")] - assert x, "PATH not found" - - x = [s for s in stdout if s.startswith("HOME=")] - assert x, "HOME not found" diff --git a/parsl/tests/test_utils/test_execute_wait.py b/parsl/tests/test_utils/test_execute_wait.py new file mode 100644 index 0000000000..44488c239c --- /dev/null +++ b/parsl/tests/test_utils/test_execute_wait.py @@ -0,0 +1,35 @@ +import pytest + +from parsl.utils import execute_wait + + +@pytest.mark.local +def test_env(): + ''' Regression testing for issue #27 + ''' + + rc, stdout, stderr = execute_wait("env", 1) + + stdout = stdout.split('\n') + x = [s for s in stdout if s.startswith("PATH=")] + assert x, "PATH not found" + + x = [s for s in stdout if s.startswith("HOME=")] + assert x, "HOME not found" + + +@pytest.mark.local +def test_large_output_2210(): + """Regression test for #2210. + execute_wait was hanging if the specified command gave too + much output, due to a race condition between process exiting and + pipes filling up. + """ + + # this will output 128kb of stdout + execute_wait("yes | dd count=128 bs=1024", walltime=60) + + # if this test fails, execute_wait should raise a timeout + # exception. + + # The contents out the output is not verified by this test diff --git a/parsl/utils.py b/parsl/utils.py index 0ea5d7d9eb..b6544d63d2 100644 --- a/parsl/utils.py +++ b/parsl/utils.py @@ -458,3 +458,38 @@ def sanitize_dns_subdomain_rfc1123(raw_string: str) -> str: raise ValueError(f"Sanitized DNS subdomain is empty for input '{raw_string}'") return sanitized + + +def execute_wait(cmd: str, walltime: Optional[int] = None) -> Tuple[int, str, str]: + ''' Synchronously execute a commandline string on the shell. + + Args: + - cmd (string) : Commandline string to execute + - walltime (int) : walltime in seconds + + Returns: + - retcode : Return code from the execution + - stdout : stdout string + - stderr : stderr string + ''' + try: + logger.debug("Creating process with command '%s'", cmd) + proc = subprocess.Popen( + cmd, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + shell=True, + preexec_fn=os.setpgrp + ) + logger.debug("Created process with pid %s. Performing communicate", proc.pid) + (stdout, stderr) = proc.communicate(timeout=walltime) + retcode = proc.returncode + logger.debug("Process %s returned %s", proc.pid, proc.returncode) + + except Exception: + logger.exception(f"Execution of command failed:\n{cmd}") + raise + else: + logger.debug("Execution of command in process %s completed normally", proc.pid) + + return (retcode, stdout.decode("utf-8"), stderr.decode("utf-8")) From 3a18a4a98435a22551ac61977b242b055cf71efd Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 3 Dec 2024 08:09:13 +0000 Subject: [PATCH 14/39] Move monitoring radios to own modules (#3707) In subsequent PRs, these modules will get more radio-specific code, as part of PR #3315 monitoring radio plugin work: for example, the receiving code for each radio should move here too. I used `git show --color-moved` to check that the moved RadioSender definitions were not changed. # Changed Behaviour none ## Type of change - Code maintenance/cleanup --- parsl/executors/base.py | 2 +- .../executors/high_throughput/interchange.py | 3 +- parsl/monitoring/monitoring.py | 2 +- parsl/monitoring/radios.py | 191 ------------------ parsl/monitoring/radios/__init__.py | 0 parsl/monitoring/radios/base.py | 13 ++ parsl/monitoring/radios/filesystem.py | 52 +++++ parsl/monitoring/radios/htex.py | 57 ++++++ parsl/monitoring/radios/multiprocessing.py | 17 ++ parsl/monitoring/radios/udp.py | 56 +++++ parsl/monitoring/radios/zmq.py | 17 ++ parsl/monitoring/remote.py | 10 +- parsl/monitoring/router.py | 2 +- 13 files changed, 221 insertions(+), 201 deletions(-) delete mode 100644 parsl/monitoring/radios.py create mode 100644 parsl/monitoring/radios/__init__.py create mode 100644 parsl/monitoring/radios/base.py create mode 100644 parsl/monitoring/radios/filesystem.py create mode 100644 parsl/monitoring/radios/htex.py create mode 100644 parsl/monitoring/radios/multiprocessing.py create mode 100644 parsl/monitoring/radios/udp.py create mode 100644 parsl/monitoring/radios/zmq.py diff --git a/parsl/executors/base.py b/parsl/executors/base.py index a112b9eb00..fc97db89d3 100644 --- a/parsl/executors/base.py +++ b/parsl/executors/base.py @@ -5,7 +5,7 @@ from typing_extensions import Literal, Self -from parsl.monitoring.radios import MonitoringRadioSender +from parsl.monitoring.radios.base import MonitoringRadioSender class ParslExecutor(metaclass=ABCMeta): diff --git a/parsl/executors/high_throughput/interchange.py b/parsl/executors/high_throughput/interchange.py index 88bb6c7156..12d3e07f31 100644 --- a/parsl/executors/high_throughput/interchange.py +++ b/parsl/executors/high_throughput/interchange.py @@ -20,7 +20,8 @@ from parsl.executors.high_throughput.manager_record import ManagerRecord from parsl.executors.high_throughput.manager_selector import ManagerSelector from parsl.monitoring.message_type import MessageType -from parsl.monitoring.radios import MonitoringRadioSender, ZMQRadioSender +from parsl.monitoring.radios.base import MonitoringRadioSender +from parsl.monitoring.radios.zmq import ZMQRadioSender from parsl.process_loggers import wrap_with_logs from parsl.serialize import serialize as serialize_object from parsl.utils import setproctitle diff --git a/parsl/monitoring/monitoring.py b/parsl/monitoring/monitoring.py index e82c8fb688..3fbe5736ba 100644 --- a/parsl/monitoring/monitoring.py +++ b/parsl/monitoring/monitoring.py @@ -14,7 +14,7 @@ from parsl.log_utils import set_file_logger from parsl.monitoring.errors import MonitoringHubStartError -from parsl.monitoring.radios import MultiprocessingQueueRadioSender +from parsl.monitoring.radios.multiprocessing import MultiprocessingQueueRadioSender from parsl.monitoring.router import router_starter from parsl.monitoring.types import TaggedMonitoringMessage from parsl.multiprocessing import ForkProcess, SizedQueue diff --git a/parsl/monitoring/radios.py b/parsl/monitoring/radios.py deleted file mode 100644 index 14dc046557..0000000000 --- a/parsl/monitoring/radios.py +++ /dev/null @@ -1,191 +0,0 @@ -import logging -import os -import pickle -import socket -import uuid -from abc import ABCMeta, abstractmethod -from multiprocessing.queues import Queue - -import zmq - -logger = logging.getLogger(__name__) - - -class MonitoringRadioSender(metaclass=ABCMeta): - @abstractmethod - def send(self, message: object) -> None: - pass - - -class FilesystemRadioSender(MonitoringRadioSender): - """A MonitoringRadioSender that sends messages over a shared filesystem. - - The messsage directory structure is based on maildir, - https://en.wikipedia.org/wiki/Maildir - - The writer creates a message in tmp/ and then when it is fully - written, moves it atomically into new/ - - The reader ignores tmp/ and only reads and deletes messages from - new/ - - This avoids a race condition of reading partially written messages. - - This radio is likely to give higher shared filesystem load compared to - the UDP radio, but should be much more reliable. - """ - - def __init__(self, *, monitoring_url: str, timeout: int = 10, run_dir: str): - logger.info("filesystem based monitoring channel initializing") - self.base_path = f"{run_dir}/monitor-fs-radio/" - self.tmp_path = f"{self.base_path}/tmp" - self.new_path = f"{self.base_path}/new" - - os.makedirs(self.tmp_path, exist_ok=True) - os.makedirs(self.new_path, exist_ok=True) - - def send(self, message: object) -> None: - logger.info("Sending a monitoring message via filesystem") - - unique_id = str(uuid.uuid4()) - - tmp_filename = f"{self.tmp_path}/{unique_id}" - new_filename = f"{self.new_path}/{unique_id}" - buffer = message - - # this will write the message out then atomically - # move it into new/, so that a partially written - # file will never be observed in new/ - with open(tmp_filename, "wb") as f: - pickle.dump(buffer, f) - os.rename(tmp_filename, new_filename) - - -class HTEXRadioSender(MonitoringRadioSender): - - def __init__(self, monitoring_url: str, timeout: int = 10): - """ - Parameters - ---------- - - monitoring_url : str - URL of the form ://: - timeout : int - timeout, default=10s - """ - logger.info("htex-based monitoring channel initialising") - - def send(self, message: object) -> None: - """ Sends a message to the UDP receiver - - Parameter - --------- - - message: object - Arbitrary pickle-able object that is to be sent - - Returns: - None - """ - - import parsl.executors.high_throughput.monitoring_info - - result_queue = parsl.executors.high_throughput.monitoring_info.result_queue - - # this message needs to go in the result queue tagged so that it is treated - # i) as a monitoring message by the interchange, and then further more treated - # as a RESOURCE_INFO message when received by monitoring (rather than a NODE_INFO - # which is the implicit default for messages from the interchange) - - # for the interchange, the outer wrapper, this needs to be a dict: - - interchange_msg = { - 'type': 'monitoring', - 'payload': message - } - - if result_queue: - result_queue.put(pickle.dumps(interchange_msg)) - else: - logger.error("result_queue is uninitialized - cannot put monitoring message") - - return - - -class UDPRadioSender(MonitoringRadioSender): - - def __init__(self, monitoring_url: str, timeout: int = 10): - """ - Parameters - ---------- - - monitoring_url : str - URL of the form ://: - timeout : int - timeout, default=10s - """ - self.monitoring_url = monitoring_url - self.sock_timeout = timeout - try: - self.scheme, self.ip, port = (x.strip('/') for x in monitoring_url.split(':')) - self.port = int(port) - except Exception: - raise Exception("Failed to parse monitoring url: {}".format(monitoring_url)) - - self.sock = socket.socket(socket.AF_INET, - socket.SOCK_DGRAM, - socket.IPPROTO_UDP) # UDP - self.sock.settimeout(self.sock_timeout) - - def send(self, message: object) -> None: - """ Sends a message to the UDP receiver - - Parameter - --------- - - message: object - Arbitrary pickle-able object that is to be sent - - Returns: - None - """ - try: - buffer = pickle.dumps(message) - except Exception: - logging.exception("Exception during pickling", exc_info=True) - return - - try: - self.sock.sendto(buffer, (self.ip, self.port)) - except socket.timeout: - logging.error("Could not send message within timeout limit") - return - return - - -class MultiprocessingQueueRadioSender(MonitoringRadioSender): - """A monitoring radio which connects over a multiprocessing Queue. - This radio is intended to be used on the submit side, where components - in the submit process, or processes launched by multiprocessing, will have - access to a Queue shared with the monitoring database code (bypassing the - monitoring router). - """ - def __init__(self, queue: Queue) -> None: - self.queue = queue - - def send(self, message: object) -> None: - self.queue.put(message) - - -class ZMQRadioSender(MonitoringRadioSender): - """A monitoring radio which connects over ZMQ. This radio is not - thread-safe, because its use of ZMQ is not thread-safe. - """ - - def __init__(self, hub_address: str, hub_zmq_port: int) -> None: - self._hub_channel = zmq.Context().socket(zmq.DEALER) - self._hub_channel.set_hwm(0) - self._hub_channel.connect(f"tcp://{hub_address}:{hub_zmq_port}") - - def send(self, message: object) -> None: - self._hub_channel.send_pyobj(message) diff --git a/parsl/monitoring/radios/__init__.py b/parsl/monitoring/radios/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/parsl/monitoring/radios/base.py b/parsl/monitoring/radios/base.py new file mode 100644 index 0000000000..2bb799f256 --- /dev/null +++ b/parsl/monitoring/radios/base.py @@ -0,0 +1,13 @@ +import logging +from abc import ABCMeta, abstractmethod +from typing import Optional + +_db_manager_excepts: Optional[Exception] + +logger = logging.getLogger(__name__) + + +class MonitoringRadioSender(metaclass=ABCMeta): + @abstractmethod + def send(self, message: object) -> None: + pass diff --git a/parsl/monitoring/radios/filesystem.py b/parsl/monitoring/radios/filesystem.py new file mode 100644 index 0000000000..accff87d36 --- /dev/null +++ b/parsl/monitoring/radios/filesystem.py @@ -0,0 +1,52 @@ +import logging +import os +import pickle +import uuid + +from parsl.monitoring.radios.base import MonitoringRadioSender + +logger = logging.getLogger(__name__) + + +class FilesystemRadioSender(MonitoringRadioSender): + """A MonitoringRadioSender that sends messages over a shared filesystem. + + The messsage directory structure is based on maildir, + https://en.wikipedia.org/wiki/Maildir + + The writer creates a message in tmp/ and then when it is fully + written, moves it atomically into new/ + + The reader ignores tmp/ and only reads and deletes messages from + new/ + + This avoids a race condition of reading partially written messages. + + This radio is likely to give higher shared filesystem load compared to + the UDP radio, but should be much more reliable. + """ + + def __init__(self, *, monitoring_url: str, timeout: int = 10, run_dir: str): + logger.info("filesystem based monitoring channel initializing") + self.base_path = f"{run_dir}/monitor-fs-radio/" + self.tmp_path = f"{self.base_path}/tmp" + self.new_path = f"{self.base_path}/new" + + os.makedirs(self.tmp_path, exist_ok=True) + os.makedirs(self.new_path, exist_ok=True) + + def send(self, message: object) -> None: + logger.info("Sending a monitoring message via filesystem") + + unique_id = str(uuid.uuid4()) + + tmp_filename = f"{self.tmp_path}/{unique_id}" + new_filename = f"{self.new_path}/{unique_id}" + buffer = message + + # this will write the message out then atomically + # move it into new/, so that a partially written + # file will never be observed in new/ + with open(tmp_filename, "wb") as f: + pickle.dump(buffer, f) + os.rename(tmp_filename, new_filename) diff --git a/parsl/monitoring/radios/htex.py b/parsl/monitoring/radios/htex.py new file mode 100644 index 0000000000..bdb893b303 --- /dev/null +++ b/parsl/monitoring/radios/htex.py @@ -0,0 +1,57 @@ +import logging +import pickle + +from parsl.monitoring.radios.base import MonitoringRadioSender + +logger = logging.getLogger(__name__) + + +class HTEXRadioSender(MonitoringRadioSender): + + def __init__(self, monitoring_url: str, timeout: int = 10): + """ + Parameters + ---------- + + monitoring_url : str + URL of the form ://: + timeout : int + timeout, default=10s + """ + logger.info("htex-based monitoring channel initialising") + + def send(self, message: object) -> None: + """ Sends a message to the UDP receiver + + Parameter + --------- + + message: object + Arbitrary pickle-able object that is to be sent + + Returns: + None + """ + + import parsl.executors.high_throughput.monitoring_info + + result_queue = parsl.executors.high_throughput.monitoring_info.result_queue + + # this message needs to go in the result queue tagged so that it is treated + # i) as a monitoring message by the interchange, and then further more treated + # as a RESOURCE_INFO message when received by monitoring (rather than a NODE_INFO + # which is the implicit default for messages from the interchange) + + # for the interchange, the outer wrapper, this needs to be a dict: + + interchange_msg = { + 'type': 'monitoring', + 'payload': message + } + + if result_queue: + result_queue.put(pickle.dumps(interchange_msg)) + else: + logger.error("result_queue is uninitialized - cannot put monitoring message") + + return diff --git a/parsl/monitoring/radios/multiprocessing.py b/parsl/monitoring/radios/multiprocessing.py new file mode 100644 index 0000000000..6274bbfca8 --- /dev/null +++ b/parsl/monitoring/radios/multiprocessing.py @@ -0,0 +1,17 @@ +from multiprocessing.queues import Queue + +from parsl.monitoring.radios.base import MonitoringRadioSender + + +class MultiprocessingQueueRadioSender(MonitoringRadioSender): + """A monitoring radio which connects over a multiprocessing Queue. + This radio is intended to be used on the submit side, where components + in the submit process, or processes launched by multiprocessing, will have + access to a Queue shared with the monitoring database code (bypassing the + monitoring router). + """ + def __init__(self, queue: Queue) -> None: + self.queue = queue + + def send(self, message: object) -> None: + self.queue.put(message) diff --git a/parsl/monitoring/radios/udp.py b/parsl/monitoring/radios/udp.py new file mode 100644 index 0000000000..f2a652e9ac --- /dev/null +++ b/parsl/monitoring/radios/udp.py @@ -0,0 +1,56 @@ +import logging +import pickle +import socket + +from parsl.monitoring.radios.base import MonitoringRadioSender + + +class UDPRadioSender(MonitoringRadioSender): + + def __init__(self, monitoring_url: str, timeout: int = 10): + """ + Parameters + ---------- + + monitoring_url : str + URL of the form ://: + timeout : int + timeout, default=10s + """ + self.monitoring_url = monitoring_url + self.sock_timeout = timeout + try: + self.scheme, self.ip, port = (x.strip('/') for x in monitoring_url.split(':')) + self.port = int(port) + except Exception: + raise Exception("Failed to parse monitoring url: {}".format(monitoring_url)) + + self.sock = socket.socket(socket.AF_INET, + socket.SOCK_DGRAM, + socket.IPPROTO_UDP) # UDP + self.sock.settimeout(self.sock_timeout) + + def send(self, message: object) -> None: + """ Sends a message to the UDP receiver + + Parameter + --------- + + message: object + Arbitrary pickle-able object that is to be sent + + Returns: + None + """ + try: + buffer = pickle.dumps(message) + except Exception: + logging.exception("Exception during pickling", exc_info=True) + return + + try: + self.sock.sendto(buffer, (self.ip, self.port)) + except socket.timeout: + logging.error("Could not send message within timeout limit") + return + return diff --git a/parsl/monitoring/radios/zmq.py b/parsl/monitoring/radios/zmq.py new file mode 100644 index 0000000000..397c943568 --- /dev/null +++ b/parsl/monitoring/radios/zmq.py @@ -0,0 +1,17 @@ +import zmq + +from parsl.monitoring.radios.base import MonitoringRadioSender + + +class ZMQRadioSender(MonitoringRadioSender): + """A monitoring radio which connects over ZMQ. This radio is not + thread-safe, because its use of ZMQ is not thread-safe. + """ + + def __init__(self, hub_address: str, hub_zmq_port: int) -> None: + self._hub_channel = zmq.Context().socket(zmq.DEALER) + self._hub_channel.set_hwm(0) + self._hub_channel.connect(f"tcp://{hub_address}:{hub_zmq_port}") + + def send(self, message: object) -> None: + self._hub_channel.send_pyobj(message) diff --git a/parsl/monitoring/remote.py b/parsl/monitoring/remote.py index d72b54dc3c..530b39f935 100644 --- a/parsl/monitoring/remote.py +++ b/parsl/monitoring/remote.py @@ -7,12 +7,10 @@ from typing import Any, Callable, Dict, List, Sequence, Tuple from parsl.monitoring.message_type import MessageType -from parsl.monitoring.radios import ( - FilesystemRadioSender, - HTEXRadioSender, - MonitoringRadioSender, - UDPRadioSender, -) +from parsl.monitoring.radios.base import MonitoringRadioSender +from parsl.monitoring.radios.filesystem import FilesystemRadioSender +from parsl.monitoring.radios.htex import HTEXRadioSender +from parsl.monitoring.radios.udp import UDPRadioSender from parsl.multiprocessing import ForkProcess from parsl.process_loggers import wrap_with_logs diff --git a/parsl/monitoring/router.py b/parsl/monitoring/router.py index 04e7480a7a..0926712c36 100644 --- a/parsl/monitoring/router.py +++ b/parsl/monitoring/router.py @@ -14,7 +14,7 @@ import zmq from parsl.log_utils import set_file_logger -from parsl.monitoring.radios import MultiprocessingQueueRadioSender +from parsl.monitoring.radios.multiprocessing import MultiprocessingQueueRadioSender from parsl.monitoring.types import TaggedMonitoringMessage from parsl.process_loggers import wrap_with_logs from parsl.utils import setproctitle From a9a5b4b66bae55a75499ff3446972113d8cdf4e6 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 3 Dec 2024 16:36:47 +0000 Subject: [PATCH 15/39] Remove now-unused channel errors (#3713) The final use of channel errors was removed in PR #3690 This is part of #3515 channel removal project # Changed Behaviour Any user catching these errors explicitly will now get an error about the exceptions being undefined. Because these don't exist any more, it should be safe to remove those catches. ## Type of change - Code maintenance/cleanup --- docs/reference.rst | 2 -- parsl/channels/errors.py | 30 ------------------------------ 2 files changed, 32 deletions(-) delete mode 100644 parsl/channels/errors.py diff --git a/docs/reference.rst b/docs/reference.rst index 45f83ad36f..d4c70dd7bd 100644 --- a/docs/reference.rst +++ b/docs/reference.rst @@ -169,8 +169,6 @@ Exceptions parsl.providers.errors.ScaleOutFailed parsl.providers.errors.SchedulerMissingArgs parsl.providers.errors.ScriptPathError - parsl.channels.errors.ChannelError - parsl.channels.errors.FileCopyException parsl.executors.high_throughput.errors.WorkerLost parsl.executors.high_throughput.interchange.ManagerLost parsl.serialize.errors.DeserializationError diff --git a/parsl/channels/errors.py b/parsl/channels/errors.py deleted file mode 100644 index effaea9548..0000000000 --- a/parsl/channels/errors.py +++ /dev/null @@ -1,30 +0,0 @@ -''' Exceptions raise by Apps. -''' -from parsl.errors import ParslError - - -class ChannelError(ParslError): - """ Base class for all exceptions - - Only to be invoked when only a more specific error is not available. - """ - def __init__(self, reason: str, e: Exception, hostname: str) -> None: - self.reason = reason - self.e = e - self.hostname = hostname - - def __str__(self) -> str: - return "Hostname:{0}, Reason:{1}".format(self.hostname, self.reason) - - -class FileCopyException(ChannelError): - ''' File copy operation failed - - Contains: - reason(string) - e (paramiko exception object) - hostname (string) - ''' - - def __init__(self, e: Exception, hostname: str) -> None: - super().__init__("File copy failed due to {0}".format(e), e, hostname) From 76e532844443cd028931dbe53ab63dc6448302de Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 3 Dec 2024 16:48:34 +0000 Subject: [PATCH 16/39] Remove script_dir state from channels (#3714) This leaves channels as a stateless (and useless) object that can be removed in a subsequent PR. That removal will change/break a lot of config files so I want it to keep it separate from this attribute removal. # Changed Behaviour Batch providers will now use their own script_dir attribute, rather than the channel script_dir attribute. Prior to PR #3688 these attributes were paths on different file systems: the submit side (for providers) and the batch job execution side (for channels). PR #3688 removed support for batch job commands to run somewhere that isn't the workflow submit side, and so since then, these two attributes have been roughly equivalent. In some (obscure seeming cases) they might be different: if a channel is shared between DFKs, then the script_dir used by providers in one DFK will now no longer use the channel script dir set by the other DFK. This was probably a bug anyway but I am noting it because this PR isn't completely behaviour preserving. ## Type of change - Code maintenance/cleanup --- parsl/channels/base.py | 21 ++--------------- parsl/channels/local/local.py | 23 +------------------ parsl/dataflow/dflow.py | 2 -- parsl/providers/condor/condor.py | 2 +- parsl/providers/grid_engine/grid_engine.py | 2 +- parsl/providers/lsf/lsf.py | 2 +- parsl/providers/pbspro/pbspro.py | 2 +- parsl/providers/slurm/slurm.py | 6 ++--- parsl/providers/torque/torque.py | 2 +- .../test_providers/test_pbspro_template.py | 1 - .../test_providers/test_slurm_template.py | 1 - 11 files changed, 11 insertions(+), 53 deletions(-) diff --git a/parsl/channels/base.py b/parsl/channels/base.py index 0024d45a6b..4ba3b4e02c 100644 --- a/parsl/channels/base.py +++ b/parsl/channels/base.py @@ -1,22 +1,5 @@ -from abc import ABCMeta, abstractproperty +from abc import ABCMeta class Channel(metaclass=ABCMeta): - @abstractproperty - def script_dir(self) -> str: - ''' This is a property. Returns the directory assigned for storing all internal scripts such as - scheduler submit scripts. This is usually where error logs from the scheduler would reside on the - channel destination side. - - Args: - - None - - Returns: - - Channel script dir - ''' - pass - - # DFK expects to be able to modify this, so it needs to be in the abstract class - @script_dir.setter - def script_dir(self, value: str) -> None: - pass + pass diff --git a/parsl/channels/local/local.py b/parsl/channels/local/local.py index 4c1712ef30..61d8dea17a 100644 --- a/parsl/channels/local/local.py +++ b/parsl/channels/local/local.py @@ -1,5 +1,4 @@ import logging -import os from parsl.channels.base import Channel from parsl.utils import RepresentationMixin @@ -8,24 +7,4 @@ class LocalChannel(Channel, RepresentationMixin): - ''' This is not even really a channel, since opening a local shell is not heavy - and done so infrequently that they do not need a persistent channel - ''' - - def __init__(self): - ''' Initialize the local channel. script_dir is required by set to a default. - - KwArgs: - - script_dir (string): Directory to place scripts - ''' - self.script_dir = None - - @property - def script_dir(self): - return self._script_dir - - @script_dir.setter - def script_dir(self, value): - if value is not None: - value = os.path.abspath(value) - self._script_dir = value + pass diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index 6cec168b5d..e9bf934cf1 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -1151,8 +1151,6 @@ def add_executors(self, executors: Sequence[ParslExecutor]) -> None: executor.provider.script_dir = os.path.join(self.run_dir, 'submit_scripts') os.makedirs(executor.provider.script_dir, exist_ok=True) - executor.provider.channel.script_dir = executor.provider.script_dir - self.executors[executor.label] = executor executor.start() block_executors = [e for e in executors if isinstance(e, BlockProviderExecutor)] diff --git a/parsl/providers/condor/condor.py b/parsl/providers/condor/condor.py index c8142c4026..72fd5aa243 100644 --- a/parsl/providers/condor/condor.py +++ b/parsl/providers/condor/condor.py @@ -226,7 +226,7 @@ def submit(self, command, tasks_per_node, job_name="parsl.condor"): job_config = {} job_config["job_name"] = job_name - job_config["submit_script_dir"] = self.channel.script_dir + job_config["submit_script_dir"] = self.script_dir job_config["project"] = self.project job_config["nodes"] = self.nodes_per_block job_config["scheduler_options"] = scheduler_options diff --git a/parsl/providers/grid_engine/grid_engine.py b/parsl/providers/grid_engine/grid_engine.py index ddedcaa3e8..795f1946b4 100644 --- a/parsl/providers/grid_engine/grid_engine.py +++ b/parsl/providers/grid_engine/grid_engine.py @@ -100,7 +100,7 @@ def get_configs(self, command, tasks_per_node): self.nodes_per_block, tasks_per_node)) job_config = {} - job_config["submit_script_dir"] = self.channel.script_dir + job_config["submit_script_dir"] = self.script_dir job_config["nodes"] = self.nodes_per_block job_config["walltime"] = self.walltime job_config["scheduler_options"] = self.scheduler_options diff --git a/parsl/providers/lsf/lsf.py b/parsl/providers/lsf/lsf.py index b446b063a4..dced93831b 100644 --- a/parsl/providers/lsf/lsf.py +++ b/parsl/providers/lsf/lsf.py @@ -211,7 +211,7 @@ def submit(self, command, tasks_per_node, job_name="parsl.lsf"): logger.debug("Requesting one block with {} nodes".format(self.nodes_per_block)) job_config = {} - job_config["submit_script_dir"] = self.channel.script_dir + job_config["submit_script_dir"] = self.script_dir job_config["nodes"] = self.nodes_per_block job_config["tasks_per_node"] = tasks_per_node job_config["walltime"] = wtime_to_minutes(self.walltime) diff --git a/parsl/providers/pbspro/pbspro.py b/parsl/providers/pbspro/pbspro.py index 71c958f000..b02237a226 100644 --- a/parsl/providers/pbspro/pbspro.py +++ b/parsl/providers/pbspro/pbspro.py @@ -159,7 +159,7 @@ def submit(self, command, tasks_per_node, job_name="parsl"): ) job_config = {} - job_config["submit_script_dir"] = self.channel.script_dir + job_config["submit_script_dir"] = self.script_dir job_config["nodes_per_block"] = self.nodes_per_block job_config["ncpus"] = self.cpus_per_node job_config["walltime"] = self.walltime diff --git a/parsl/providers/slurm/slurm.py b/parsl/providers/slurm/slurm.py index 9b6f38b9d9..92f1a31ad6 100644 --- a/parsl/providers/slurm/slurm.py +++ b/parsl/providers/slurm/slurm.py @@ -3,7 +3,7 @@ import os import re import time -from typing import Optional +from typing import Any, Dict, Optional import typeguard @@ -286,8 +286,8 @@ 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 = {} - job_config["submit_script_dir"] = self.channel.script_dir + job_config: Dict[str, Any] = {} + job_config["submit_script_dir"] = self.script_dir job_config["nodes"] = self.nodes_per_block job_config["tasks_per_node"] = tasks_per_node job_config["walltime"] = wtime_to_minutes(self.walltime) diff --git a/parsl/providers/torque/torque.py b/parsl/providers/torque/torque.py index 7992893abb..c1c778a42a 100644 --- a/parsl/providers/torque/torque.py +++ b/parsl/providers/torque/torque.py @@ -171,7 +171,7 @@ def submit(self, command, tasks_per_node, job_name="parsl.torque"): job_config = {} # TODO : script_path might need to change to accommodate script dir set via channels - job_config["submit_script_dir"] = self.channel.script_dir + job_config["submit_script_dir"] = self.script_dir job_config["nodes"] = self.nodes_per_block job_config["task_blocks"] = self.nodes_per_block * tasks_per_node job_config["nodes_per_block"] = self.nodes_per_block diff --git a/parsl/tests/test_providers/test_pbspro_template.py b/parsl/tests/test_providers/test_pbspro_template.py index dec987ccbb..ef9a642541 100644 --- a/parsl/tests/test_providers/test_pbspro_template.py +++ b/parsl/tests/test_providers/test_pbspro_template.py @@ -15,7 +15,6 @@ def test_submit_script_basic(tmp_path): queue="debug", channel=LocalChannel() ) provider.script_dir = tmp_path - provider.channel.script_dir = tmp_path job_id = str(random.randint(55000, 59000)) provider.execute_wait = mock.Mock(spec=PBSProProvider.execute_wait) provider.execute_wait.return_value = (0, job_id, "") diff --git a/parsl/tests/test_providers/test_slurm_template.py b/parsl/tests/test_providers/test_slurm_template.py index 57fd8e4d0b..ceedfa5ee8 100644 --- a/parsl/tests/test_providers/test_slurm_template.py +++ b/parsl/tests/test_providers/test_slurm_template.py @@ -16,7 +16,6 @@ def test_submit_script_basic(tmp_path): partition="debug", channel=LocalChannel() ) provider.script_dir = tmp_path - provider.channel.script_dir = tmp_path job_id = str(random.randint(55000, 59000)) provider.execute_wait = mock.MagicMock(spec=SlurmProvider.execute_wait) provider.execute_wait.return_value = (0, f"Submitted batch job {job_id}", "") From 63387f1020bb03b325d288c8e11cf0367956b2a0 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 4 Dec 2024 15:29:07 +0000 Subject: [PATCH 17/39] Remove unused channel test files (#3715) ## Type of change - Code maintenance/cleanup --- parsl/tests/integration/test_channels/__init__.py | 0 parsl/tests/integration/test_channels/remote_run.sh | 5 ----- 2 files changed, 5 deletions(-) delete mode 100644 parsl/tests/integration/test_channels/__init__.py delete mode 100644 parsl/tests/integration/test_channels/remote_run.sh diff --git a/parsl/tests/integration/test_channels/__init__.py b/parsl/tests/integration/test_channels/__init__.py deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/parsl/tests/integration/test_channels/remote_run.sh b/parsl/tests/integration/test_channels/remote_run.sh deleted file mode 100644 index aa4945d859..0000000000 --- a/parsl/tests/integration/test_channels/remote_run.sh +++ /dev/null @@ -1,5 +0,0 @@ -#!/bin/bash -echo "Hostname: $HOSTNAME" -echo "Cpu info -----" -cat /proc/cpuinfo -echo "Done----------" From 113061dde91f736488159ac3701f680990efcb00 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 4 Dec 2024 15:32:35 +0000 Subject: [PATCH 18/39] Remove channel classes and parameters (#3716) Before this PR, all channel functionality was removed and the channel classes and config parameters remained as stubs waiting to be removed. This PR removes them. # Changed Behaviour Any place that a user is specifying `channel=LocalChannel()` in their configuration will now break. That option can be removed by the user because that option should not have any effect in the codebase immediately prior to this PR. ## Type of change - Code maintenance/cleanup --- parsl/channels/__init__.py | 4 ---- parsl/channels/base.py | 5 ----- parsl/channels/local/__init__.py | 0 parsl/channels/local/local.py | 10 ---------- parsl/configs/cc_in2p3.py | 2 -- parsl/configs/frontera.py | 2 -- parsl/configs/htex_local.py | 2 -- parsl/providers/cluster_provider.py | 4 ---- parsl/providers/condor/condor.py | 7 ------- parsl/providers/grid_engine/grid_engine.py | 5 ----- parsl/providers/local/local.py | 3 --- parsl/providers/lsf/lsf.py | 5 ----- parsl/providers/pbspro/pbspro.py | 7 +------ parsl/providers/slurm/slurm.py | 6 ------ parsl/providers/torque/torque.py | 6 ------ parsl/tests/configs/cc_in2p3.py | 2 -- parsl/tests/configs/frontera.py | 2 -- parsl/tests/configs/htex_local.py | 2 -- parsl/tests/configs/htex_local_alternate.py | 2 -- parsl/tests/configs/htex_local_intask_staging.py | 2 -- parsl/tests/configs/htex_local_rsync_staging.py | 2 -- parsl/tests/configs/slurm_local.py | 2 -- parsl/tests/manual_tests/htex_local.py | 2 -- parsl/tests/manual_tests/test_memory_limits.py | 2 -- parsl/tests/scaling_tests/htex_local.py | 2 -- parsl/tests/sites/test_affinity.py | 2 -- parsl/tests/sites/test_worker_info.py | 2 -- parsl/tests/test_htex/test_drain.py | 2 -- .../tests/test_htex/test_manager_selector_by_block.py | 2 -- .../test_htex_init_blocks_vs_monitoring.py | 2 -- parsl/tests/test_providers/test_local_provider.py | 3 +-- parsl/tests/test_providers/test_pbspro_template.py | 3 +-- parsl/tests/test_providers/test_slurm_template.py | 3 +-- parsl/tests/test_scaling/test_regression_1621.py | 2 -- .../test_regression_3568_scaledown_vs_MISSING.py | 1 - parsl/tests/test_scaling/test_scale_down.py | 2 -- .../test_scaling/test_scale_down_htex_auto_scale.py | 2 -- .../test_scaling/test_scale_down_htex_unregistered.py | 2 -- parsl/tests/test_scaling/test_shutdown_scalein.py | 2 -- .../test_worker_interchange_bad_messages_3262.py | 2 -- parsl/tests/test_staging/test_zip_in.py | 1 - parsl/tests/test_staging/test_zip_out.py | 1 - parsl/tests/test_staging/test_zip_to_zip.py | 1 - 43 files changed, 4 insertions(+), 119 deletions(-) delete mode 100644 parsl/channels/__init__.py delete mode 100644 parsl/channels/base.py delete mode 100644 parsl/channels/local/__init__.py delete mode 100644 parsl/channels/local/local.py diff --git a/parsl/channels/__init__.py b/parsl/channels/__init__.py deleted file mode 100644 index c81f6a8bf1..0000000000 --- a/parsl/channels/__init__.py +++ /dev/null @@ -1,4 +0,0 @@ -from parsl.channels.base import Channel -from parsl.channels.local.local import LocalChannel - -__all__ = ['Channel', 'LocalChannel'] diff --git a/parsl/channels/base.py b/parsl/channels/base.py deleted file mode 100644 index 4ba3b4e02c..0000000000 --- a/parsl/channels/base.py +++ /dev/null @@ -1,5 +0,0 @@ -from abc import ABCMeta - - -class Channel(metaclass=ABCMeta): - pass diff --git a/parsl/channels/local/__init__.py b/parsl/channels/local/__init__.py deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/parsl/channels/local/local.py b/parsl/channels/local/local.py deleted file mode 100644 index 61d8dea17a..0000000000 --- a/parsl/channels/local/local.py +++ /dev/null @@ -1,10 +0,0 @@ -import logging - -from parsl.channels.base import Channel -from parsl.utils import RepresentationMixin - -logger = logging.getLogger(__name__) - - -class LocalChannel(Channel, RepresentationMixin): - pass diff --git a/parsl/configs/cc_in2p3.py b/parsl/configs/cc_in2p3.py index 631d76f9f5..f0140da55a 100644 --- a/parsl/configs/cc_in2p3.py +++ b/parsl/configs/cc_in2p3.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.providers import GridEngineProvider @@ -10,7 +9,6 @@ label='cc_in2p3_htex', max_workers_per_node=2, provider=GridEngineProvider( - channel=LocalChannel(), nodes_per_block=1, init_blocks=2, max_blocks=2, diff --git a/parsl/configs/frontera.py b/parsl/configs/frontera.py index a7b6f27b6c..25682bbe86 100644 --- a/parsl/configs/frontera.py +++ b/parsl/configs/frontera.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SrunLauncher @@ -15,7 +14,6 @@ max_workers_per_node=1, # Set number of workers per node provider=SlurmProvider( cmd_timeout=60, # Add extra time for slow scheduler responses - channel=LocalChannel(), nodes_per_block=2, init_blocks=1, min_blocks=1, diff --git a/parsl/configs/htex_local.py b/parsl/configs/htex_local.py index 721dea767e..57549a4728 100644 --- a/parsl/configs/htex_local.py +++ b/parsl/configs/htex_local.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.providers import LocalProvider @@ -10,7 +9,6 @@ label="htex_local", cores_per_worker=1, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, ), diff --git a/parsl/providers/cluster_provider.py b/parsl/providers/cluster_provider.py index f2feb2ddf3..db7ef9eaac 100644 --- a/parsl/providers/cluster_provider.py +++ b/parsl/providers/cluster_provider.py @@ -18,8 +18,6 @@ class ClusterProvider(ExecutionProvider): ---------- label : str Label for this provider. - channel : Channel - Channel for accessing this provider. walltime : str Walltime requested per block in HH:MM:SS. launcher : Launcher @@ -45,7 +43,6 @@ class ClusterProvider(ExecutionProvider): def __init__(self, label, - channel, nodes_per_block, init_blocks, min_blocks, @@ -56,7 +53,6 @@ def __init__(self, cmd_timeout=10): self._label = label - self.channel = channel self.nodes_per_block = nodes_per_block self.init_blocks = init_blocks self.min_blocks = min_blocks diff --git a/parsl/providers/condor/condor.py b/parsl/providers/condor/condor.py index 72fd5aa243..150c72dfe7 100644 --- a/parsl/providers/condor/condor.py +++ b/parsl/providers/condor/condor.py @@ -5,7 +5,6 @@ import typeguard -from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher from parsl.launchers.base import Launcher @@ -18,8 +17,6 @@ from typing import Dict, List, Optional -from parsl.channels.base import Channel - # See http://pages.cs.wisc.edu/~adesmet/status.html translate_table = { '1': JobState.PENDING, @@ -36,8 +33,6 @@ class CondorProvider(RepresentationMixin, ClusterProvider): Parameters ---------- - channel : Channel - Channel for accessing this provider. nodes_per_block : int Nodes to provision per block. cores_per_slot : int @@ -79,7 +74,6 @@ class CondorProvider(RepresentationMixin, ClusterProvider): """ @typeguard.typechecked def __init__(self, - channel: Channel = LocalChannel(), nodes_per_block: int = 1, cores_per_slot: Optional[int] = None, mem_per_slot: Optional[float] = None, @@ -100,7 +94,6 @@ def __init__(self, label = 'condor' super().__init__(label, - channel, nodes_per_block, init_blocks, min_blocks, diff --git a/parsl/providers/grid_engine/grid_engine.py b/parsl/providers/grid_engine/grid_engine.py index 795f1946b4..b01e86cd5d 100644 --- a/parsl/providers/grid_engine/grid_engine.py +++ b/parsl/providers/grid_engine/grid_engine.py @@ -2,7 +2,6 @@ import os import time -from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher from parsl.providers.cluster_provider import ClusterProvider @@ -36,8 +35,6 @@ class GridEngineProvider(ClusterProvider, RepresentationMixin): Parameters ---------- - channel : Channel - Channel for accessing this provider. nodes_per_block : int Nodes to provision per block. min_blocks : int @@ -62,7 +59,6 @@ class GridEngineProvider(ClusterProvider, RepresentationMixin): """ def __init__(self, - channel=LocalChannel(), nodes_per_block=1, init_blocks=1, min_blocks=0, @@ -76,7 +72,6 @@ def __init__(self, queue=None): label = 'grid_engine' super().__init__(label, - channel, nodes_per_block, init_blocks, min_blocks, diff --git a/parsl/providers/local/local.py b/parsl/providers/local/local.py index 6357c85cba..55994c31c3 100644 --- a/parsl/providers/local/local.py +++ b/parsl/providers/local/local.py @@ -2,7 +2,6 @@ import os import time -from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher from parsl.providers.base import ExecutionProvider @@ -37,7 +36,6 @@ class LocalProvider(ExecutionProvider, RepresentationMixin): """ def __init__(self, - channel=LocalChannel(), nodes_per_block=1, launcher=SingleNodeLauncher(), init_blocks=1, @@ -46,7 +44,6 @@ def __init__(self, worker_init='', cmd_timeout=30, parallelism=1): - self.channel = channel self._label = 'local' self.nodes_per_block = nodes_per_block self.launcher = launcher diff --git a/parsl/providers/lsf/lsf.py b/parsl/providers/lsf/lsf.py index dced93831b..f2d3f88652 100644 --- a/parsl/providers/lsf/lsf.py +++ b/parsl/providers/lsf/lsf.py @@ -3,7 +3,6 @@ import os import time -from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher from parsl.providers.cluster_provider import ClusterProvider @@ -32,8 +31,6 @@ class LSFProvider(ClusterProvider, RepresentationMixin): Parameters ---------- - channel : Channel - Channel for accessing this provider. nodes_per_block : int Nodes to provision per block. When request_by_nodes is False, it is computed by cores_per_block / cores_per_node. @@ -77,7 +74,6 @@ class LSFProvider(ClusterProvider, RepresentationMixin): """ def __init__(self, - channel=LocalChannel(), nodes_per_block=1, cores_per_block=None, cores_per_node=None, @@ -96,7 +92,6 @@ def __init__(self, launcher=SingleNodeLauncher()): label = 'LSF' super().__init__(label, - channel, nodes_per_block, init_blocks, min_blocks, diff --git a/parsl/providers/pbspro/pbspro.py b/parsl/providers/pbspro/pbspro.py index b02237a226..aa2e5e2f54 100644 --- a/parsl/providers/pbspro/pbspro.py +++ b/parsl/providers/pbspro/pbspro.py @@ -3,7 +3,6 @@ import os import time -from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher from parsl.providers.pbspro.template import template_string @@ -17,8 +16,6 @@ class PBSProProvider(TorqueProvider): Parameters ---------- - channel : Channel - Channel for accessing this provider. account : str Account the job will be charged against. queue : str @@ -51,7 +48,6 @@ class PBSProProvider(TorqueProvider): :class:`~parsl.launchers.SingleNodeLauncher`. """ def __init__(self, - channel=LocalChannel(), account=None, queue=None, scheduler_options='', @@ -66,8 +62,7 @@ def __init__(self, launcher=SingleNodeLauncher(), walltime="00:20:00", cmd_timeout=120): - super().__init__(channel, - account, + super().__init__(account, queue, scheduler_options, worker_init, diff --git a/parsl/providers/slurm/slurm.py b/parsl/providers/slurm/slurm.py index 92f1a31ad6..cf50058522 100644 --- a/parsl/providers/slurm/slurm.py +++ b/parsl/providers/slurm/slurm.py @@ -7,8 +7,6 @@ import typeguard -from parsl.channels import LocalChannel -from parsl.channels.base import Channel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import SingleNodeLauncher from parsl.launchers.base import Launcher @@ -73,8 +71,6 @@ class SlurmProvider(ClusterProvider, RepresentationMixin): clusters : str Slurm cluster name, or comma seperated cluster list, used to choose between different clusters in a federated Slurm instance. If unspecified or ``None``, no slurm directive for clusters will be added. - channel : Channel - Channel for accessing this provider. nodes_per_block : int Nodes to provision per block. cores_per_node : int @@ -119,7 +115,6 @@ def __init__(self, qos: Optional[str] = None, constraint: Optional[str] = None, clusters: Optional[str] = None, - channel: Channel = LocalChannel(), nodes_per_block: int = 1, cores_per_node: Optional[int] = None, mem_per_node: Optional[int] = None, @@ -136,7 +131,6 @@ def __init__(self, launcher: Launcher = SingleNodeLauncher()): label = 'slurm' super().__init__(label, - channel, nodes_per_block, init_blocks, min_blocks, diff --git a/parsl/providers/torque/torque.py b/parsl/providers/torque/torque.py index c1c778a42a..6958c8348a 100644 --- a/parsl/providers/torque/torque.py +++ b/parsl/providers/torque/torque.py @@ -2,7 +2,6 @@ import os import time -from parsl.channels import LocalChannel from parsl.jobs.states import JobState, JobStatus from parsl.launchers import AprunLauncher from parsl.providers.cluster_provider import ClusterProvider @@ -33,8 +32,6 @@ class TorqueProvider(ClusterProvider, RepresentationMixin): Parameters ---------- - channel : Channel - Channel for accessing this provider. account : str Account the job will be charged against. queue : str @@ -65,7 +62,6 @@ class TorqueProvider(ClusterProvider, RepresentationMixin): """ def __init__(self, - channel=LocalChannel(), account=None, queue=None, scheduler_options='', @@ -80,7 +76,6 @@ def __init__(self, cmd_timeout=120): label = 'torque' super().__init__(label, - channel, nodes_per_block, init_blocks, min_blocks, @@ -170,7 +165,6 @@ def submit(self, command, tasks_per_node, job_name="parsl.torque"): tasks_per_node) job_config = {} - # TODO : script_path might need to change to accommodate script dir set via channels job_config["submit_script_dir"] = self.script_dir job_config["nodes"] = self.nodes_per_block job_config["task_blocks"] = self.nodes_per_block * tasks_per_node diff --git a/parsl/tests/configs/cc_in2p3.py b/parsl/tests/configs/cc_in2p3.py index 38c817cccd..5d9edf9d0c 100644 --- a/parsl/tests/configs/cc_in2p3.py +++ b/parsl/tests/configs/cc_in2p3.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.providers import GridEngineProvider @@ -14,7 +13,6 @@ def fresh_config(): max_workers_per_node=1, encrypted=True, provider=GridEngineProvider( - channel=LocalChannel(), nodes_per_block=2, init_blocks=2, max_blocks=2, diff --git a/parsl/tests/configs/frontera.py b/parsl/tests/configs/frontera.py index 537d6f78e0..41e2985e41 100644 --- a/parsl/tests/configs/frontera.py +++ b/parsl/tests/configs/frontera.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SrunLauncher @@ -20,7 +19,6 @@ def fresh_config(): encrypted=True, provider=SlurmProvider( cmd_timeout=60, # Add extra time for slow scheduler responses - channel=LocalChannel(), nodes_per_block=2, init_blocks=1, min_blocks=1, diff --git a/parsl/tests/configs/htex_local.py b/parsl/tests/configs/htex_local.py index 3dae57b758..712570b628 100644 --- a/parsl/tests/configs/htex_local.py +++ b/parsl/tests/configs/htex_local.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SimpleLauncher @@ -15,7 +14,6 @@ def fresh_config(): cores_per_worker=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, launcher=SimpleLauncher(), diff --git a/parsl/tests/configs/htex_local_alternate.py b/parsl/tests/configs/htex_local_alternate.py index 5667f3ff8c..cc69d56186 100644 --- a/parsl/tests/configs/htex_local_alternate.py +++ b/parsl/tests/configs/htex_local_alternate.py @@ -17,7 +17,6 @@ import os -from parsl.channels import LocalChannel from parsl.config import Config from parsl.data_provider.file_noop import NoOpFileStaging from parsl.data_provider.ftp import FTPInTaskStaging @@ -48,7 +47,6 @@ def fresh_config(): poll_period=100, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=0, min_blocks=0, max_blocks=5, diff --git a/parsl/tests/configs/htex_local_intask_staging.py b/parsl/tests/configs/htex_local_intask_staging.py index eef88ea2e1..69649b2bcb 100644 --- a/parsl/tests/configs/htex_local_intask_staging.py +++ b/parsl/tests/configs/htex_local_intask_staging.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.data_provider.file_noop import NoOpFileStaging from parsl.data_provider.ftp import FTPInTaskStaging @@ -15,7 +14,6 @@ cores_per_worker=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, launcher=SimpleLauncher(), diff --git a/parsl/tests/configs/htex_local_rsync_staging.py b/parsl/tests/configs/htex_local_rsync_staging.py index 044d2b9f89..d24b1da66b 100644 --- a/parsl/tests/configs/htex_local_rsync_staging.py +++ b/parsl/tests/configs/htex_local_rsync_staging.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.data_provider.ftp import FTPInTaskStaging from parsl.data_provider.http import HTTPInTaskStaging @@ -16,7 +15,6 @@ working_dir="./rsync-workdir/", encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, launcher=SimpleLauncher(), diff --git a/parsl/tests/configs/slurm_local.py b/parsl/tests/configs/slurm_local.py index 2a63f68e51..c0281cf340 100644 --- a/parsl/tests/configs/slurm_local.py +++ b/parsl/tests/configs/slurm_local.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SrunLauncher @@ -13,7 +12,6 @@ def fresh_config(): encrypted=True, provider=SlurmProvider( cmd_timeout=60, # Add extra time for slow scheduler responses - channel=LocalChannel(), nodes_per_block=1, init_blocks=1, min_blocks=1, diff --git a/parsl/tests/manual_tests/htex_local.py b/parsl/tests/manual_tests/htex_local.py index bcfdbf34ec..e85da878b1 100644 --- a/parsl/tests/manual_tests/htex_local.py +++ b/parsl/tests/manual_tests/htex_local.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor @@ -15,7 +14,6 @@ cores_per_worker=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, # tasks_per_node=1, # For HighThroughputExecutor, this option should in most cases be 1 diff --git a/parsl/tests/manual_tests/test_memory_limits.py b/parsl/tests/manual_tests/test_memory_limits.py index b353ba0929..f98a779e82 100644 --- a/parsl/tests/manual_tests/test_memory_limits.py +++ b/parsl/tests/manual_tests/test_memory_limits.py @@ -5,7 +5,6 @@ import parsl from parsl.app.app import python_app # , bash_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SingleNodeLauncher @@ -30,7 +29,6 @@ def test_simple(mem_per_worker): suppress_failure=True, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, launcher=SingleNodeLauncher(), diff --git a/parsl/tests/scaling_tests/htex_local.py b/parsl/tests/scaling_tests/htex_local.py index f2bcf86963..f16f46af23 100644 --- a/parsl/tests/scaling_tests/htex_local.py +++ b/parsl/tests/scaling_tests/htex_local.py @@ -1,4 +1,3 @@ -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.providers import LocalProvider @@ -12,7 +11,6 @@ max_workers_per_node=8, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, ), diff --git a/parsl/tests/sites/test_affinity.py b/parsl/tests/sites/test_affinity.py index 50d08ce830..792d490e03 100644 --- a/parsl/tests/sites/test_affinity.py +++ b/parsl/tests/sites/test_affinity.py @@ -5,7 +5,6 @@ import pytest from parsl import python_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.providers import LocalProvider @@ -22,7 +21,6 @@ def local_config(): available_accelerators=2, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, ), diff --git a/parsl/tests/sites/test_worker_info.py b/parsl/tests/sites/test_worker_info.py index 4d9865da84..e55064a507 100644 --- a/parsl/tests/sites/test_worker_info.py +++ b/parsl/tests/sites/test_worker_info.py @@ -3,7 +3,6 @@ import pytest from parsl import python_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.providers import LocalProvider @@ -18,7 +17,6 @@ def local_config(): max_workers_per_node=4, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=1, ), diff --git a/parsl/tests/test_htex/test_drain.py b/parsl/tests/test_htex/test_drain.py index efd0405863..663978f4c8 100644 --- a/parsl/tests/test_htex/test_drain.py +++ b/parsl/tests/test_htex/test_drain.py @@ -3,7 +3,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SimpleLauncher @@ -28,7 +27,6 @@ def local_config(): cores_per_worker=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, min_blocks=0, max_blocks=0, diff --git a/parsl/tests/test_htex/test_manager_selector_by_block.py b/parsl/tests/test_htex/test_manager_selector_by_block.py index 0933b581ff..1b2a4ee1f7 100644 --- a/parsl/tests/test_htex/test_manager_selector_by_block.py +++ b/parsl/tests/test_htex/test_manager_selector_by_block.py @@ -4,7 +4,6 @@ import parsl from parsl.app.app import bash_app, python_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.executors.high_throughput.manager_selector import ( @@ -31,7 +30,6 @@ def test_block_id_selection(try_assert): max_workers_per_node=1, manager_selector=BlockIdManagerSelector(), provider=LocalProvider( - channel=LocalChannel(), init_blocks=BLOCK_COUNT, max_blocks=BLOCK_COUNT, min_blocks=BLOCK_COUNT, diff --git a/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py b/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py index c54486f011..67db42671c 100644 --- a/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py +++ b/parsl/tests/test_monitoring/test_htex_init_blocks_vs_monitoring.py @@ -5,7 +5,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SimpleLauncher @@ -22,7 +21,6 @@ def fresh_config(run_dir, strategy, db_url): cores_per_worker=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, # min and max are set to 0 to ensure that we don't get # a block from ongoing strategy scaling, only from diff --git a/parsl/tests/test_providers/test_local_provider.py b/parsl/tests/test_providers/test_local_provider.py index 8d56d89ddf..c750814463 100644 --- a/parsl/tests/test_providers/test_local_provider.py +++ b/parsl/tests/test_providers/test_local_provider.py @@ -11,7 +11,6 @@ import pytest -from parsl.channels import LocalChannel from parsl.jobs.states import JobState from parsl.launchers import SingleNodeLauncher from parsl.providers import LocalProvider @@ -63,7 +62,7 @@ def _run_tests(p: LocalProvider): def test_local_channel(): with tempfile.TemporaryDirectory() as script_dir: script_dir = tempfile.mkdtemp() - p = LocalProvider(channel=LocalChannel(), launcher=SingleNodeLauncher(debug=False)) + p = LocalProvider(launcher=SingleNodeLauncher(debug=False)) p.script_dir = script_dir _run_tests(p) diff --git a/parsl/tests/test_providers/test_pbspro_template.py b/parsl/tests/test_providers/test_pbspro_template.py index ef9a642541..1264731120 100644 --- a/parsl/tests/test_providers/test_pbspro_template.py +++ b/parsl/tests/test_providers/test_pbspro_template.py @@ -3,7 +3,6 @@ import pytest -from parsl.channels import LocalChannel from parsl.providers import PBSProProvider @@ -12,7 +11,7 @@ def test_submit_script_basic(tmp_path): """Test slurm resources table""" provider = PBSProProvider( - queue="debug", channel=LocalChannel() + queue="debug" ) provider.script_dir = tmp_path job_id = str(random.randint(55000, 59000)) diff --git a/parsl/tests/test_providers/test_slurm_template.py b/parsl/tests/test_providers/test_slurm_template.py index ceedfa5ee8..55074fefe3 100644 --- a/parsl/tests/test_providers/test_slurm_template.py +++ b/parsl/tests/test_providers/test_slurm_template.py @@ -4,7 +4,6 @@ import pytest -from parsl.channels import LocalChannel from parsl.providers import SlurmProvider @@ -13,7 +12,7 @@ def test_submit_script_basic(tmp_path): """Test slurm resources table""" provider = SlurmProvider( - partition="debug", channel=LocalChannel() + partition="debug" ) provider.script_dir = tmp_path job_id = str(random.randint(55000, 59000)) diff --git a/parsl/tests/test_scaling/test_regression_1621.py b/parsl/tests/test_scaling/test_regression_1621.py index 5aec750068..ea7d8f2631 100644 --- a/parsl/tests/test_scaling/test_regression_1621.py +++ b/parsl/tests/test_scaling/test_regression_1621.py @@ -3,7 +3,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SimpleLauncher @@ -43,7 +42,6 @@ def test_one_block(tmpd_cwd): one app is invoked. this is a regression test. """ oneshot_provider = OneShotLocalProvider( - channel=LocalChannel(), init_blocks=0, min_blocks=0, max_blocks=10, diff --git a/parsl/tests/test_scaling/test_regression_3568_scaledown_vs_MISSING.py b/parsl/tests/test_scaling/test_regression_3568_scaledown_vs_MISSING.py index a56b53af10..0c4a474b19 100644 --- a/parsl/tests/test_scaling/test_regression_3568_scaledown_vs_MISSING.py +++ b/parsl/tests/test_scaling/test_regression_3568_scaledown_vs_MISSING.py @@ -3,7 +3,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import WrappedLauncher diff --git a/parsl/tests/test_scaling/test_scale_down.py b/parsl/tests/test_scaling/test_scale_down.py index 7fb72ba507..a53630374f 100644 --- a/parsl/tests/test_scaling/test_scale_down.py +++ b/parsl/tests/test_scaling/test_scale_down.py @@ -5,7 +5,6 @@ import parsl from parsl import File, python_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SingleNodeLauncher @@ -29,7 +28,6 @@ def local_config(): max_workers_per_node=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=0, max_blocks=_max_blocks, min_blocks=_min_blocks, 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 index 016a51dc48..831bdf82af 100644 --- a/parsl/tests/test_scaling/test_scale_down_htex_auto_scale.py +++ b/parsl/tests/test_scaling/test_scale_down_htex_auto_scale.py @@ -4,7 +4,6 @@ import parsl from parsl import File, python_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SingleNodeLauncher @@ -26,7 +25,6 @@ def local_config(): max_workers_per_node=1, encrypted=True, provider=LocalProvider( - channel=LocalChannel(), init_blocks=0, max_blocks=_max_blocks, min_blocks=_min_blocks, diff --git a/parsl/tests/test_scaling/test_scale_down_htex_unregistered.py b/parsl/tests/test_scaling/test_scale_down_htex_unregistered.py index 529877eac7..90a9b9ff1b 100644 --- a/parsl/tests/test_scaling/test_scale_down_htex_unregistered.py +++ b/parsl/tests/test_scaling/test_scale_down_htex_unregistered.py @@ -5,7 +5,6 @@ import parsl from parsl import File, python_app -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.jobs.states import TERMINAL_STATES, JobState @@ -31,7 +30,6 @@ def local_config(): encrypted=True, launch_cmd="sleep inf", provider=LocalProvider( - channel=LocalChannel(), init_blocks=1, max_blocks=_max_blocks, min_blocks=_min_blocks, diff --git a/parsl/tests/test_scaling/test_shutdown_scalein.py b/parsl/tests/test_scaling/test_shutdown_scalein.py index 1d1557ebb1..2505c79aca 100644 --- a/parsl/tests/test_scaling/test_shutdown_scalein.py +++ b/parsl/tests/test_scaling/test_shutdown_scalein.py @@ -4,7 +4,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SimpleLauncher @@ -47,7 +46,6 @@ def test_shutdown_scalein_blocks(tmpd_cwd, try_assert): scaled in at DFK shutdown. """ accumulating_provider = AccumulatingLocalProvider( - channel=LocalChannel(), init_blocks=BLOCK_COUNT, min_blocks=0, max_blocks=0, diff --git a/parsl/tests/test_scaling/test_worker_interchange_bad_messages_3262.py b/parsl/tests/test_scaling/test_worker_interchange_bad_messages_3262.py index eee128634e..414f67cab6 100644 --- a/parsl/tests/test_scaling/test_worker_interchange_bad_messages_3262.py +++ b/parsl/tests/test_scaling/test_worker_interchange_bad_messages_3262.py @@ -6,7 +6,6 @@ import zmq import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.executors import HighThroughputExecutor from parsl.launchers import SimpleLauncher @@ -24,7 +23,6 @@ def fresh_config(): cores_per_worker=1, encrypted=False, provider=LocalProvider( - channel=LocalChannel(), init_blocks=0, min_blocks=0, max_blocks=0, diff --git a/parsl/tests/test_staging/test_zip_in.py b/parsl/tests/test_staging/test_zip_in.py index 1f74f7e11b..9d43a4ab49 100644 --- a/parsl/tests/test_staging/test_zip_in.py +++ b/parsl/tests/test_staging/test_zip_in.py @@ -4,7 +4,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.data_provider.files import File from parsl.data_provider.zip import ZipAuthorityError, ZipFileStaging diff --git a/parsl/tests/test_staging/test_zip_out.py b/parsl/tests/test_staging/test_zip_out.py index e369031033..79fbb504d5 100644 --- a/parsl/tests/test_staging/test_zip_out.py +++ b/parsl/tests/test_staging/test_zip_out.py @@ -3,7 +3,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.data_provider.data_manager import default_staging from parsl.data_provider.files import File diff --git a/parsl/tests/test_staging/test_zip_to_zip.py b/parsl/tests/test_staging/test_zip_to_zip.py index 2c78e3bec2..3fea42167c 100644 --- a/parsl/tests/test_staging/test_zip_to_zip.py +++ b/parsl/tests/test_staging/test_zip_to_zip.py @@ -4,7 +4,6 @@ import pytest import parsl -from parsl.channels import LocalChannel from parsl.config import Config from parsl.data_provider.files import File from parsl.data_provider.zip import ZipAuthorityError, ZipFileStaging From a01f7e4b9b00ae011d7a531e6777f50decfe9604 Mon Sep 17 00:00:00 2001 From: Sophie Bui Date: Mon, 9 Dec 2024 16:50:20 -0600 Subject: [PATCH 19/39] Add NumFOCUS badge (#3719) # Description To complete our NumFOCUS onboarding, we must add their badge to our repository and website. Adding the "Powered by NumFOCUS" badge to our README. # Changed Behaviour N/A # Fixes N/A ## Type of change Choose which options apply, and delete the ones which do not apply. - Update to human readable text: Documentation/error messages/comments --- README.rst | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/README.rst b/README.rst index a8254e2e40..20988d2d86 100644 --- a/README.rst +++ b/README.rst @@ -1,6 +1,6 @@ Parsl - Parallel Scripting Library ================================== -|licence| |docs| |NSF-1550588| |NSF-1550476| |NSF-1550562| |NSF-1550528| |CZI-EOSS| +|licence| |docs| |NSF-1550588| |NSF-1550476| |NSF-1550562| |NSF-1550528| |NumFOCUS| |CZI-EOSS| Parsl extends parallelism in Python beyond a single computer. @@ -64,6 +64,9 @@ then explore the `parallel computing patterns Date: Sat, 14 Dec 2024 13:48:31 -0500 Subject: [PATCH 20/39] Make radical bulk collector exit more explicitly (#3718) In addition to the work happening [here](https://github.com/radical-cybertools/radical.pilot/pull/3269) to address this issue https://github.com/Parsl/parsl/issues/3708. This PR ensures the cleanup of threads generated by RPEX. - Bug fix (partially https://github.com/Parsl/parsl/issues/3708) --------- Co-authored-by: Ben Clifford --- parsl/executors/radical/executor.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/parsl/executors/radical/executor.py b/parsl/executors/radical/executor.py index 93b4b38bbd..aa191f49e5 100644 --- a/parsl/executors/radical/executor.py +++ b/parsl/executors/radical/executor.py @@ -133,6 +133,7 @@ def __init__(self, self.resource = resource self._uid = RPEX.lower() self.bulk_mode = bulk_mode + self._terminate = mt.Event() self.working_dir = working_dir self.pilot_kwargs = rpex_pilot_kwargs self.future_tasks: Dict[str, Future] = {} @@ -532,7 +533,7 @@ def _bulk_collector(self): bulk = list() - while True: + while not self._terminate.is_set(): now = time.time() # time of last submission @@ -552,6 +553,9 @@ def _bulk_collector(self): if len(bulk) >= self._max_bulk_size: break + if self._terminate.is_set(): + break + if bulk: logger.debug('submit bulk: %d', len(bulk)) self.tmgr.submit_tasks(bulk) @@ -588,6 +592,13 @@ def submit(self, func, resource_specification, *args, **kwargs): def shutdown(self, hub=True, targets='all', block=False): """Shutdown the executor, including all RADICAL-Pilot components.""" logger.info("RadicalPilotExecutor is terminating...") + + self._terminate.set() + + # ensure we are in the bulk submssion mode + if self.bulk_mode: + self._bulk_thread.join() + self.session.close(download=True) logger.info("RadicalPilotExecutor is terminated.") From cbe3d3041597d325534d6c443f473aca37f06a18 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 17 Dec 2024 13:57:47 +0000 Subject: [PATCH 21/39] Upgrade RADICAL-Pilot version pin (#3725) This brings in fixes for a few issues that are fixed on the radical side of things: #3722 - a race condition on task completion #3708 - cleaner shutdown handling as part of #3397 #3646 - Python 3.13 support # Changed Behaviour whatever has changed in radical-pilot # Fixes #3722 ## Type of change - Bug fix --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index cace8c0252..bed98934e7 100755 --- a/setup.py +++ b/setup.py @@ -33,7 +33,7 @@ 'workqueue': ['work_queue'], 'flux': ['pyyaml', 'cffi', 'jsonschema'], 'proxystore': ['proxystore'], - 'radical-pilot': ['radical.pilot==1.60', 'radical.utils==1.60'], + 'radical-pilot': ['radical.pilot==1.90', 'radical.utils==1.90'], # Disabling psi-j since github direct links are not allowed by pypi # 'psij': ['psi-j-parsl@git+https://github.com/ExaWorks/psi-j-parsl'] } From 833310c801435eb89d9ec6c7df33a8748176c949 Mon Sep 17 00:00:00 2001 From: Logan Ward Date: Sun, 29 Dec 2024 12:25:04 -0500 Subject: [PATCH 22/39] Clean up glossary formatting (#3730) # Description Makes the style of the glossary section for the documentation match the rest of the docs better. - Only have it appear once in table of contents - Remove bolding from section titles (Sphinx style already bolds them) - Remove : from end of section title (to match other rst documents) - Make underscores for titles match length (mismatch just bugs me) # Changed Behaviour N/A # Fixes Fixes #3426 (we've got the general prototype done) ## Type of change - Update to human readable text: Documentation/error messages/comments --- docs/index.rst | 2 +- docs/userguide/glossary.rst | 125 ++++++++++++++++++------------------ docs/userguide/index.rst | 1 - 3 files changed, 63 insertions(+), 65 deletions(-) diff --git a/docs/index.rst b/docs/index.rst index c5057c0899..88b0c7bb4c 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -107,7 +107,7 @@ Table of Contents quickstart 1-parsl-introduction.ipynb userguide/index - userguide/glossary + userguide/glossary faq reference devguide/index diff --git a/docs/userguide/glossary.rst b/docs/userguide/glossary.rst index a1a773b6a8..62a735ff88 100644 --- a/docs/userguide/glossary.rst +++ b/docs/userguide/glossary.rst @@ -1,5 +1,5 @@ -Glossary of Parsl Terms -======================= +Glossary +======== This glossary defines terms based on their usage within Parsl. By defining our terminology, we hope to create understanding across our community and reduce confusion. When asking for or providing support to fellow Parsl users, please use these terms as defined. @@ -9,211 +9,210 @@ Our glossary is organized alphabetically in English. Feel free to contribute ter .. _appglossary: -**App:** ----------- +App +--- In Parsl, an app is a small, self-contained program that does a specific job. It's a piece of code, such as a Python function or a Bash script, that can run separately from your main program. Think of it as a mini-tool within your larger toolbox. .. _appfutureglossary: -**AppFuture:** ------------------ +AppFuture +--------- An AppFuture is a placeholder for the result of an app that runs in the background. It's like a ticket you get when you order food at a restaurant – you get the ticket right away, but you have to wait for the food to be ready. Similarly, you get an AppFuture immediately when you start an app, but you have to wait for the app to finish before you can see the results. .. _bashappglossary: -**Bash App:** ---------------- +Bash App +-------- A Bash app is a special kind of app in Parsl that lets you run commands from your computer's terminal (like the ones you type in the command prompt or shell). It's a way to use Parsl to automate tasks that you would normally do manually in the terminal. .. _blockglossary: -**Block:** ------------- +Block +----- A block is a group of resources, such as nodes or computational units, allocated for executing tasks. Parsl manages the distribution of work across these resources to expedite task completion. .. _checkpointingglossary: -**Checkpointing:** ---------------------- +Checkpointing +------------- Checkpointing is like saving your progress in a video game. If something goes wrong, you can restart from the last saved point instead of starting over. In Parsl, checkpointing saves the state of your work so you can resume it later if interrupted. .. _concurrencyglossary: -**Concurrency:** -------------------- +Concurrency +----------- Concurrency means doing multiple things at the same time. In Parsl, it enables your apps to run in parallel across different resources, significantly speeding up program execution. It's like a chef preparing multiple dishes in a single kitchen, switching between all of them quickly. .. _configurationglossary: -**Configuration:** ---------------------- +Configuration +------------- Configuration sets up the rules for how Parsl should work. It's like adjusting the settings on your phone – you can choose how you want things to look and behave. In Parsl, you can configure things like how many resources to use, where to store data, and how to handle errors. .. _datafutureglossary: -**DataFuture:** ------------------- +DataFuture +---------- A DataFuture is a placeholder for a file that an app is creating. It's like a receipt for a package you're expecting – you get the receipt right away, but you have to wait for the package to arrive. Similarly, you get a DataFuture immediately when an app starts creating a file, but you have to wait for the file to be finished before you can use it. .. _dfkglossary: -**DataFlowKernel (DFK):** ------------------------------- +DataFlowKernel (DFK) +-------------------- The DataFlowKernel is like the brain of Parsl. It's the part that controls how your apps run and how they share information. It's like the conductor of an orchestra, making sure that all the musicians play together in harmony. .. _elasticityglossary: -**Elasticity:** ------------------ +Elasticity +---------- Elasticity refers to the ability to scale resources up or down as needed. In Parsl, it allows you to add or remove blocks of computational resources based on workload demands. .. _executionproviderglossary: -**Execution Provider:** --------------------------- +Execution Provider +------------------ An execution provider acts as a bridge between Parsl and the resources you want to use, such as your laptop, a cluster, or a cloud service. It handles communication with these resources to execute tasks. .. _executorglossary: -**Executor:** ----------------- +Executor +-------- An executor is a manager that determines which app runs on which resource and when. It directs the flow of apps to ensure efficient task execution. It's like a traffic controller, directing the flow of apps to make sure they all get where they need to go. .. _futureglossary: -**Future:** -------------- +Future +------ A future is a placeholder for the result of a task that hasn't finished yet. Both AppFuture and DataFuture are types of Futures. You can use the ``.result()`` method to get the actual result when it's ready. .. _jobglossary: -**Job:** ---------- +Job +--- A job in Parsl is a unit of work submitted to an execution environment (such as a cluster or cloud) for processing. It can consist of one or more apps executed on computational resources. .. _launcherglossary: -**Launcher:** ----------------- +Launcher +-------- A launcher in Parsl is responsible for placing the workers onto each computer, preparing them to run the apps. It’s like a bus driver who brings the players to the stadium, ensuring they are ready to start, but not directly involved in telling them what to do once they arrive. .. _managerglossary: -**Manager:** --------------- +Manager +------- A manager in Parsl is responsible for overseeing the execution of tasks on specific compute resources. It's like a supervisor who ensures that all workers (or workers within a block) are carrying out their tasks correctly and efficiently. .. _memoizationglossary: -**Memoization:** -------------------- +Memoization +----------- Memoization is like remembering something so you don't have to do it again. In Parsl, if you are using memoization and you run an app with the same inputs multiple times, Parsl will remember the result from the first time and give it to you again instead of running the app again. This can save a lot of time. .. _mpiappglossary: -**MPI App:** ---------------- +MPI App +------- An MPI app is a specialized app that uses the Message Passing Interface (MPI) for communication, which can occur both across nodes and within a single node. MPI enables different parts of the app to communicate and coordinate their activities, similar to how a walkie-talkie allows different teams to stay in sync. .. _nodeglossary: -**Node:** ------------- +Node +---- A node in Parsl is like a workstation in a factory. It's a physical or virtual machine that provides the computational power needed to run tasks. Each node can host several workers that execute tasks. .. _parallelismglossary: -**Parallelism:** -------------------- +Parallelism +----------- Parallelism means doing multiple things at the same time but not necessarily in the same location or using the same resources. In Parsl, it involves running apps simultaneously across different nodes or computational resources, accelerating program execution. Unlike concurrency which is like a chef preparing multiple dishes in a single kitchen, parallelism is like multiple chefs preparing different dishes in separate kitchens, at the same time. .. _parslscriptglossary: -**Parsl Script:** ---------------------- +Parsl Script +------------ A Parsl script is a Python program that uses the Parsl library to define and run apps in parallel. It's like a recipe that tells you what ingredients to use and how to combine them. .. _pluginglossary: -**Plugin:** ---------------- +Plugin +------ A plugin is an add-on for Parsl. It's a piece of code that you can add to Parsl to give it new features or change how it works. It's like an extra tool that you can add to your toolbox. .. _pythonappglossary: -**Python App:** ------------------- +Python App +---------- A Python app is a special kind of app in Parsl that's written as a Python function. It's a way to use Parsl to run your Python code in parallel. .. _resourceglossary: -**Resource:** ---------------- +Resource +-------- A resource in Parsl refers to any computational asset that can be used to execute tasks, such as CPU cores, memory, or entire nodes. It's like the tools and materials you need to get a job done. Resources, often grouped in nodes or clusters, are essential for processing workloads. .. _serializationglossary: -**Serialization:** --------------------- +Serialization +------------- Serialization is like packing your belongings into a suitcase so you can take them on a trip. In Parsl, it means converting your data into a format that can be sent over a network to another computer. .. _stagingglossary: -**Staging:** ---------------- +Staging +------- Staging in Parsl involves moving data to the appropriate location before an app starts running and can also include moving data back after the app finishes. This process ensures that all necessary data is available where it needs to be for the app to execute properly and that the output data is returned to a specified location once the execution is complete. .. _taskglossary: -**Task:** ------------- +Task +---- A task in Parsl is the execution of an app, it is the smallest unit of work that can be executed. It's like a single step in a larger process, where each task is part of a broader workflow or job. .. _threadglossary: -**Thread:** -------------- +Thread +------ A thread is like a smaller part of a program that can run independently. It's like a worker in a factory who can do their job at the same time as other workers. Threads are commonly used for parallelism within a single node. .. _workerglossary: -**Worker:** -------------- +Worker +------ A worker in Parsl is an independent process that runs on a node to execute tasks. Unlike threads, which share resources within a single process, workers operate as separate entities, each potentially handling different tasks on the same or different nodes. .. _workflowglossary: -**Workflow:** ----------------- +Workflow +-------- A workflow is like a series of steps that you follow to complete a task. In Parsl, it's a way to describe how your apps should run and how they depend on each other, like a flowchart that shows you the order in which things need to happen. A workflow is typically expressed in a Parsl script, which is a Python program that leverages the Parsl library to orchestrate these tasks in a structured manner. - diff --git a/docs/userguide/index.rst b/docs/userguide/index.rst index 2a97a6104e..12254cd6e2 100644 --- a/docs/userguide/index.rst +++ b/docs/userguide/index.rst @@ -21,4 +21,3 @@ User guide usage_tracking plugins parsl_perf - glossary From a514a686251515260a1a497eb2af0a270867e730 Mon Sep 17 00:00:00 2001 From: Logan Ward Date: Mon, 30 Dec 2024 12:47:27 -0500 Subject: [PATCH 23/39] Overhaul documentation on structuring software (#3731) # Description Makes changes that we discussed during a community call back in May: - Emphasize keeping decorators out of the main library - Discuss benefits of modules for serializing functions - Propose defining separate "workflow-ready" functions in module - General clean up of dicsussion cc: @Andrew-S-Rosen # Changed Behaviour N/A # Fixes None ## Type of change - Update to human readable text: Documentation/error messages/comments --------- Co-authored-by: Ben Clifford --- docs/userguide/apps.rst | 4 + docs/userguide/examples/config.py | 13 --- docs/userguide/examples/library.py | 6 -- docs/userguide/examples/library/__init__.py | 0 docs/userguide/examples/library/app.py | 9 ++ docs/userguide/examples/library/config.py | 23 ++++ docs/userguide/examples/library/logic.py | 15 +++ docs/userguide/examples/pyproject.toml | 7 ++ docs/userguide/examples/run.py | 35 ++++++ docs/userguide/examples/run_increment.py | 9 -- docs/userguide/modularizing.rst | 113 ++++++++++++++++---- 11 files changed, 183 insertions(+), 51 deletions(-) delete mode 100644 docs/userguide/examples/config.py delete mode 100644 docs/userguide/examples/library.py create mode 100644 docs/userguide/examples/library/__init__.py create mode 100644 docs/userguide/examples/library/app.py create mode 100644 docs/userguide/examples/library/config.py create mode 100644 docs/userguide/examples/library/logic.py create mode 100644 docs/userguide/examples/pyproject.toml create mode 100644 docs/userguide/examples/run.py delete mode 100644 docs/userguide/examples/run_increment.py diff --git a/docs/userguide/apps.rst b/docs/userguide/apps.rst index 1ef105b4fe..41a988db6d 100644 --- a/docs/userguide/apps.rst +++ b/docs/userguide/apps.rst @@ -111,6 +111,8 @@ Practically, this means return input_list +.. _functions-from-modules: + Functions from Modules ++++++++++++++++++++++ @@ -194,6 +196,8 @@ Learn more about the types of data allowed in `the data section `_. Any changes to mutable input arguments will be ignored. +.. _special-kwargs: + Special Keyword Arguments +++++++++++++++++++++++++ diff --git a/docs/userguide/examples/config.py b/docs/userguide/examples/config.py deleted file mode 100644 index 68057d2b01..0000000000 --- a/docs/userguide/examples/config.py +++ /dev/null @@ -1,13 +0,0 @@ -from parsl.config import Config -from parsl.executors import HighThroughputExecutor -from parsl.providers import LocalProvider - -htex_config = Config( - executors=[ - HighThroughputExecutor( - label="htex_local", - cores_per_worker=1, - provider=LocalProvider(), - ) - ], -) diff --git a/docs/userguide/examples/library.py b/docs/userguide/examples/library.py deleted file mode 100644 index 2992a2dfc2..0000000000 --- a/docs/userguide/examples/library.py +++ /dev/null @@ -1,6 +0,0 @@ -from parsl import python_app - - -@python_app -def increment(x): - return x + 1 diff --git a/docs/userguide/examples/library/__init__.py b/docs/userguide/examples/library/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/docs/userguide/examples/library/app.py b/docs/userguide/examples/library/app.py new file mode 100644 index 0000000000..d16746ab75 --- /dev/null +++ b/docs/userguide/examples/library/app.py @@ -0,0 +1,9 @@ +"""Functions used as part of the workflow""" +from typing import List, Tuple + +from .logic import convert_to_binary + + +def convert_many_to_binary(xs: List[int]) -> List[Tuple[bool, ...]]: + """Convert a list of nonnegative integers to binary""" + return [convert_to_binary(x) for x in xs] diff --git a/docs/userguide/examples/library/config.py b/docs/userguide/examples/library/config.py new file mode 100644 index 0000000000..4b2301fb66 --- /dev/null +++ b/docs/userguide/examples/library/config.py @@ -0,0 +1,23 @@ +from parsl.config import Config +from parsl.executors import HighThroughputExecutor +from parsl.providers import LocalProvider + + +def make_local_config(cores_per_worker: int = 1) -> Config: + """Generate a configuration which runs all tasks on the local system + + Args: + cores_per_worker: Number of cores to dedicate for each task + Returns: + Configuration object with the requested settings + """ + return Config( + executors=[ + HighThroughputExecutor( + label="htex_local", + cores_per_worker=cores_per_worker, + cpu_affinity='block', + provider=LocalProvider(), + ) + ], + ) diff --git a/docs/userguide/examples/library/logic.py b/docs/userguide/examples/library/logic.py new file mode 100644 index 0000000000..769e6158d5 --- /dev/null +++ b/docs/userguide/examples/library/logic.py @@ -0,0 +1,15 @@ +from typing import Tuple + + +def convert_to_binary(x: int) -> Tuple[bool, ...]: + """Convert a nonnegative integer into a binary + + Args: + x: Number to be converted + Returns: + The binary number represented as list of booleans + """ + if x < 0: + raise ValueError('`x` must be nonnegative') + bin_as_string = bin(x) + return tuple(i == '1' for i in bin_as_string[2:]) diff --git a/docs/userguide/examples/pyproject.toml b/docs/userguide/examples/pyproject.toml new file mode 100644 index 0000000000..4c1639bfb4 --- /dev/null +++ b/docs/userguide/examples/pyproject.toml @@ -0,0 +1,7 @@ +[project] +name = "library" +version = '0.0.0' +description = 'Example library for Parsl documentation' + +[tool.setuptools.packages.find] +include = ['library*'] diff --git a/docs/userguide/examples/run.py b/docs/userguide/examples/run.py new file mode 100644 index 0000000000..9ca5b40219 --- /dev/null +++ b/docs/userguide/examples/run.py @@ -0,0 +1,35 @@ +from argparse import ArgumentParser + +import parsl + +from library.config import make_local_config +from library.app import convert_many_to_binary +from parsl.app.python import PythonApp + +# Protect the script from running twice. +# See "Safe importing of main module" in Python multiprocessing docs +# https://docs.python.org/3/library/multiprocessing.html#multiprocessing-programming +if __name__ == "__main__": + # Get user instructions + parser = ArgumentParser() + parser.add_argument('--numbers-per-batch', default=8, type=int) + parser.add_argument('numbers', nargs='+', type=int) + args = parser.parse_args() + + # Prepare the workflow functions + convert_app = PythonApp(convert_many_to_binary, cache=False) + + # Load the configuration + # As a context manager so resources are shutdown on exit + with parsl.load(make_local_config()): + + # Spawn tasks + futures = [ + convert_app(args.numbers[start:start + args.numbers_per_batch]) + for start in range(0, len(args.numbers), args.numbers_per_batch) + ] + + # Retrieve task results + for future in futures: + for x, b in zip(future.task_record['args'][0], future.result()): + print(f'{x} -> {"".join("1" if i else "0" for i in b)}') diff --git a/docs/userguide/examples/run_increment.py b/docs/userguide/examples/run_increment.py deleted file mode 100644 index b265640edc..0000000000 --- a/docs/userguide/examples/run_increment.py +++ /dev/null @@ -1,9 +0,0 @@ -from config import htex_config -from library import increment - -import parsl - -parsl.load(htex_config) - -for i in range(5): - print('{} + 1 = {}'.format(i, increment(i).result())) diff --git a/docs/userguide/modularizing.rst b/docs/userguide/modularizing.rst index 93b23575b9..143a4ebcd8 100644 --- a/docs/userguide/modularizing.rst +++ b/docs/userguide/modularizing.rst @@ -3,40 +3,107 @@ Structuring Parsl programs -------------------------- -Parsl programs can be developed in many ways. When developing a simple program it is -often convenient to include the app definitions and control logic in a single script. -However, as a program inevitably grows and changes, like any code, there are significant -benefits to be obtained by modularizing the program, including: +While convenient to build simple Parsl programs as a single Python file, +splitting a Parsl programs into multiple files and a Python module +has significant benefits, including: 1. Better readability 2. Logical separation of components (e.g., apps, config, and control logic) 3. Ease of reuse of components -The following example illustrates how a Parsl project can be organized into modules. +Large applications that use Parsl often divide into several core components: -The configuration(s) can be defined in a module or file (e.g., ``config.py``) -which can be imported into the control script depending on which execution resources -should be used. +.. contents:: + :local: + :depth: 2 -.. literalinclude:: examples/config.py +The following sections use an example where each component is in a separate file: -Parsl apps can be defined in separate file(s) or module(s) (e.g., ``library.py``) -grouped by functionality. +.. code-block:: + examples/logic.py + examples/app.py + examples/config.py + examples/__init__.py + run.py + pyproject.toml -.. literalinclude:: examples/library.py +Run the application by first installing the Python library and then executing the "run.py" script. -Finally, the control logic for the Parsl program can then be implemented in a -separate file (e.g., ``run_increment.py``). This file must the import the -configuration from ``config.py`` before calling the ``increment`` app from -``library.py``: +.. code-block:: bash -.. literalinclude:: examples/run_increment.py + pip install . # Install module so it can be imported by workers + python run.py -Which produces the following output:: - 0 + 1 = 1 - 1 + 1 = 2 - 2 + 1 = 3 - 3 + 1 = 4 - 4 + 1 = 5 +Core application logic +====================== + +The core application logic should be developed without any deference to Parsl. +Implement capabilities, write unit tests, and prepare documentation +in which ever way works best for the problem at hand. + +Parallelization with Parsl will be easy if the software already follows best practices. + +The example defines a function to convert a single integer into binary. + +.. literalinclude:: examples/library/logic.py + :caption: library/logic.py + +Workflow functions +================== + +Tasks within a workflow may require unique combinations of core functions. +Functions to be run in parallel must also meet :ref:`specific requirements ` +that may complicate writing the core logic effectively. +As such, separating functions to be used as Apps is often beneficial. + +The example includes a function to convert many integers into binary. + +Key points to note: + +- It is not necessary to have import statements inside the function. + Parsl will serialize this function by reference, as described in :ref:`functions-from-modules`. + +- The function is not yet marked as a Parsl PythonApp. + Keeping Parsl out of the function definitions simplifies testing + because you will not need to run Parsl when testing the code. + +- *Advanced*: Consider including Parsl decorators in the library if using complex workflow patterns, + such as :ref:`join apps ` or functions which take :ref:`special arguments `. + +.. literalinclude:: examples/library/app.py + :caption: library/app.py + + +Parsl configuration functions +============================= + +Create Parsl configurations specific to your application needs as functions. +While not necessary, including the Parsl configuration functions inside the module +ensures they can be imported into other scripts easily. + +Generating Parsl :class:`~parsl.config.Config` objects from a function +makes it possible to change the configuration without editing the module. + +The example function provides a configuration suited for a single node. + +.. literalinclude:: examples/library/config.py + :caption: library/config.py + +Orchestration Scripts +===================== + +The last file defines the workflow itself. + +Such orchestration scripts, at minimum, perform at least four tasks: + +1. *Load execution options* using a tool like :mod:`argparse`. +2. *Prepare workflow functions for execution* by creating :class:`~parsl.app.python.PythonApp` wrappers over each function. +3. *Create configuration then start Parsl* with the :meth:`parsl.load` function. +4. *Launch tasks and retrieve results* depending on the needs of the application. + +An example run script is as follows + +.. literalinclude:: examples/run.py + :caption: run.py From b09ad8a4b81b062949245378b7380b68473867ec Mon Sep 17 00:00:00 2001 From: Logan Ward Date: Sun, 5 Jan 2025 12:41:20 -0500 Subject: [PATCH 24/39] Group user guide pages by subject (#3732) # Description Split user guide documentation based on which part of Parsl they deal with: App definition, Config, Workflow. Also includes an "advanced" section for things that I couldn't figure out how to group and weren't specific enough to get their own header. 1. Divided into four sections, placing simple introductions where necessary 1. Did not reduce any contents. Only reshaping 1. Split long documents into multiple sections so they show up better on the table of contents # Changed Behaviour N/A # Fixes N/A ## Type of change - Update to human readable text: Documentation/error messages/comments --- docs/devguide/roadmap.rst | 2 +- docs/index.rst | 8 +- docs/quickstart.rst | 14 +- .../examples/library/__init__.py | 0 .../{ => advanced}/examples/library/app.py | 0 .../{ => advanced}/examples/library/config.py | 0 .../{ => advanced}/examples/library/logic.py | 0 .../{ => advanced}/examples/pyproject.toml | 0 docs/userguide/{ => advanced}/examples/run.py | 0 docs/userguide/advanced/index.rst | 13 + docs/userguide/advanced/modularizing.rst | 109 +++ docs/userguide/advanced/monitoring.rst | 121 ++++ docs/userguide/advanced/parsl_perf.rst | 53 ++ docs/userguide/advanced/plugins.rst | 106 +++ docs/userguide/advanced/usage_tracking.rst | 171 +++++ docs/userguide/app.rst | 9 + docs/userguide/apps/bash.rst | 66 ++ docs/userguide/apps/index.rst | 26 + docs/userguide/apps/joins.rst | 257 +++++++ docs/userguide/apps/mpi_apps.rst | 153 ++++ docs/userguide/{apps.rst => apps/python.rst} | 96 +-- docs/userguide/checkpoints.rst | 302 +------- docs/userguide/configuration/data.rst | 415 +++++++++++ docs/userguide/configuration/elasticity.rst | 115 +++ docs/userguide/configuration/encryption.rst | 45 ++ docs/userguide/configuration/examples.rst | 333 +++++++++ docs/userguide/configuration/execution.rst | 227 ++++++ .../userguide/configuration/heterogeneous.rst | 106 +++ .../{ => configuration/img}/aws_image.png | Bin .../img}/parsl_parallelism.gif | Bin .../{ => configuration/img}/parsl_scaling.gif | Bin docs/userguide/configuration/index.rst | 88 +++ docs/userguide/configuration/pinning.rst | 125 ++++ docs/userguide/configuring.rst | 659 +----------------- docs/userguide/data.rst | 448 +----------- docs/userguide/exceptions.rst | 174 +---- docs/userguide/execution.rst | 392 +---------- docs/userguide/futures.rst | 168 +---- docs/userguide/index.rst | 28 +- docs/userguide/joins.rst | 260 +------ docs/userguide/lifted_ops.rst | 59 +- docs/userguide/modularizing.rst | 112 +-- docs/userguide/monitoring.rst | 124 +--- docs/userguide/mpi_apps.rst | 156 +---- docs/userguide/overview.rst | 148 ++-- docs/userguide/parsl_perf.rst | 56 +- docs/userguide/plugins.rst | 109 +-- docs/userguide/usage_tracking.rst | 172 +---- docs/userguide/workflow.rst | 246 +------ docs/userguide/workflows/checkpoints.rst | 299 ++++++++ docs/userguide/workflows/exceptions.rst | 171 +++++ docs/userguide/workflows/futures.rst | 165 +++++ docs/userguide/workflows/index.rst | 18 + docs/userguide/workflows/lifted_ops.rst | 56 ++ docs/userguide/workflows/workflow.rst | 243 +++++++ 55 files changed, 3680 insertions(+), 3543 deletions(-) rename docs/userguide/{ => advanced}/examples/library/__init__.py (100%) rename docs/userguide/{ => advanced}/examples/library/app.py (100%) rename docs/userguide/{ => advanced}/examples/library/config.py (100%) rename docs/userguide/{ => advanced}/examples/library/logic.py (100%) rename docs/userguide/{ => advanced}/examples/pyproject.toml (100%) rename docs/userguide/{ => advanced}/examples/run.py (100%) create mode 100644 docs/userguide/advanced/index.rst create mode 100644 docs/userguide/advanced/modularizing.rst create mode 100644 docs/userguide/advanced/monitoring.rst create mode 100644 docs/userguide/advanced/parsl_perf.rst create mode 100644 docs/userguide/advanced/plugins.rst create mode 100644 docs/userguide/advanced/usage_tracking.rst create mode 100644 docs/userguide/app.rst create mode 100644 docs/userguide/apps/bash.rst create mode 100644 docs/userguide/apps/index.rst create mode 100644 docs/userguide/apps/joins.rst create mode 100644 docs/userguide/apps/mpi_apps.rst rename docs/userguide/{apps.rst => apps/python.rst} (71%) create mode 100644 docs/userguide/configuration/data.rst create mode 100644 docs/userguide/configuration/elasticity.rst create mode 100644 docs/userguide/configuration/encryption.rst create mode 100644 docs/userguide/configuration/examples.rst create mode 100644 docs/userguide/configuration/execution.rst create mode 100644 docs/userguide/configuration/heterogeneous.rst rename docs/userguide/{ => configuration/img}/aws_image.png (100%) rename docs/userguide/{ => configuration/img}/parsl_parallelism.gif (100%) rename docs/userguide/{ => configuration/img}/parsl_scaling.gif (100%) create mode 100644 docs/userguide/configuration/index.rst create mode 100644 docs/userguide/configuration/pinning.rst create mode 100644 docs/userguide/workflows/checkpoints.rst create mode 100644 docs/userguide/workflows/exceptions.rst create mode 100644 docs/userguide/workflows/futures.rst create mode 100644 docs/userguide/workflows/index.rst create mode 100644 docs/userguide/workflows/lifted_ops.rst create mode 100644 docs/userguide/workflows/workflow.rst diff --git a/docs/devguide/roadmap.rst b/docs/devguide/roadmap.rst index a1fe8e44e0..ccb5abce30 100644 --- a/docs/devguide/roadmap.rst +++ b/docs/devguide/roadmap.rst @@ -15,7 +15,7 @@ Code Maintenance * **Type Annotations and Static Type Checking**: Add static type annotations throughout the codebase and add typeguard checks. * **Release Process**: `Improve the overall release process `_ to synchronize docs and code releases, automatically produce changelog documentation. * **Components Maturity Model**: Defines the `component maturity model `_ and tags components with their appropriate maturity level. -* **Define and Document Interfaces**: Identify and document interfaces via which `external components `_ can augment the Parsl ecosystem. +* **Define and Document Interfaces**: Identify and document interfaces via which `external components `_ can augment the Parsl ecosystem. * **Distributed Testing Process**: All tests should be run against all possible schedulers, using different executors, on a variety of remote systems. Explore the use of containerized schedulers and remote testing on real systems. New Features and Integrations diff --git a/docs/index.rst b/docs/index.rst index 88b0c7bb4c..a9c5c99881 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -4,7 +4,7 @@ Parsl - Parallel Scripting Library Parsl extends parallelism in Python beyond a single computer. You can use Parsl -`just like Python's parallel executors `_ +`just like Python's parallel executors `_ but across *multiple cores and nodes*. However, the real power of Parsl is in expressing multi-step workflows of functions. Parsl lets you chain functions together and will launch each function as inputs and computing resources are available. @@ -37,8 +37,8 @@ Parsl lets you chain functions together and will launch each function as inputs Start with the `configuration quickstart `_ to learn how to tell Parsl how to use your computing resource, -see if `a template configuration for your supercomputer `_ is already available, -then explore the `parallel computing patterns `_ to determine how to use parallelism best in your application. +see if `a template configuration for your supercomputer `_ is already available, +then explore the `parallel computing patterns `_ to determine how to use parallelism best in your application. Parsl is an open-source code, and available on GitHub: https://github.com/parsl/parsl/ @@ -57,7 +57,7 @@ Parsl works everywhere *Parsl can run parallel functions on a laptop and the world's fastest supercomputers.* Scaling from laptop to supercomputer is often as simple as changing the resource configuration. -Parsl is tested `on many of the top supercomputers `_. +Parsl is tested `on many of the top supercomputers `_. Parsl is flexible ----------------- diff --git a/docs/quickstart.rst b/docs/quickstart.rst index d54763ee58..9c0d119fad 100644 --- a/docs/quickstart.rst +++ b/docs/quickstart.rst @@ -70,7 +70,7 @@ We describe these components briefly here, and link to more details in the `User .. note:: - Parsl's documentation includes `templates for many supercomputers `_. + Parsl's documentation includes `templates for many supercomputers `_. Even though you may not need to write a configuration from a blank slate, understanding the basic terminology below will be very useful. @@ -112,7 +112,7 @@ with hello world Python and Bash apps. with open('hello-stdout', 'r') as f: print(f.read()) -Learn more about the types of Apps and their options `here `__. +Learn more about the types of Apps and their options `here `__. Executors ^^^^^^^^^ @@ -127,7 +127,7 @@ You can dynamically set the number of workers based on available memory and pin each worker to specific GPUs or CPU cores among other powerful features. -Learn more about Executors `here `__. +Learn more about Executors `here `__. Execution Providers ^^^^^^^^^^^^^^^^^^^ @@ -141,7 +141,7 @@ Another key role of Providers is defining how to start an Executor on a remote c Often, this simply involves specifying the correct Python environment and (described below) how to launch the Executor on each acquired computers. -Learn more about Providers `here `__. +Learn more about Providers `here `__. Launchers ^^^^^^^^^ @@ -151,7 +151,7 @@ A common example is an :class:`~parsl.launchers.launchers.MPILauncher`, which us for starting a single program on multiple computing nodes. Like Providers, Parsl comes packaged with Launchers for most supercomputers and clouds. -Learn more about Launchers `here `__. +Learn more about Launchers `here `__. Benefits of a Data-Flow Kernel @@ -164,7 +164,7 @@ and performs the many other functions needed to execute complex workflows. The flexibility and performance of the DFK enables applications with intricate dependencies between tasks to execute on thousands of parallel workers. -Start with the Tutorial or the `parallel patterns `_ +Start with the Tutorial or the `parallel patterns `_ to see the complex types of workflows you can make with Parsl. Starting Parsl @@ -210,7 +210,7 @@ An example which launches 4 workers on 1 node of the Polaris supercomputer looks ) -The documentation has examples for other supercomputers `here `__. +The documentation has examples for other supercomputers `here `_. The next step is to load the configuration diff --git a/docs/userguide/examples/library/__init__.py b/docs/userguide/advanced/examples/library/__init__.py similarity index 100% rename from docs/userguide/examples/library/__init__.py rename to docs/userguide/advanced/examples/library/__init__.py diff --git a/docs/userguide/examples/library/app.py b/docs/userguide/advanced/examples/library/app.py similarity index 100% rename from docs/userguide/examples/library/app.py rename to docs/userguide/advanced/examples/library/app.py diff --git a/docs/userguide/examples/library/config.py b/docs/userguide/advanced/examples/library/config.py similarity index 100% rename from docs/userguide/examples/library/config.py rename to docs/userguide/advanced/examples/library/config.py diff --git a/docs/userguide/examples/library/logic.py b/docs/userguide/advanced/examples/library/logic.py similarity index 100% rename from docs/userguide/examples/library/logic.py rename to docs/userguide/advanced/examples/library/logic.py diff --git a/docs/userguide/examples/pyproject.toml b/docs/userguide/advanced/examples/pyproject.toml similarity index 100% rename from docs/userguide/examples/pyproject.toml rename to docs/userguide/advanced/examples/pyproject.toml diff --git a/docs/userguide/examples/run.py b/docs/userguide/advanced/examples/run.py similarity index 100% rename from docs/userguide/examples/run.py rename to docs/userguide/advanced/examples/run.py diff --git a/docs/userguide/advanced/index.rst b/docs/userguide/advanced/index.rst new file mode 100644 index 0000000000..39a89f7ecd --- /dev/null +++ b/docs/userguide/advanced/index.rst @@ -0,0 +1,13 @@ +Advanced Topics +=============== + +More to learn about Parsl after starting a project. + +.. toctree:: + :maxdepth: 2 + + modularizing + usage_tracking + monitoring + parsl_perf + plugins diff --git a/docs/userguide/advanced/modularizing.rst b/docs/userguide/advanced/modularizing.rst new file mode 100644 index 0000000000..143a4ebcd8 --- /dev/null +++ b/docs/userguide/advanced/modularizing.rst @@ -0,0 +1,109 @@ +.. _codebases: + +Structuring Parsl programs +-------------------------- + +While convenient to build simple Parsl programs as a single Python file, +splitting a Parsl programs into multiple files and a Python module +has significant benefits, including: + + 1. Better readability + 2. Logical separation of components (e.g., apps, config, and control logic) + 3. Ease of reuse of components + +Large applications that use Parsl often divide into several core components: + +.. contents:: + :local: + :depth: 2 + +The following sections use an example where each component is in a separate file: + +.. code-block:: + + examples/logic.py + examples/app.py + examples/config.py + examples/__init__.py + run.py + pyproject.toml + +Run the application by first installing the Python library and then executing the "run.py" script. + +.. code-block:: bash + + pip install . # Install module so it can be imported by workers + python run.py + + +Core application logic +====================== + +The core application logic should be developed without any deference to Parsl. +Implement capabilities, write unit tests, and prepare documentation +in which ever way works best for the problem at hand. + +Parallelization with Parsl will be easy if the software already follows best practices. + +The example defines a function to convert a single integer into binary. + +.. literalinclude:: examples/library/logic.py + :caption: library/logic.py + +Workflow functions +================== + +Tasks within a workflow may require unique combinations of core functions. +Functions to be run in parallel must also meet :ref:`specific requirements ` +that may complicate writing the core logic effectively. +As such, separating functions to be used as Apps is often beneficial. + +The example includes a function to convert many integers into binary. + +Key points to note: + +- It is not necessary to have import statements inside the function. + Parsl will serialize this function by reference, as described in :ref:`functions-from-modules`. + +- The function is not yet marked as a Parsl PythonApp. + Keeping Parsl out of the function definitions simplifies testing + because you will not need to run Parsl when testing the code. + +- *Advanced*: Consider including Parsl decorators in the library if using complex workflow patterns, + such as :ref:`join apps ` or functions which take :ref:`special arguments `. + +.. literalinclude:: examples/library/app.py + :caption: library/app.py + + +Parsl configuration functions +============================= + +Create Parsl configurations specific to your application needs as functions. +While not necessary, including the Parsl configuration functions inside the module +ensures they can be imported into other scripts easily. + +Generating Parsl :class:`~parsl.config.Config` objects from a function +makes it possible to change the configuration without editing the module. + +The example function provides a configuration suited for a single node. + +.. literalinclude:: examples/library/config.py + :caption: library/config.py + +Orchestration Scripts +===================== + +The last file defines the workflow itself. + +Such orchestration scripts, at minimum, perform at least four tasks: + +1. *Load execution options* using a tool like :mod:`argparse`. +2. *Prepare workflow functions for execution* by creating :class:`~parsl.app.python.PythonApp` wrappers over each function. +3. *Create configuration then start Parsl* with the :meth:`parsl.load` function. +4. *Launch tasks and retrieve results* depending on the needs of the application. + +An example run script is as follows + +.. literalinclude:: examples/run.py + :caption: run.py diff --git a/docs/userguide/advanced/monitoring.rst b/docs/userguide/advanced/monitoring.rst new file mode 100644 index 0000000000..c1285cb9b3 --- /dev/null +++ b/docs/userguide/advanced/monitoring.rst @@ -0,0 +1,121 @@ +Monitoring +========== + +Parsl includes a monitoring system to capture task state as well as resource +usage over time. The Parsl monitoring system aims to provide detailed +information and diagnostic capabilities to help track the state of your +programs, down to the individual apps that are executed on remote machines. + +The monitoring system records information to an SQLite database while a +workflow runs. This information can then be visualised in a web dashboard +using the ``parsl-visualize`` tool, or queried using SQL using regular +SQLite tools. + + +Monitoring configuration +------------------------ + +Parsl monitoring is only supported with the `parsl.executors.HighThroughputExecutor`. + +The following example shows how to enable monitoring in the Parsl +configuration. Here the `parsl.monitoring.MonitoringHub` is specified to use port +55055 to receive monitoring messages from workers every 10 seconds. + +.. code-block:: python + + import parsl + from parsl.monitoring.monitoring import MonitoringHub + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + from parsl.addresses import address_by_hostname + + import logging + + config = Config( + executors=[ + HighThroughputExecutor( + label="local_htex", + cores_per_worker=1, + max_workers_per_node=4, + address=address_by_hostname(), + ) + ], + monitoring=MonitoringHub( + hub_address=address_by_hostname(), + monitoring_debug=False, + resource_monitoring_interval=10, + ), + strategy='none' + ) + + +Visualization +------------- + +To run the web dashboard utility ``parsl-visualize`` you first need to install +its dependencies: + + $ pip install 'parsl[monitoring,visualization]' + +To view the web dashboard while or after a Parsl program has executed, run +the ``parsl-visualize`` utility:: + + $ parsl-visualize + +By default, this command expects that the default ``monitoring.db`` database is used +in the runinfo directory. Other databases can be loaded by passing +the database URI on the command line. For example, if the full path +to the database is ``/tmp/my_monitoring.db``, run:: + + $ parsl-visualize sqlite:////tmp/my_monitoring.db + +By default, the visualization web server listens on ``127.0.0.1:8080``. If the web server is deployed on a machine with a web browser, the dashboard can be accessed in the browser at ``127.0.0.1:8080``. If the web server is deployed on a remote machine, such as the login node of a cluster, you will need to use an ssh tunnel from your local machine to the cluster:: + + $ ssh -L 50000:127.0.0.1:8080 username@cluster_address + +This command will bind your local machine's port 50000 to the remote cluster's port 8080. +The dashboard can then be accessed via the local machine's browser at ``127.0.0.1:50000``. + +.. warning:: Alternatively you can deploy the visualization server on a public interface. However, first check that this is allowed by the cluster's security policy. The following example shows how to deploy the web server on a public port (i.e., open to Internet via ``public_IP:55555``):: + + $ parsl-visualize --listen 0.0.0.0 --port 55555 + + +Workflows Page +^^^^^^^^^^^^^^ + +The workflows page lists all Parsl workflows that have been executed with monitoring enabled +with the selected database. +It provides a high level summary of workflow state as shown below: + +.. image:: ../../images/mon_workflows_page.png + +Throughout the dashboard, all blue elements are clickable. For example, clicking a specific worklow +name from the table takes you to the Workflow Summary page described in the next section. + +Workflow Summary +^^^^^^^^^^^^^^^^ + +The workflow summary page captures the run level details of a workflow, including start and end times +as well as task summary statistics. The workflow summary section is followed by the *App Summary* that lists +the various apps and invocation count for each. + +.. image:: ../../images/mon_workflow_summary.png + + +The workflow summary also presents three different views of the workflow: + +* Workflow DAG - with apps differentiated by colors: This visualization is useful to visually inspect the dependency + structure of the workflow. Hovering over the nodes in the DAG shows a tooltip for the app represented by the node and it's task ID. + +.. image:: ../../images/mon_task_app_grouping.png + +* Workflow DAG - with task states differentiated by colors: This visualization is useful to identify what tasks have been completed, failed, or are currently pending. + +.. image:: ../../images/mon_task_state_grouping.png + +* Workflow resource usage: This visualization provides resource usage information at the workflow level. + For example, cumulative CPU/Memory utilization across workers over time. + +.. image:: ../../images/mon_resource_summary.png + diff --git a/docs/userguide/advanced/parsl_perf.rst b/docs/userguide/advanced/parsl_perf.rst new file mode 100644 index 0000000000..2ea1adb00f --- /dev/null +++ b/docs/userguide/advanced/parsl_perf.rst @@ -0,0 +1,53 @@ +.. _label-parsl-perf: + +Measuring performance with parsl-perf +===================================== + +``parsl-perf`` is tool for making basic performance measurements of Parsl +configurations. + +It runs increasingly large numbers of no-op apps until a batch takes +(by default) 120 seconds, giving a measurement of tasks per second. + +This can give a basic measurement of some of the overheads in task +execution. + +``parsl-perf`` must be invoked with a configuration file, which is a Python +file containing a variable ``config`` which contains a `Config` object, or +a function ``fresh_config`` which returns a `Config` object. The +``fresh_config`` format is the same as used with the pytest test suite. + +To specify a ``parsl_resource_specification`` for tasks, add a ``--resources`` +argument. + +To change the target runtime from the default of 120 seconds, add a +``--time`` parameter. + +For example: + +.. code-block:: bash + + + $ python -m parsl.benchmark.perf --config parsl/tests/configs/workqueue_ex.py --resources '{"cores":1, "memory":0, "disk":0}' + ==== Iteration 1 ==== + Will run 10 tasks to target 120 seconds runtime + Submitting tasks / invoking apps + warning: using plain-text when communicating with workers. + warning: use encryption with a key and cert when creating the manager. + All 10 tasks submitted ... waiting for completion + Submission took 0.008 seconds = 1248.676 tasks/second + Runtime: actual 3.668s vs target 120s + Tasks per second: 2.726 + + [...] + + ==== Iteration 4 ==== + Will run 57640 tasks to target 120 seconds runtime + Submitting tasks / invoking apps + All 57640 tasks submitted ... waiting for completion + Submission took 34.839 seconds = 1654.487 tasks/second + Runtime: actual 364.387s vs target 120s + Tasks per second: 158.184 + Cleaning up DFK + The end + diff --git a/docs/userguide/advanced/plugins.rst b/docs/userguide/advanced/plugins.rst new file mode 100644 index 0000000000..cd9244960c --- /dev/null +++ b/docs/userguide/advanced/plugins.rst @@ -0,0 +1,106 @@ +Plugins +======= + +Parsl has several places where code can be plugged in. Parsl usually provides +several implementations that use each plugin point. + +This page gives a brief summary of those places and why you might want +to use them, with links to the API guide. + +Executors +--------- +When the parsl dataflow kernel is ready for a task to run, it passes that +task to an `ParslExecutor`. The executor is then responsible for running the task's +Python code and returning the result. This is the abstraction that allows one +executor to run code on the local submitting host, while another executor can +run the same code on a large supercomputer. + + +Providers and Launchers +----------------------- +Some executors are based on blocks of workers (for example the +`parsl.executors.HighThroughputExecutor`: the submit side requires a +batch system (eg slurm, kubernetes) to start worker processes, which then +execute tasks. + +The particular way in which a system makes those workers start is implemented +by providers and launchers. + +An `ExecutionProvider` allows a command line to be submitted as a request to the +underlying batch system to be run inside an allocation of nodes. + +A `Launcher` modifies that command line when run inside the allocation to +add on any wrappers that are needed to launch the command (eg srun inside +slurm). Providers and launchers are usually paired together for a particular +system type. + +File staging +------------ +Parsl can copy input files from an arbitrary URL into a task's working +environment, and copy output files from a task's working environment to +an arbitrary URL. A small set of data staging providers is installed by default, +for ``file://`` ``http://`` and ``ftp://`` URLs. More data staging providers can +be added in the workflow configuration, in the ``storage`` parameter of the +relevant `ParslExecutor`. Each provider should subclass the `Staging` class. + + +Default stdout/stderr name generation +------------------------------------- +Parsl can choose names for your bash apps stdout and stderr streams +automatically, with the parsl.AUTO_LOGNAME parameter. The choice of path is +made by a function which can be configured with the ``std_autopath`` +parameter of Parsl `Config`. By default, ``DataFlowKernel.default_std_autopath`` +will be used. + + +Memoization/checkpointing +------------------------- + +When parsl memoizes/checkpoints an app parameter, it does so by computing a +hash of that parameter that should be the same if that parameter is the same +on subsequent invocations. This isn't straightforward to do for arbitrary +objects, so parsl implements a checkpointing hash function for a few common +types, and raises an exception on unknown types: + +.. code-block:: + + ValueError("unknown type for memoization ...") + +You can plug in your own type-specific hash code for additional types that +you need and understand using `id_for_memo`. + + +Invoking other asynchronous components +-------------------------------------- + +Parsl code can invoke other asynchronous components which return Futures, and +integrate those Futures into the task graph: Parsl apps can be given any +`concurrent.futures.Future` as a dependency, even if those futures do not come +from invoking a Parsl app. This includes as the return value of a +``join_app``. + +An specific example of this is integrating Globus Compute tasks into a Parsl +task graph. See :ref:`label-join-globus-compute` + +Dependency resolution +--------------------- + +When Parsl examines the arguments to an app, it uses a `DependencyResolver`. +The default `DependencyResolver` will cause Parsl to wait for +``concurrent.futures.Future`` instances (including `AppFuture` and +`DataFuture`), and pass through other arguments without waiting. + +This behaviour is pluggable: Parsl comes with another dependency resolver, +`DEEP_DEPENDENCY_RESOLVER` which knows about futures contained with structures +such as tuples, lists, sets and dicts. + +This plugin interface might be used to interface other task-like or future-like +objects to the Parsl dependency mechanism, by describing how they can be +interpreted as a Future. + +Removed interfaces +------------------ + +Parsl had a deprecated ``Channel`` abstraction. See +`issue 3515 `_ +for further discussion on its removal. diff --git a/docs/userguide/advanced/usage_tracking.rst b/docs/userguide/advanced/usage_tracking.rst new file mode 100644 index 0000000000..da8ac9b79d --- /dev/null +++ b/docs/userguide/advanced/usage_tracking.rst @@ -0,0 +1,171 @@ +.. _label-usage-tracking: + +Usage Statistics Collection +=========================== + +Parsl uses an **Opt-in** model for usage tracking, allowing users to decide if they wish to participate. Usage statistics are crucial for improving software reliability and help focus development and maintenance efforts on the most used components of Parsl. The collected data is used solely for enhancements and reporting and is not shared in its raw form outside of the Parsl team. + +Why are we doing this? +---------------------- + +The Parsl development team relies on funding from government agencies. To sustain this funding and advocate for continued support, it is essential to show that the research community benefits from these investments. + +By opting in to share usage data, you actively support the ongoing development and maintenance of Parsl. (See:ref:`What is sent? ` below). + +Opt-In Model +------------ + +We use an **opt-in model** for usage tracking to respect user privacy and provide full control over shared information. We hope that developers and researchers will choose to send us this information. The reason is that we need this data - it is a requirement for funding. + +Choose the data you share with Usage Tracking Levels. + +**Usage Tracking Levels:** + +* **Level 1:** Only basic information such as Python version, Parsl version, and platform name (Linux, MacOS, etc.) +* **Level 2:** Level 1 information and configuration information including provider, executor, and launcher names. +* **Level 3:** Level 2 information and workflow execution details, including the number of applications run, failures, and execution time. + +By enabling usage tracking, you support Parsl's development. + +**To opt-in, set** ``usage_tracking`` **to the desired level (1, 2, or 3) in the configuration object** (``parsl.config.Config``) **.** + +Example: + +.. code-block:: python3 + + config = Config( + executors=[ + HighThroughputExecutor( + ... + ) + ], + usage_tracking=3 + ) + +.. _what-is-sent: + +What is sent? +------------- + +The data collected depends on the tracking level selected: + +* **Level 1:** Only basic information such as Python version, Parsl version, and platform name (Linux, MacOS, etc.) +* **Level 2:** Level 1 information and configuration information including provider, executor, and launcher names. +* **Level 3:** Level 2 information and workflow execution details, including the number of applications run, failures, and execution time. + +**Example Messages:** + +- At launch: + + .. code-block:: json + + { + "correlator":"6bc7484e-5693-48b2-b6c0-5889a73f7f4e", + "parsl_v":"1.3.0-dev", + "python_v":"3.12.2", + "platform.system":"Darwin", + "tracking_level":3, + "components":[ + { + "c":"parsl.config.Config", + "executors_len":1, + "dependency_resolver":false + }, + "parsl.executors.threads.ThreadPoolExecutor" + ], + "start":1727156153 + } + +- On closure (Tracking Level 3 only): + + .. code-block:: json + + { + "correlator":"6bc7484e-5693-48b2-b6c0-5889a73f7f4e", + "execution_time":31, + "components":[ + { + "c":"parsl.dataflow.dflow.DataFlowKernel", + "app_count":3, + "app_fails":0 + }, + { + "c":"parsl.config.Config", + "executors_len":1, + "dependency_resolver":false + }, + "parsl.executors.threads.ThreadPoolExecutor" + ], + "end":1727156156 + } + +**All messages sent are logged in the** ``parsl.log`` **file, ensuring complete transparency.** + +How is the data sent? +--------------------- + +Data is sent using **UDP** to minimize the impact on workflow performance. While this may result in some data loss, it significantly reduces the chances of usage tracking affecting the software's operation. + +The data is processed through AWS CloudWatch to generate a monitoring dashboard, providing valuable insights into usage patterns. + +When is the data sent? +---------------------- + +Data is sent twice per run: + +1. At the start of the script. +2. Upon script completion (for Tracking Level 3). + +What will the data be used for? +------------------------------- + +The data will help the Parsl team understand Parsl usage and make development and maintenance decisions, including: + +* Focus development and maintenance on the most-used components of Parsl. +* Determine which Python versions to continue supporting. +* Track the age of Parsl installations. +* Assess how long it takes for most users to adopt new changes. +* Track usage statistics to report to funders. + +Usage Statistics Dashboard +-------------------------- + +The collected data is aggregated and displayed on a publicly accessible dashboard. This dashboard provides an overview of how Parsl is being used across different environments and includes metrics such as: + +* Total workflows executed over time +* Most-used Python and Parsl versions +* Most common platforms and executors and more + +`Find the dashboard here `_ + +Leaderboard +----------- + +**Opting in to usage tracking also allows you to participate in the Parsl Leaderboard. +To participate in the leaderboard, you can deanonymize yourself using the** ``project_name`` **parameter in the parsl configuration object** (``parsl.config.Config``) **.** + +`Find the Parsl Leaderboard here `_ + +Example: + +.. code-block:: python3 + + config = Config( + executors=[ + HighThroughputExecutor( + ... + ) + ], + usage_tracking=3, + project_name="my-test-project" + ) + +Every run of parsl with usage tracking **Level 1** or **Level 2** earns you **1 point**. And every run with usage tracking **Level 3**, earns you **2 points**. + +Feedback +-------- + +Please send us your feedback at parsl@googlegroups.com. Feedback from our user communities will be +useful in determining our path forward with usage tracking in the future. + +**Please consider turning on usage tracking to support the continued development of Parsl.** diff --git a/docs/userguide/app.rst b/docs/userguide/app.rst new file mode 100644 index 0000000000..5e58276c3d --- /dev/null +++ b/docs/userguide/app.rst @@ -0,0 +1,9 @@ +:orphan: + +.. meta:: + :content http-equiv="refresh": 0;url=apps/index.html + +Redirect +-------- + +This page has been `moved `_ diff --git a/docs/userguide/apps/bash.rst b/docs/userguide/apps/bash.rst new file mode 100644 index 0000000000..9f99eb4d95 --- /dev/null +++ b/docs/userguide/apps/bash.rst @@ -0,0 +1,66 @@ + +Bash Apps +--------- + +.. code-block:: python + + @bash_app + def echo( + name: str, + stdout=parsl.AUTO_LOGNAME # Requests Parsl to return the stdout + ): + return f'echo "Hello, {name}!"' + + future = echo('user') + future.result() # block until task has completed + + with open(future.stdout, 'r') as f: + print(f.read()) + + +A Parsl Bash app executes an external application by making a command-line execution. +Parsl will execute the string returned by the function as a command-line script on a remote worker. + +Rules for Function Contents +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Bash Apps follow the same rules :ref:`as Python Apps `. +For example, imports may need to be inside functions and global variables will be inaccessible. + +Inputs and Outputs +^^^^^^^^^^^^^^^^^^ + +Bash Apps can use the same kinds of inputs as Python Apps, but only communicate results with Files. + +Bash Apps, unlike Python Apps, can also return the content printed to the Standard Output and Error. + +Special Keywords Arguments +++++++++++++++++++++++++++ + +In addition to the ``inputs``, ``outputs``, and ``walltime`` keyword arguments +described above, a Bash app can accept the following keywords: + +1. stdout: (string, tuple or ``parsl.AUTO_LOGNAME``) The path to a file to which standard output should be redirected. If set to ``parsl.AUTO_LOGNAME``, the log will be automatically named according to task id and saved under ``task_logs`` in the run directory. If set to a tuple ``(filename, mode)``, standard output will be redirected to the named file, opened with the specified mode as used by the Python `open `_ function. +2. stderr: (string or ``parsl.AUTO_LOGNAME``) Like stdout, but for the standard error stream. +3. label: (string) If the app is invoked with ``stdout=parsl.AUTO_LOGNAME`` or ``stderr=parsl.AUTO_LOGNAME``, this argument will be appended to the log name. + +Outputs ++++++++ + +If the Bash app exits with Unix exit code 0, then the AppFuture will complete. If the Bash app +exits with any other code, Parsl will treat this as a failure, and the AppFuture will instead +contain an `BashExitFailure` exception. The Unix exit code can be accessed through the +``exitcode`` attribute of that `BashExitFailure`. + + +Execution Options +^^^^^^^^^^^^^^^^^ + +Bash Apps have the same execution options (e.g., pinning to specific sites) as the Python Apps. + +MPI Apps +^^^^^^^^ + +Applications which employ MPI to span multiple nodes are a special case of Bash apps, +and require special modification of Parsl's `execution environment <../configuration/execution.html>`_ to function. +Support for MPI applications is described `in a later section `_. diff --git a/docs/userguide/apps/index.rst b/docs/userguide/apps/index.rst new file mode 100644 index 0000000000..be46d4be29 --- /dev/null +++ b/docs/userguide/apps/index.rst @@ -0,0 +1,26 @@ +.. _apps: + +Writing Parsl Apps +================== + +An **App** defines a computation that will be executed asynchronously by Parsl. +Apps are Python functions marked with a decorator which +designates that the function will run asynchronously and cause it to return +a :class:`~concurrent.futures.Future` instead of the result. + +Apps can be one of three types of functions, each with their own type of decorator + +- ``@python_app``: Most Python functions +- ``@bash_app``: A Python function which returns a command line program to execute +- ``@join_app``: A function which launches one or more new Apps + +Start by learning how to write Python Apps, which define most of the rules needed to write +other types of Apps. + +.. toctree:: + :maxdepth: 1 + + python + bash + mpi_apps + joins diff --git a/docs/userguide/apps/joins.rst b/docs/userguide/apps/joins.rst new file mode 100644 index 0000000000..defb0ad012 --- /dev/null +++ b/docs/userguide/apps/joins.rst @@ -0,0 +1,257 @@ +.. _label-joinapp: + +Join Apps +========= + +Join apps, defined with the ``@join_app`` decorator, are a form of app that can +launch other pieces of a workflow: for example a Parsl sub-workflow, or a task +that runs in some other system. + +Parsl sub-workflows +------------------- + +One reason for launching Parsl apps from inside a join app, rather than +directly in the main workflow code, is because the definitions of those tasks +are not known well enough at the start of the workflow. + +For example, a workflow might run an expensive step to detect some objects +in an image, and then on each object, run a further expensive step. Because +the number of objects is not known at the start of the workflow, but instead +only after an expensive step has completed, the subsequent tasks cannot be +defined until after that step has completed. + +In simple cases, the main workflow script can be stopped using +``Future.result()`` and join apps are not necessary, but in more complicated +cases, that approach can severely limit concurrency. + +Join apps allow more naunced dependencies to be expressed that can help with: + +* increased concurrency - helping with strong scaling +* more focused error propagation - allowing more of an ultimately failing workflow to complete +* more useful monitoring information + +Using Futures from other components +----------------------------------- + +Sometimes, a workflow might need to incorporate tasks from other systems that +run asynchronously but do not need a Parsl worker allocated for their entire +run. An example of this is delegating some work into Globus Compute: work can +be given to Globus Compute, but Parsl does not need to keep a worker allocated +to that task while it runs. Instead, Parsl can be told to wait for the ``Future`` +returned by Globus Compute to complete. + +Usage +----- + +A `join_app` looks quite like a `python_app`, but should return one or more +``Future`` objects, rather than a value. Once the Python code has run, the +app will wait for those Futures to complete without occuping a Parsl worker, +and when those Futures complete, their contents will be the return value +of the `join_app`. + +For example: + +.. code-block:: python + + @python_app + def some_app(): + return 3 + + @join_app + def example(): + x: Future = some_app() + return x # note that x is a Future, not a value + + assert example.result() == 3 + +Example of a Parsl sub-workflow +------------------------------- + +This example workflow shows a preprocessing step, followed by +a middle stage that is chosen by the result of the pre-processing step +(either option 1 or option 2) followed by a know post-processing step. + +.. code-block:: python + + @python_app + def pre_process(): + return 3 + + @python_app + def option_one(x): + return x*2 + + @python_app + def option_two(x): + return (-x) * 2 + + @join_app + def process(x): + if x > 0: + return option_one(x) + else: + return option_two(x) + + @python_app + def post_process(x): + return str(x) + + assert post_process(process(pre_process()))).result() == "6" + +* Why can't process be a regular python function? + +``process`` needs to inspect the value of ``x`` to make a decision about +what app to launch. So it needs to defer execution until after the +pre-processing stage has completed. In Parsl, the way to defer that is +using apps: even though ``process`` is invoked at the start of the workflow, +it will execute later on, when the Future returned by ``pre_process`` has a +value. + +* Why can't process be a @python_app? + +A Python app, if run in a `parsl.executors.ThreadPoolExecutor`, can launch +more parsl apps; so a ``python_app`` implementation of process() would be able +to inspect x and choose and invoke the appropriate ``option_{one, two}``. + +From launching the ``option_{one, two}`` app, the app body python code would +get a ``Future[int]`` - a ``Future`` that will eventually contain ``int``. + +But, we want to invoke ``post_process`` at submission time near the start of +workflow so that Parsl knows about as many tasks as possible. But we don't +want it to execute until the value of the chosen ``option_{one, two}`` app +is known. + +If we don't have join apps, how can we do this? + +We could make process wait for ``option_{one, two}`` to complete, before +returning, like this: + +.. code-block:: python + + @python_app + def process(x): + if x > 0: + f = option_one(x) + else: + f = option_two(x) + return f.result() + +but this will block the worker running ``process`` until ``option_{one, two}`` +has completed. If there aren't enough workers to run ``option_{one, two}`` this +can even deadlock. (principle: apps should not wait on completion of other +apps and should always allow parsl to handle this through dependencies) + +We could make process return the ``Future`` to the main workflow thread: + +.. code-block:: python + + @python_app + def process(x): + if x > 0: + f = option_one(x) + else: + f = option_two(x) + return f # f is a Future[int] + + # process(3) is a Future[Future[int]] + + +What comes out of invoking ``process(x)`` now is a nested ``Future[Future[int]]`` +- it's a promise that eventually process will give you a promise (from +``option_one, two}``) that will eventually give you an int. + +We can't pass that future into post_process... because post_process wants the +final int, and that future will complete before the int is ready, and that +(outer) future will have as its value the inner future (which won't be complete yet). + +So we could wait for the result in the main workflow thread: + +.. code-block:: python + + f_outer = process(pre_process()) # Future[Future[int]] + f_inner = f_outer.result # Future[int] + result = post_process(f_inner) + # result == "6" + +But this now blocks the main workflow thread. If we really only need to run +these three lines, that's fine, but what about if we are in a for loop that +sets up 1000 parametrised iterations: + +.. code-block:: python + + for x in [1..1000]: + f_outer = process(pre_process(x)) # Future[Future[int]] + f_inner = f_outer.result() # Future[int] + result = post_process(f_inner) + +The ``for`` loop can only iterate after pre_processing is done for each +iteration - it is unnecessarily serialised by the ``.result()`` call, +so that pre_processing cannot run in parallel. + +So, the rule about not calling ``.result()`` applies in the main workflow thread +too. + +What join apps add is the ability for parsl to unwrap that Future[Future[int]] into a +Future[int] in a "sensible" way (eg it doesn't need to block a worker). + + +.. _label-join-globus-compute: + +Example of invoking a Futures-driven task from another system +------------------------------------------------------------- + + +This example shows launching some activity in another system, without +occupying a Parsl worker while that activity happens: in this example, work is +delegated to Globus Compute, which performs the work elsewhere. When the work +is completed, Globus Compute will put the result into the future that it +returns, and then (because the Parsl app is a ``@join_app``), that result will +be used as the result of the Parsl app. + +As above, the motivation for doing this inside an app, rather than in the +top level is that sufficient information to launch the Globus Compute task +might not be available at start of the workflow. + +This workflow will run a first stage, ``const_five``, on a Parsl worker, +then using the result of that stage, pass the result as a parameter to a +Globus Compute task, getting a ``Future`` from that submission. Then, the +results of the Globus Compute task will be passed onto a second Parsl +local task, ``times_two``. + +.. code-block:: python + + import parsl + from globus_compute_sdk import Executor + + tutorial_endpoint_uuid = '4b116d3c-1703-4f8f-9f6f-39921e5864df' + gce = Executor(endpoint_id=tutorial_endpoint_uuid) + + def increment_in_funcx(n): + return n+1 + + @parsl.join_app + def increment_in_parsl(n): + future = gce.submit(increment_in_funcx, n) + return future + + @parsl.python_app + def times_two(n): + return n*2 + + @parsl.python_app + def const_five(): + return 5 + + parsl.load() + + workflow = times_two(increment_in_parsl(const_five())) + + r = workflow.result() + + assert r == (5+1)*2 + + +Terminology +----------- + +The term "join" comes from use of monads in functional programming, especially Haskell. diff --git a/docs/userguide/apps/mpi_apps.rst b/docs/userguide/apps/mpi_apps.rst new file mode 100644 index 0000000000..82123123b6 --- /dev/null +++ b/docs/userguide/apps/mpi_apps.rst @@ -0,0 +1,153 @@ +MPI and Multi-node Apps +======================= + +The :class:`~parsl.executors.MPIExecutor` supports running MPI applications or other computations which can +run on multiple compute nodes. + +Background +---------- + +MPI applications run multiple copies of a program that complete a single task by +coordinating using messages passed within or across nodes. + +Starting MPI application requires invoking a "launcher" code (e.g., ``mpiexec``) +with options that define how the copies of a program should be distributed. + +The launcher includes options that control how copies of the program are distributed +across the nodes (e.g., how many copies per node) and +how each copy is configured (e.g., which CPU cores it can use). + +The options for launchers vary between MPI implementations and compute clusters. + +Configuring ``MPIExecutor`` +--------------------------- + +The :class:`~parsl.executors.MPIExecutor` is a wrapper over +:class:`~parsl.executors.high_throughput.executor.HighThroughputExecutor` +which eliminates options that are irrelevant for MPI applications. + +Define a configuration for :class:`~parsl.executors.MPIExecutor` by + +1. Setting ``max_workers_per_block`` to the maximum number of tasks to run per block of compute nodes. + This value is typically the number of nodes per block divided by the number of nodes per task. +2. Setting ``mpi_launcher`` to the launcher used for your application. +3. Specifying a provider that matches your cluster and use the :class:`~parsl.launchers.SimpleLauncher`, + which will ensure that no Parsl processes are placed on the compute nodes. + +An example for ALCF's Polaris supercomputer that will run 3 MPI tasks of 2 nodes each at the same time: + +.. code-block:: python + + config = Config( + executors=[ + MPIExecutor( + address=address_by_interface('bond0'), + max_workers_per_block=3, # Assuming 2 nodes per task + provider=PBSProProvider( + account="parsl", + worker_init=f"""module load miniconda; source activate /lus/eagle/projects/parsl/env""", + walltime="1:00:00", + queue="debug", + scheduler_options="#PBS -l filesystems=home:eagle:grand", + launcher=SimpleLauncher(), + select_options="ngpus=4", + nodes_per_block=6, + max_blocks=1, + cpus_per_node=64, + ), + ), + ] + ) + + +.. warning:: + Please note that ``Provider`` options that specify per-task or per-node resources, for example, + ``SlurmProvider(cores_per_node=N, ...)`` should not be used with :class:`~parsl.executors.high_throughput.MPIExecutor`. + Parsl primarily uses a pilot job model and assumptions from that context do not translate to the MPI context. For + more info refer to : + `github issue #3006 `_ + +Writing an MPI App +------------------ + +:class:`~parsl.executors.high_throughput.MPIExecutor` can execute both Python or Bash Apps which invoke an MPI application. + +Create the app by first defining a function which includes ``parsl_resource_specification`` keyword argument. +The resource specification is a dictionary which defines the number of nodes and ranks used by the application: + +.. code-block:: python + + resource_specification = { + 'num_nodes': , # Number of nodes required for the application instance + 'ranks_per_node': , # Number of ranks / application elements to be launched per node + 'num_ranks': , # Number of ranks in total + } + +Then, replace the call to the MPI launcher with ``$PARSL_MPI_PREFIX``. +``$PARSL_MPI_PREFIX`` references an environmental variable which will be replaced with +the correct MPI launcher configured for the resource list provided when calling the function +and with options that map the task to nodes which Parsl knows to be available. + +The function can be a Bash app + +.. code-block:: python + + @bash_app + def lammps_mpi_application(infile: File, parsl_resource_specification: Dict): + # PARSL_MPI_PREFIX will resolve to `mpiexec -n 4 -ppn 2 -hosts NODE001,NODE002` + return f"$PARSL_MPI_PREFIX lmp_mpi -in {infile.filepath}" + + +or a Python app: + +.. code-block:: python + + @python_app + def lammps_mpi_application(infile: File, parsl_resource_specification: Dict): + from subprocess import run + with open('stdout.lmp', 'w') as fp, open('stderr.lmp', 'w') as fe: + proc = run(['$PARSL_MPI_PREFIX', '-i', 'in.lmp'], stdout=fp, stderr=fe) + return proc.returncode + + +Run either App by calling with its arguments and a resource specification which defines how to execute it + +.. code-block:: python + + # Resources in terms of nodes and how ranks are to be distributed are set on a per app + # basis via the resource_spec dictionary. + resource_spec = { + "num_nodes": 2, + "ranks_per_node": 2, + "num_ranks": 4, + } + future = lammps_mpi_application(File('in.file'), parsl_resource_specification=resource_spec) + +Advanced: More Environment Variables +++++++++++++++++++++++++++++++++++++ + +Parsl Apps which run using :class:`~parsl.executors.high_throughput.MPIExecutor` +can make their own MPI invocation using other environment variables. + +These other variables include versions of the launch command for different launchers + +- ``PARSL_MPIEXEC_PREFIX``: mpiexec launch command which works for a large number of batch systems especially PBS systems +- ``PARSL_SRUN_PREFIX``: srun launch command for Slurm based clusters +- ``PARSL_APRUN_PREFIX``: aprun launch command prefix for some Cray machines + +And the information used by Parsl when assembling the launcher commands: + +- ``PARSL_NUM_RANKS``: Total number of ranks to use for the MPI application +- ``PARSL_NUM_NODES``: Number of nodes to use for the calculation +- ``PARSL_MPI_NODELIST``: List of assigned nodes separated by commas (Eg, NODE1,NODE2) +- ``PARSL_RANKS_PER_NODE``: Number of ranks per node + +Limitations ++++++++++++ + +Support for MPI tasks in HTEX is limited. It is designed for running many multi-node MPI applications within a single +batch job. + +#. MPI tasks may not span across nodes from more than one block. +#. Parsl does not correctly determine the number of execution slots per block (`Issue #1647 `_) +#. The executor uses a Python process per task, which can use a lot of memory (`Issue #2264 `_) \ No newline at end of file diff --git a/docs/userguide/apps.rst b/docs/userguide/apps/python.rst similarity index 71% rename from docs/userguide/apps.rst rename to docs/userguide/apps/python.rst index 41a988db6d..00fac2ac30 100644 --- a/docs/userguide/apps.rst +++ b/docs/userguide/apps/python.rst @@ -1,22 +1,3 @@ -.. _apps: - -Apps -==== - -An **App** defines a computation that will be executed asynchronously by Parsl. -Apps are Python functions marked with a decorator which -designates that the function will run asynchronously and cause it to return -a :class:`~concurrent.futures.Future` instead of the result. - -Apps can be one of three types of functions, each with their own type of decorator - -- ``@python_app``: Most Python functions -- ``@bash_app``: A Python function which returns a command line program to execute -- ``@join_app``: A function which launches one or more new Apps - -The intricacies of Python and Bash apps are documented below. Join apps are documented in a later -section (see :ref:`label-joinapp`). - Python Apps ----------- @@ -187,10 +168,10 @@ There are several classes of allowed types, each with different rules. capital_future = capitalize(first_line_future) print(capital_future.result()) - See the section on `Futures `_ for more details. + See the section on `Futures <../workflows/futures.html>`_ for more details. -Learn more about the types of data allowed in `the data section `_. +Learn more about the types of data allowed in `the data section <../configuration/data.html>`_. .. note:: @@ -203,7 +184,7 @@ Special Keyword Arguments Some keyword arguments to the Python function are treated differently by Parsl -1. inputs: (list) This keyword argument defines a list of input :ref:`label-futures` or files. +1. inputs: (list) This keyword argument defines a list of input :ref:`label-futures` or files. Parsl will wait for the results of any listed :ref:`label-futures` to be resolved before executing the app. The ``inputs`` argument is useful both for passing files as arguments and when one wishes to pass in an arbitrary number of futures at call time. @@ -225,7 +206,7 @@ Some keyword arguments to the Python function are treated differently by Parsl 2. outputs: (list) This keyword argument defines a list of files that will be produced by the app. For each file thus listed, Parsl will create a future, - track the file, and ensure that it is correctly created. The future + track the file, and ensure that it is correctly created. The future can then be passed to other apps as an input argument. .. code-block:: python @@ -253,7 +234,7 @@ Outputs +++++++ A Python app returns an AppFuture (see :ref:`label-futures`) as a proxy for the results that will be returned by the -app once it is executed. This future can be inspected to obtain task status; +app once it is executed. This future can be inspected to obtain task status; and it can be used to wait for the result, and when complete, present the output Python object(s) returned by the app. In case of an error or app failure, the future holds the exception raised by the app. @@ -282,70 +263,3 @@ To summarize, any Python function can be made a Python App with a few restrictio 2. Functions must explicitly import any required modules if they are defined in script which starts Parsl. 3. Parsl uses dill and pickle to serialize Python objects to/from apps. Therefore, Parsl require that all input and output objects can be serialized by dill or pickle. See :ref:`label_serialization_error`. 4. STDOUT and STDERR produced by Python apps remotely are not captured. - - -Bash Apps ---------- - -.. code-block:: python - - @bash_app - def echo( - name: str, - stdout=parsl.AUTO_LOGNAME # Requests Parsl to return the stdout - ): - return f'echo "Hello, {name}!"' - - future = echo('user') - future.result() # block until task has completed - - with open(future.stdout, 'r') as f: - print(f.read()) - - -A Parsl Bash app executes an external application by making a command-line execution. -Parsl will execute the string returned by the function as a command-line script on a remote worker. - -Rules for Function Contents -^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Bash Apps follow the same rules :ref:`as Python Apps `. -For example, imports may need to be inside functions and global variables will be inaccessible. - -Inputs and Outputs -^^^^^^^^^^^^^^^^^^ - -Bash Apps can use the same kinds of inputs as Python Apps, but only communicate results with Files. - -The Bash Apps, unlike Python Apps, can also return the content printed to the Standard Output and Error. - -Special Keywords Arguments -++++++++++++++++++++++++++ - -In addition to the ``inputs``, ``outputs``, and ``walltime`` keyword arguments -described above, a Bash app can accept the following keywords: - -1. stdout: (string, tuple or ``parsl.AUTO_LOGNAME``) The path to a file to which standard output should be redirected. If set to ``parsl.AUTO_LOGNAME``, the log will be automatically named according to task id and saved under ``task_logs`` in the run directory. If set to a tuple ``(filename, mode)``, standard output will be redirected to the named file, opened with the specified mode as used by the Python `open `_ function. -2. stderr: (string or ``parsl.AUTO_LOGNAME``) Like stdout, but for the standard error stream. -3. label: (string) If the app is invoked with ``stdout=parsl.AUTO_LOGNAME`` or ``stderr=parsl.AUTO_LOGNAME``, this argument will be appended to the log name. - -Outputs -+++++++ - -If the Bash app exits with Unix exit code 0, then the AppFuture will complete. If the Bash app -exits with any other code, Parsl will treat this as a failure, and the AppFuture will instead -contain an `BashExitFailure` exception. The Unix exit code can be accessed through the -``exitcode`` attribute of that `BashExitFailure`. - - -Execution Options -^^^^^^^^^^^^^^^^^ - -Bash Apps have the same execution options (e.g., pinning to specific sites) as the Python Apps. - -MPI Apps -^^^^^^^^ - -Applications which employ MPI to span multiple nodes are a special case of Bash apps, -and require special modification of Parsl's `execution environment `_ to function. -Support for MPI applications is described `in a later section `_. diff --git a/docs/userguide/checkpoints.rst b/docs/userguide/checkpoints.rst index 8867107b7a..23af844c17 100644 --- a/docs/userguide/checkpoints.rst +++ b/docs/userguide/checkpoints.rst @@ -1,299 +1,9 @@ -.. _label-memos: +:orphan: -Memoization and checkpointing ------------------------------ +.. meta:: + :content http-equiv="refresh": 0;url=workflows/checkpoints.html -When an app is invoked several times with the same parameters, Parsl can -reuse the result from the first invocation without executing the app again. +Redirect +-------- -This can save time and computational resources. - -This is done in two ways: - -* Firstly, *app caching* will allow reuse of results within the same run. - -* Building on top of that, *checkpointing* will store results on the filesystem - and reuse those results in later runs. - -.. _label-appcaching: - -App caching -=========== - - -There are many situations in which a program may be re-executed -over time. Often, large fragments of the program will not have changed -and therefore, re-execution of apps will waste valuable time and -computation resources. Parsl's app caching solves this problem by -storing results from apps that have successfully completed -so that they can be re-used. - -App caching is enabled by setting the ``cache`` -argument in the :func:`~parsl.app.app.python_app` or :func:`~parsl.app.app.bash_app` -decorator to ``True`` (by default it is ``False``). - -.. code-block:: python - - @bash_app(cache=True) - def hello (msg, stdout=None): - return 'echo {}'.format(msg) - -App caching can be globally disabled by setting ``app_cache=False`` -in the :class:`~parsl.config.Config`. - -App caching can be particularly useful when developing interactive programs such as when -using a Jupyter notebook. In this case, cells containing apps are often re-executed -during development. Using app caching will ensure that only modified apps are re-executed. - - -App equivalence -^^^^^^^^^^^^^^^ - -Parsl determines app equivalence using the name of the app function: -if two apps have the same name, then they are equivalent under this -relation. - -Changes inside the app, or by functions called by an app will not invalidate -cached values. - -There are lots of other ways functions might be compared for equivalence, -and `parsl.dataflow.memoization.id_for_memo` provides a hook to plug in -alternate application-specific implementations. - - -Invocation equivalence -^^^^^^^^^^^^^^^^^^^^^^ - -Two app invocations are determined to be equivalent if their -input arguments are identical. - -In simple cases, this follows obvious rules: - -.. code-block:: python - - # these two app invocations are the same and the second invocation will - # reuse any cached input from the first invocation - x = 7 - f(x).result() - - y = 7 - f(y).result() - - -Internally, equivalence is determined by hashing the input arguments, and -comparing the hash to hashes from previous app executions. - -This approach can only be applied to data types for which a deterministic hash -can be computed. - -By default Parsl can compute sensible hashes for basic data types: -str, int, float, None, as well as more some complex types: -functions, and dictionaries and lists containing hashable types. - -Attempting to cache apps invoked with other, non-hashable, data types will -lead to an exception at invocation. - -In that case, mechanisms to hash new types can be registered by a program by -implementing the `parsl.dataflow.memoization.id_for_memo` function for -the new type. - -Ignoring arguments -^^^^^^^^^^^^^^^^^^ - -On occasion one may wish to ignore particular arguments when determining -app invocation equivalence - for example, when generating log file -names automatically based on time or run information. -Parsl allows developers to list the arguments to be ignored -in the ``ignore_for_cache`` app decorator parameter: - -.. code-block:: python - - @bash_app(cache=True, ignore_for_cache=['stdout']) - def hello (msg, stdout=None): - return 'echo {}'.format(msg) - - -Caveats -^^^^^^^ - -It is important to consider several important issues when using app caching: - -- Determinism: App caching is generally useful only when the apps are deterministic. - If the outputs may be different for identical inputs, app caching will obscure - this non-deterministic behavior. For instance, caching an app that returns - a random number will result in every invocation returning the same result. - -- Timing: If several identical calls to an app are made concurrently having - not yet cached a result, many instances of the app will be launched. - Once one invocation completes and the result is cached - all subsequent calls will return immediately with the cached result. - -- Performance: If app caching is enabled, there may be some performance - overhead especially if a large number of short duration tasks are launched rapidly. - This overhead has not been quantified. - -.. _label-checkpointing: - -Checkpointing -============= - -Large-scale Parsl programs are likely to encounter errors due to node failures, -application or environment errors, and myriad other issues. Parsl offers an -application-level checkpointing model to improve resilience, fault tolerance, and -efficiency. - -.. note:: - Checkpointing builds on top of app caching, and so app caching must be - enabled. If app caching is disabled in the config ``Config.app_cache``, checkpointing will - not work. - -Parsl follows an incremental checkpointing model, where each checkpoint file contains -all results that have been updated since the last checkpoint. - -When a Parsl program loads a checkpoint file and is executed, it will use -checkpointed results for any apps that have been previously executed. -Like app caching, checkpoints -use the hash of the app and the invocation input parameters to identify previously computed -results. If multiple checkpoints exist for an app (with the same hash) -the most recent entry will be used. - -Parsl provides four checkpointing modes: - -1. ``task_exit``: a checkpoint is created each time an app completes or fails - (after retries if enabled). This mode minimizes the risk of losing information - from completed tasks. - - .. code-block:: python - - from parsl.configs.local_threads import config - config.checkpoint_mode = 'task_exit' - -2. ``periodic``: a checkpoint is created periodically using a user-specified - checkpointing interval. Results will be saved to the checkpoint file for - all tasks that have completed during this period. - - .. code-block:: python - - from parsl.configs.local_threads import config - config.checkpoint_mode = 'periodic' - config.checkpoint_period = "01:00:00" - -3. ``dfk_exit``: checkpoints are created when Parsl is - about to exit. This reduces the risk of losing results due to - premature program termination from exceptions, terminate signals, etc. However - it is still possible that information might be lost if the program is - terminated abruptly (machine failure, SIGKILL, etc.) - - .. code-block:: python - - from parsl.configs.local_threads import config - config.checkpoint_mode = 'dfk_exit' - -4. ``manual``: in addition to these automated checkpointing modes, it is also possible - to manually initiate a checkpoint by calling ``DataFlowKernel.checkpoint()`` in the - Parsl program code. - - .. code-block:: python - - import parsl - from parsl.configs.local_threads import config - dfk = parsl.load(config) - .... - dfk.checkpoint() - -In all cases the checkpoint file is written out to the ``runinfo/RUN_ID/checkpoint/`` directory. - -.. Note:: Checkpoint modes ``periodic``, ``dfk_exit``, and ``manual`` can interfere with garbage collection. - In these modes task information will be retained after completion, until checkpointing events are triggered. - - -Creating a checkpoint -^^^^^^^^^^^^^^^^^^^^^ - -Automated checkpointing must be explicitly enabled in the Parsl configuration. -There is no need to modify a Parsl program as checkpointing will occur transparently. -In the following example, checkpointing is enabled at task exit. The results of -each invocation of the ``slow_double`` app will be stored in the checkpoint file. - -.. code-block:: python - - import parsl - from parsl.app.app import python_app - from parsl.configs.local_threads import config - - config.checkpoint_mode = 'task_exit' - - parsl.load(config) - - @python_app(cache=True) - def slow_double(x): - import time - time.sleep(5) - return x * 2 - - d = [] - for i in range(5): - d.append(slow_double(i)) - - print([d[i].result() for i in range(5)]) - -Alternatively, manual checkpointing can be used to explictly specify when the checkpoint -file should be saved. The following example shows how manual checkpointing can be used. -Here, the ``dfk.checkpoint()`` function will save the results of the prior invocations -of the ``slow_double`` app. - -.. code-block:: python - - import parsl - from parsl import python_app - from parsl.configs.local_threads import config - - dfk = parsl.load(config) - - @python_app(cache=True) - def slow_double(x, sleep_dur=1): - import time - time.sleep(sleep_dur) - return x * 2 - - N = 5 # Number of calls to slow_double - d = [] # List to store the futures - for i in range(0, N): - d.append(slow_double(i)) - - # Wait for the results - [i.result() for i in d] - - cpt_dir = dfk.checkpoint() - print(cpt_dir) # Prints the checkpoint dir - - -Resuming from a checkpoint -^^^^^^^^^^^^^^^^^^^^^^^^^^ - -When resuming a program from a checkpoint Parsl allows the user to select -which checkpoint file(s) to use. -Checkpoint files are stored in the ``runinfo/RUNID/checkpoint`` directory. - -The example below shows how to resume using all available checkpoints. -Here, the program re-executes the same calls to the ``slow_double`` app -as above and instead of waiting for results to be computed, the values -from the checkpoint file are are immediately returned. - -.. code-block:: python - - import parsl - from parsl.tests.configs.local_threads import config - from parsl.utils import get_all_checkpoints - - config.checkpoint_files = get_all_checkpoints() - - parsl.load(config) - - # Rerun the same workflow - d = [] - for i in range(5): - d.append(slow_double(i)) - - # wait for results - print([d[i].result() for i in range(5)]) +This page has been `moved `_ diff --git a/docs/userguide/configuration/data.rst b/docs/userguide/configuration/data.rst new file mode 100644 index 0000000000..0c4c4b334d --- /dev/null +++ b/docs/userguide/configuration/data.rst @@ -0,0 +1,415 @@ +.. _label-data: + +Staging data files +================== + +Parsl apps can take and return data files. A file may be passed as an input +argument to an app, or returned from an app after execution. Parsl +provides support to automatically transfer (stage) files between +the main Parsl program, worker nodes, and external data storage systems. + +Input files can be passed as regular arguments, or a list of them may be +specified in the special ``inputs`` keyword argument to an app invocation. + +Inside an app, the ``filepath`` attribute of a `File` can be read to determine +where on the execution-side file system the input file has been placed. + +Output `File` objects must also be passed in at app invocation, through the +outputs parameter. In this case, the `File` object specifies where Parsl +should place output after execution. + +Inside an app, the ``filepath`` attribute of an output +`File` provides the path at which the corresponding output file should be +placed so that Parsl can find it after execution. + +If the output from an app is to be used as the input to a subsequent app, +then a `DataFuture` that represents whether the output file has been created +must be extracted from the first app's AppFuture, and that must be passed +to the second app. This causes app +executions to be properly ordered, in the same way that passing AppFutures +to subsequent apps causes execution ordering based on an app returning. + +In a Parsl program, file handling is split into two pieces: files are named in an +execution-location independent manner using :py:class:`~parsl.data_provider.files.File` +objects, and executors are configured to stage those files in to and out of +execution locations using instances of the :py:class:`~parsl.data_provider.staging.Staging` +interface. + + +Parsl files +----------- + +Parsl uses a custom :py:class:`~parsl.data_provider.files.File` to provide a +location-independent way of referencing and accessing files. +Parsl files are defined by specifying the URL *scheme* and a path to the file. +Thus a file may represent an absolute path on the submit-side file system +or a URL to an external file. + +The scheme defines the protocol via which the file may be accessed. +Parsl supports the following schemes: file, ftp, http, https, and globus. +If no scheme is specified Parsl will default to the file scheme. + +The following example shows creation of two files with different +schemes: a locally-accessible data.txt file and an HTTPS-accessible +README file. + +.. code-block:: python + + File('file://home/parsl/data.txt') + File('https://github.com/Parsl/parsl/blob/master/README.rst') + + +Parsl automatically translates the file's location relative to the +environment in which it is accessed (e.g., the Parsl program or an app). +The following example shows how a file can be accessed in the app +irrespective of where that app executes. + +.. code-block:: python + + @python_app + def print_file(inputs=()): + with open(inputs[0].filepath, 'r') as inp: + content = inp.read() + return(content) + + # create an remote Parsl file + f = File('https://github.com/Parsl/parsl/blob/master/README.rst') + + # call the print_file app with the Parsl file + r = print_file(inputs=[f]) + r.result() + +As described below, the method by which this files are transferred +depends on the scheme and the staging providers specified in the Parsl +configuration. + +Staging providers +----------------- + +Parsl is able to transparently stage files between at-rest locations and +execution locations by specifying a list of +:py:class:`~parsl.data_provider.staging.Staging` instances for an executor. +These staging instances define how to transfer files in and out of an execution +location. This list should be supplied as the ``storage_access`` +parameter to an executor when it is constructed. + +Parsl includes several staging providers for moving files using the +schemes defined above. By default, Parsl executors are created with +three common staging providers: +the NoOpFileStaging provider for local and shared file systems +and the HTTP(S) and FTP staging providers for transferring +files to and from remote storage locations. The following +example shows how to explicitly set the default staging providers. + +.. code-block:: python + + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + from parsl.data_provider.data_manager import default_staging + + config = Config( + executors=[ + HighThroughputExecutor( + storage_access=default_staging, + # equivalent to the following + # storage_access=[NoOpFileStaging(), FTPSeparateTaskStaging(), HTTPSeparateTaskStaging()], + ) + ] + ) + + +Parsl further differentiates when staging occurs relative to +the app invocation that requires or produces files. +Staging either occurs with the executing task (*in-task staging*) +or as a separate task (*separate task staging*) before app execution. +In-task staging +uses a wrapper that is executed around the Parsl task and thus +occurs on the resource on which the task is executed. Separate +task staging inserts a new Parsl task in the graph and associates +a dependency between the staging task and the task that depends +on that file. Separate task staging may occur on either the submit-side +(e.g., when using Globus) or on the execution-side (e.g., HTTPS, FTP). + + +NoOpFileStaging for Local/Shared File Systems +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +The NoOpFileStaging provider assumes that files specified either +with a path or with the ``file`` URL scheme are available both +on the submit and execution side. This occurs, for example, when there is a +shared file system. In this case, files will not moved, and the +File object simply presents the same file path to the Parsl program +and any executing tasks. + +Files defined as follows will be handled by the NoOpFileStaging provider. + +.. code-block:: python + + File('file://home/parsl/data.txt') + File('/home/parsl/data.txt') + + +The NoOpFileStaging provider is enabled by default on all +executors. It can be explicitly set as the only +staging provider as follows. + +.. code-block:: python + + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + from parsl.data_provider.file_noop import NoOpFileStaging + + config = Config( + executors=[ + HighThroughputExecutor( + storage_access=[NoOpFileStaging()] + ) + ] + ) + + +FTP, HTTP, HTTPS: separate task staging +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Files named with the ``ftp``, ``http`` or ``https`` URL scheme will be +staged in using HTTP GET or anonymous FTP commands. These commands +will be executed as a separate +Parsl task that will complete before the corresponding app +executes. These providers cannot be used to stage out output files. + +The following example defines a file accessible on a remote FTP server. + +.. code-block:: python + + File('ftp://www.iana.org/pub/mirror/rirstats/arin/ARIN-STATS-FORMAT-CHANGE.txt') + +When such a file object is passed as an input to an app, Parsl will download the file to whatever location is selected for the app to execute. +The following example illustrates how the remote file is implicitly downloaded from an FTP server and then converted. Note that the app does not need to know the location of the downloaded file on the remote computer, as Parsl abstracts this translation. + +.. code-block:: python + + @python_app + def convert(inputs=(), outputs=()): + with open(inputs[0].filepath, 'r') as inp: + content = inp.read() + with open(outputs[0].filepath, 'w') as out: + out.write(content.upper()) + + # create an remote Parsl file + inp = File('ftp://www.iana.org/pub/mirror/rirstats/arin/ARIN-STATS-FORMAT-CHANGE.txt') + + # create a local Parsl file + out = File('file:///tmp/ARIN-STATS-FORMAT-CHANGE.txt') + + # call the convert app with the Parsl file + f = convert(inputs=[inp], outputs=[out]) + f.result() + +HTTP and FTP separate task staging providers can be configured as follows. + +.. code-block:: python + + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + from parsl.data_provider.http import HTTPSeparateTaskStaging + from parsl.data_provider.ftp import FTPSeparateTaskStaging + + config = Config( + executors=[ + HighThroughputExecutor( + storage_access=[HTTPSeparateTaskStaging(), FTPSeparateTaskStaging()] + ) + ] + ) + +FTP, HTTP, HTTPS: in-task staging +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +These staging providers are intended for use on executors that do not have +a file system shared between each executor node. + +These providers will use the same HTTP GET/anonymous FTP as the separate +task staging providers described above, but will do so in a wrapper around +individual app invocations, which guarantees that they will stage files to +a file system visible to the app. + +A downside of this staging approach is that the staging tasks are less visible +to Parsl, as they are not performed as separate Parsl tasks. + +In-task staging providers can be configured as follows. + +.. code-block:: python + + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + from parsl.data_provider.http import HTTPInTaskStaging + from parsl.data_provider.ftp import FTPInTaskStaging + + config = Config( + executors=[ + HighThroughputExecutor( + storage_access=[HTTPInTaskStaging(), FTPInTaskStaging()] + ) + ] + ) + + +Globus +^^^^^^ + +The ``Globus`` staging provider is used to transfer files that can be accessed +using Globus. A guide to using Globus is available `here +`_). + +A file using the Globus scheme must specify the UUID of the Globus +endpoint and a path to the file on the endpoint, for example: + +.. code-block:: python + + File('globus://037f054a-15cf-11e8-b611-0ac6873fc732/unsorted.txt') + +Note: a Globus endpoint's UUID can be found in the Globus `Manage Endpoints `_ page. + +There must also be a Globus endpoint available with access to a +execute-side file system, because Globus file transfers happen +between two Globus endpoints. + +Globus Configuration +"""""""""""""""""""" + +In order to manage where files are staged, users must configure the default ``working_dir`` on a remote location. This information is specified in the :class:`~parsl.executors.base.ParslExecutor` via the ``working_dir`` parameter in the :class:`~parsl.config.Config` instance. For example: + +.. code-block:: python + + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + + config = Config( + executors=[ + HighThroughputExecutor( + working_dir="/home/user/data" + ) + ] + ) + +Parsl requires knowledge of the Globus endpoint that is associated with an executor. This is done by specifying the ``endpoint_name`` (the UUID of the Globus endpoint that is associated with the system) in the configuration. + +In some cases, for example when using a Globus `shared endpoint `_ or when a Globus endpoint is mounted on a supercomputer, the path seen by Globus is not the same as the local path seen by Parsl. In this case the configuration may optionally specify a mapping between the ``endpoint_path`` (the common root path seen in Globus), and the ``local_path`` (the common root path on the local file system), as in the following. In most cases, ``endpoint_path`` and ``local_path`` are the same and do not need to be specified. + +.. code-block:: python + + from parsl.config import Config + from parsl.executors import HighThroughputExecutor + from parsl.data_provider.globus import GlobusStaging + from parsl.data_provider.data_manager import default_staging + + config = Config( + executors=[ + HighThroughputExecutor( + working_dir="/home/user/parsl_script", + storage_access=default_staging + [GlobusStaging( + endpoint_uuid="7d2dc622-2edb-11e8-b8be-0ac6873fc732", + endpoint_path="/", + local_path="/home/user" + )] + ) + ] + ) + + +Globus Authorization +"""""""""""""""""""" + +In order to transfer files with Globus, the user must first authenticate. +The first time that Globus is used with Parsl on a computer, the program +will prompt the user to follow an authentication and authorization +procedure involving a web browser. Users can authorize out of band by +running the parsl-globus-auth utility. This is useful, for example, +when running a Parsl program in a batch system where it will be unattended. + +.. code-block:: bash + + $ parsl-globus-auth + Parsl Globus command-line authorizer + If authorization to Globus is necessary, the library will prompt you now. + Otherwise it will do nothing + Authorization complete + +rsync +^^^^^ + +The ``rsync`` utility can be used to transfer files in the ``file`` scheme in configurations where +workers cannot access the submit-side file system directly, such as when executing +on an AWS EC2 instance or on a cluster without a shared file system. +However, the submit-side file system must be exposed using rsync. + +rsync Configuration +""""""""""""""""""" + +``rsync`` must be installed on both the submit and worker side. It can usually be installed +by using the operating system package manager: for example, by ``apt-get install rsync``. + +An `RSyncStaging` option must then be added to the Parsl configuration file, as in the following. +The parameter to RSyncStaging should describe the prefix to be passed to each rsync +command to connect from workers to the submit-side host. This will often be the username +and public IP address of the submitting system. + +.. code-block:: python + + from parsl.data_provider.rsync import RSyncStaging + + config = Config( + executors=[ + HighThroughputExecutor( + storage_access=[HTTPInTaskStaging(), FTPInTaskStaging(), RSyncStaging("benc@" + public_ip)], + ... + ) + ) + +rsync Authorization +""""""""""""""""""" + +The rsync staging provider delegates all authentication and authorization to the +underlying ``rsync`` command. This command must be correctly authorized to connect back to +the submit-side system. The form of this authorization will depend on the systems in +question. + +The following example installs an ssh key from the submit-side file system and turns off host key +checking, in the ``worker_init`` initialization of an EC2 instance. The ssh key must have +sufficient privileges to run ``rsync`` over ssh on the submit-side system. + +.. code-block:: python + + with open("rsync-callback-ssh", "r") as f: + private_key = f.read() + + ssh_init = """ + mkdir .ssh + chmod go-rwx .ssh + + cat > .ssh/id_rsa < .ssh/config <`_ to encrypt all communication channels +between the executor and related nodes. + +Encryption performance +^^^^^^^^^^^^^^^^^^^^^^ + +CurveZMQ depends on `libzmq `_ and `libsodium `_, +which `pyzmq `_ (a Parsl dependency) includes as part of its +installation via ``pip``. This installation path should work on most systems, but users have +reported significant performance degradation as a result. + +If you experience a significant performance hit after enabling encryption, we recommend installing +``pyzmq`` with conda: + +.. code-block:: bash + + conda install conda-forge::pyzmq + +Alternatively, you can `install libsodium `_, then +`install libzmq `_, then build ``pyzmq`` from source: + +.. code-block:: bash + + pip3 install parsl --no-binary pyzmq diff --git a/docs/userguide/configuration/examples.rst b/docs/userguide/configuration/examples.rst new file mode 100644 index 0000000000..7e9b7ae9eb --- /dev/null +++ b/docs/userguide/configuration/examples.rst @@ -0,0 +1,333 @@ +Example configurations +====================== + +.. note:: + All configuration examples below must be customized for the user's + allocation, Python environment, file system, etc. + + +The configuration specifies what, and how, resources are to be used for executing +the Parsl program and its apps. +It is important to carefully consider the needs of the Parsl program and its apps, +and the characteristics of the compute resources, to determine an ideal configuration. +Aspects to consider include: +1) where the Parsl apps will execute; +2) how many nodes will be used to execute the apps, and how long the apps will run; +3) should Parsl request multiple nodes in an individual scheduler job; and +4) where will the main Parsl program run and how will it communicate with the apps. + +Stepping through the following question should help formulate a suitable configuration object. + +1. Where should apps be executed? + ++---------------------+-----------------------------------------------+----------------------------------------+ +| Target | Executor | Provider | ++=====================+===============================================+========================================+ +| Laptop/Workstation | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.LocalProvider` | +| | * `parsl.executors.ThreadPoolExecutor` | | +| | * `parsl.executors.WorkQueueExecutor` | | +| | * `parsl.executors.taskvine.TaskVineExecutor` | | ++---------------------+-----------------------------------------------+----------------------------------------+ +| Amazon Web Services | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.AWSProvider` | ++---------------------+-----------------------------------------------+----------------------------------------+ +| Google Cloud | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.GoogleCloudProvider` | ++---------------------+-----------------------------------------------+----------------------------------------+ +| Slurm based system | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.SlurmProvider` | +| | * `parsl.executors.WorkQueueExecutor` | | +| | * `parsl.executors.taskvine.TaskVineExecutor` | | ++---------------------+-----------------------------------------------+----------------------------------------+ +| Torque/PBS based | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.TorqueProvider` | +| system | * `parsl.executors.WorkQueueExecutor` | | ++---------------------+-----------------------------------------------+----------------------------------------+ +| GridEngine based | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.GridEngineProvider` | +| system | * `parsl.executors.WorkQueueExecutor` | | ++---------------------+-----------------------------------------------+----------------------------------------+ +| Condor based | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.CondorProvider` | +| cluster or grid | * `parsl.executors.WorkQueueExecutor` | | +| | * `parsl.executors.taskvine.TaskVineExecutor` | | ++---------------------+-----------------------------------------------+----------------------------------------+ +| Kubernetes cluster | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.KubernetesProvider` | ++---------------------+-----------------------------------------------+----------------------------------------+ + + +2. How many nodes will be used to execute the apps? What task durations are necessary to achieve good performance? + + ++--------------------------------------------+----------------------+-------------------------------------+ +| Executor | Number of Nodes [*]_ | Task duration for good performance | ++============================================+======================+=====================================+ +| `parsl.executors.ThreadPoolExecutor` | 1 (Only local) | Any | ++--------------------------------------------+----------------------+-------------------------------------+ +| `parsl.executors.HighThroughputExecutor` | <=2000 | Task duration(s)/#nodes >= 0.01 | +| | | longer tasks needed at higher scale | ++--------------------------------------------+----------------------+-------------------------------------+ +| `parsl.executors.WorkQueueExecutor` | <=1000 [*]_ | 10s+ | ++--------------------------------------------+----------------------+-------------------------------------+ +| `parsl.executors.taskvine.TaskVineExecutor`| <=1000 [*]_ | 10s+ | ++--------------------------------------------+----------------------+-------------------------------------+ + + +.. [*] Assuming 32 workers per node. If there are fewer workers launched + per node, a larger number of nodes could be supported. + +.. [*] The maximum number of nodes tested for the `parsl.executors.WorkQueueExecutor` is 10,000 GPU cores and + 20,000 CPU cores. + +.. [*] The maximum number of nodes tested for the `parsl.executors.taskvine.TaskVineExecutor` is + 10,000 GPU cores and 20,000 CPU cores. + +3. Should Parsl request multiple nodes in an individual scheduler job? +(Here the term block is equivalent to a single scheduler job.) + ++--------------------------------------------------------------------------------------------+ +| ``nodes_per_block = 1`` | ++---------------------+--------------------------+-------------------------------------------+ +| Provider | Executor choice | Suitable Launchers | ++=====================+==========================+===========================================+ +| Systems that don't | Any | * `parsl.launchers.SingleNodeLauncher` | +| use Aprun | | * `parsl.launchers.SimpleLauncher` | ++---------------------+--------------------------+-------------------------------------------+ +| Aprun based systems | Any | * `parsl.launchers.AprunLauncher` | ++---------------------+--------------------------+-------------------------------------------+ + ++---------------------------------------------------------------------------------------------------------------------+ +| ``nodes_per_block > 1`` | ++-------------------------------------+--------------------------+----------------------------------------------------+ +| Provider | Executor choice | Suitable Launchers | ++=====================================+==========================+====================================================+ +| `parsl.providers.TorqueProvider` | Any | * `parsl.launchers.AprunLauncher` | +| | | * `parsl.launchers.MpiExecLauncher` | ++-------------------------------------+--------------------------+----------------------------------------------------+ +| `parsl.providers.SlurmProvider` | Any | * `parsl.launchers.SrunLauncher` if native slurm | +| | | * `parsl.launchers.AprunLauncher`, otherwise | ++-------------------------------------+--------------------------+----------------------------------------------------+ + +.. note:: If using a Cray system, you most likely need to use the `parsl.launchers.AprunLauncher` to launch workers unless you + are on a **native Slurm** system like :ref:`configuring_nersc_cori` + +Ad-Hoc Clusters +--------------- + +Parsl's support of ad-hoc clusters of compute nodes without a scheduler +is deprecated. + +See +`issue #3515 `_ +for further discussion. + +Amazon Web Services +------------------- + +.. image:: img/aws_image.png + +.. note:: + To use AWS with Parsl, install Parsl with AWS dependencies via ``python3 -m pip install 'parsl[aws]'`` + +Amazon Web Services is a commercial cloud service which allows users to rent a range of computers and other computing services. +The following snippet shows how Parsl can be configured to provision nodes from the Elastic Compute Cloud (EC2) service. +The first time this configuration is used, Parsl will configure a Virtual Private Cloud and other networking and security infrastructure that will be +re-used in subsequent executions. The configuration uses the `parsl.providers.AWSProvider` to connect to AWS. + +.. literalinclude:: ../../../parsl/configs/ec2.py + + +ASPIRE 1 (NSCC) +--------------- + +.. image:: https://www.nscc.sg/wp-content/uploads/2017/04/ASPIRE1Img.png + +The following snippet shows an example configuration for accessing NSCC's **ASPIRE 1** supercomputer. This example uses the `parsl.executors.HighThroughputExecutor` executor and connects to ASPIRE1's PBSPro scheduler. It also shows how ``scheduler_options`` parameter could be used for scheduling array jobs in PBSPro. + +.. literalinclude:: ../../../parsl/configs/ASPIRE1.py + + + + +Illinois Campus Cluster (UIUC) +------------------------------ + +.. image:: https://campuscluster.illinois.edu/wp-content/uploads/2018/02/ND2_3633-sm.jpg + +The following snippet shows an example configuration for executing on the Illinois Campus Cluster. +The configuration assumes the user is running on a login node and uses the `parsl.providers.SlurmProvider` to interface +with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. + +.. literalinclude:: ../../../parsl/configs/illinoiscluster.py + +Bridges (PSC) +------------- + +.. image:: https://insidehpc.com/wp-content/uploads/2016/08/Bridges_FB1b.jpg + +The following snippet shows an example configuration for executing on the Bridges supercomputer at the Pittsburgh Supercomputing Center. +The configuration assumes the user is running on a login node and uses the `parsl.providers.SlurmProvider` to interface +with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. + +.. literalinclude:: ../../../parsl/configs/bridges.py + + + +CC-IN2P3 +-------- + +.. image:: https://cc.in2p3.fr/wp-content/uploads/2017/03/bandeau_accueil.jpg + +The snippet below shows an example configuration for executing from a login node on IN2P3's Computing Centre. +The configuration uses the `parsl.providers.LocalProvider` to run on a login node primarily to avoid GSISSH, which Parsl does not support. +This system uses Grid Engine which Parsl interfaces with using the `parsl.providers.GridEngineProvider`. + +.. literalinclude:: ../../../parsl/configs/cc_in2p3.py + + +CCL (Notre Dame, TaskVine) +-------------------------- + +.. image:: https://ccl.cse.nd.edu/software/taskvine/taskvine-logo.png + +To utilize TaskVine with Parsl, please install the full CCTools software package within an appropriate Anaconda or Miniconda environment +(instructions for installing Miniconda can be found `in the Conda install guide `_): + +.. code-block:: bash + + $ conda create -y --name python= conda-pack + $ conda activate + $ conda install -y -c conda-forge ndcctools parsl + +This creates a Conda environment on your machine with all the necessary tools and setup needed to utilize TaskVine with the Parsl library. + +The following snippet shows an example configuration for using the Parsl/TaskVine executor to run applications on the local machine. +This examples uses the `parsl.executors.taskvine.TaskVineExecutor` to schedule tasks, and a local worker will be started automatically. +For more information on using TaskVine, including configurations for remote execution, visit the +`TaskVine/Parsl documentation online `_. + +.. literalinclude:: ../../../parsl/configs/vineex_local.py + +TaskVine's predecessor, WorkQueue, may continue to be used with Parsl. +For more information on using WorkQueue visit the `CCTools documentation online `_. + +Expanse (SDSC) +-------------- + +.. image:: https://www.hpcwire.com/wp-content/uploads/2019/07/SDSC-Expanse-graphic-cropped.jpg + +The following snippet shows an example configuration for executing remotely on San Diego Supercomputer +Center's **Expanse** supercomputer. The example is designed to be executed on the login nodes, using the +`parsl.providers.SlurmProvider` to interface with the Slurm scheduler used by Comet and the `parsl.launchers.SrunLauncher` to launch workers. + +.. literalinclude:: ../../../parsl/configs/expanse.py + + +Improv (Argonne LCRC) +--------------------- + +.. image:: https://www.lcrc.anl.gov/sites/default/files/styles/965_wide/public/2023-12/20231214_114057.jpg?itok=A-Rz5pP9 + +**Improv** is a PBS Pro based supercomputer at Argonne's Laboratory Computing Resource +Center (LCRC). The following snippet is an example configuration that uses `parsl.providers.PBSProProvider` +and `parsl.launchers.MpiRunLauncher` to run on multinode jobs. + +.. literalinclude:: ../../../parsl/configs/improv.py + + +.. _configuring_nersc_cori: + +Perlmutter (NERSC) +------------------ + +NERSC provides documentation on `how to use Parsl on Perlmutter `_. +Perlmutter is a Slurm based HPC system and parsl uses `parsl.providers.SlurmProvider` with `parsl.launchers.SrunLauncher` +to launch tasks onto this machine. + + +Frontera (TACC) +--------------- + +.. image:: https://frontera-portal.tacc.utexas.edu/media/filer_public/2c/fb/2cfbf6ab-818d-42c8-b4d5-9b39eb9d0a05/frontera-banner-home.jpg + +Deployed in June 2019, Frontera is the 5th most powerful supercomputer in the world. Frontera replaces the NSF Blue Waters system at NCSA +and is the first deployment in the National Science Foundation's petascale computing program. The configuration below assumes that the user is +running on a login node and uses the `parsl.providers.SlurmProvider` to interface with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. + +.. literalinclude:: ../../../parsl/configs/frontera.py + + +Kubernetes Clusters +------------------- + +.. image:: https://d1.awsstatic.com/PAC/kuberneteslogo.eabc6359f48c8e30b7a138c18177f3fd39338e05.png + +Kubernetes is an open-source system for container management, such as automating deployment and scaling of containers. +The snippet below shows an example configuration for deploying pods as workers on a Kubernetes cluster. +The KubernetesProvider exploits the Python Kubernetes API, which assumes that you have kube config in ``~/.kube/config``. + +.. literalinclude:: ../../../parsl/configs/kubernetes.py + + +Midway (RCC, UChicago) +---------------------- + +.. image:: https://rcc.uchicago.edu/sites/rcc.uchicago.edu/files/styles/slideshow-image/public/uploads/images/slideshows/20140430_RCC_8978.jpg?itok=BmRuJ-wq + +This Midway cluster is a campus cluster hosted by the Research Computing Center at the University of Chicago. +The snippet below shows an example configuration for executing remotely on Midway. +The configuration assumes the user is running on a login node and uses the `parsl.providers.SlurmProvider` to interface +with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. + +.. literalinclude:: ../../../parsl/configs/midway.py + + +Open Science Grid +----------------- + +.. image:: https://www.renci.org/wp-content/uploads/2008/10/osg_logo.png + +The Open Science Grid (OSG) is a national, distributed computing Grid spanning over 100 individual sites to provide tens of thousands of CPU cores. +The snippet below shows an example configuration for executing remotely on OSG. You will need to have a valid project name on the OSG. +The configuration uses the `parsl.providers.CondorProvider` to interface with the scheduler. + +.. literalinclude:: ../../../parsl/configs/osg.py + + +Polaris (ALCF) +-------------- + +.. image:: https://www.alcf.anl.gov/sites/default/files/styles/965x543/public/2022-07/33181D_086_ALCF%20Polaris%20Crop.jpg?itok=HVAHsZtt + :width: 75% + +ALCF provides documentation on `how to use Parsl on Polaris `_. +Polaris uses `parsl.providers.PBSProProvider` and `parsl.launchers.MpiExecLauncher` to launch tasks onto the HPC system. + + + +Stampede2 (TACC) +---------------- + +.. image:: https://www.tacc.utexas.edu/documents/1084364/1413880/stampede2-0717.jpg/ + +The following snippet shows an example configuration for accessing TACC's **Stampede2** supercomputer. This example uses theHighThroughput executor and connects to Stampede2's Slurm scheduler. + +.. literalinclude:: ../../../parsl/configs/stampede2.py + + +Summit (ORNL) +------------- + +.. image:: https://www.olcf.ornl.gov/wp-content/uploads/2018/06/Summit_Exaop-1500x844.jpg + +The following snippet shows an example configuration for executing from the login node on Summit, the leadership class supercomputer hosted at the Oak Ridge National Laboratory. +The example uses the :class:`parsl.providers.LSFProvider` to provision compute nodes from the LSF cluster scheduler and the `parsl.launchers.JsrunLauncher` to launch workers across the compute nodes. + +.. literalinclude:: ../../../parsl/configs/summit.py + + +TOSS3 (LLNL) +------------ + +.. image:: https://hpc.llnl.gov/sites/default/files/Magma--2020-LLNL.jpg + +The following snippet shows an example configuration for executing on one of LLNL's **TOSS3** +machines, such as Quartz, Ruby, Topaz, Jade, or Magma. This example uses the `parsl.executors.FluxExecutor` +and connects to Slurm using the `parsl.providers.SlurmProvider`. This configuration assumes that the script +is being executed on the login nodes of one of the machines. + +.. literalinclude:: ../../../parsl/configs/toss3_llnl.py diff --git a/docs/userguide/configuration/execution.rst b/docs/userguide/configuration/execution.rst new file mode 100644 index 0000000000..ac7217032a --- /dev/null +++ b/docs/userguide/configuration/execution.rst @@ -0,0 +1,227 @@ +.. _label-execution: + +Execution +========= + +Contemporary computing environments may include a wide range of computational platforms or **execution providers**, from laptops and PCs to various clusters, supercomputers, and cloud computing platforms. Different execution providers may require or allow for the use of different **execution models**, such as threads (for efficient parallel execution on a multicore processor), processes, and pilot jobs for running many small tasks on a large parallel system. + +Parsl is designed to abstract these low-level details so that an identical Parsl program can run unchanged on different platforms or across multiple platforms. +To this end, Parsl uses a configuration file to specify which execution provider(s) and execution model(s) to use. +Parsl provides a high level abstraction, called a *block*, for providing a uniform description of a compute resource irrespective of the specific execution provider. + +.. note:: + Refer to :ref:`configuration-section` for information on how to configure the various components described + below for specific scenarios. + +Execution providers +------------------- + +Clouds, supercomputers, and local PCs offer vastly different modes of access. +To overcome these differences, and present a single uniform interface, +Parsl implements a simple provider abstraction. This +abstraction is key to Parsl's ability to enable scripts to be moved +between resources. The provider interface exposes three core actions: submit a +job for execution (e.g., sbatch for the Slurm resource manager), +retrieve the status of an allocation (e.g., squeue), and cancel a running +job (e.g., scancel). Parsl implements providers for local execution +(fork), for various cloud platforms using cloud-specific APIs, and +for clusters and supercomputers that use a Local Resource Manager +(LRM) to manage access to resources, such as Slurm and HTCondor. + +Each provider implementation may allow users to specify additional parameters for further configuration. Parameters are generally mapped to LRM submission script or cloud API options. +Examples of LRM-specific options are partition, wall clock time, +scheduler options (e.g., #SBATCH arguments for Slurm), and worker +initialization commands (e.g., loading a conda environment). Cloud +parameters include access keys, instance type, and spot bid price + +Parsl currently supports the following providers: + +1. `parsl.providers.LocalProvider`: The provider allows you to run locally on your laptop or workstation. +2. `parsl.providers.SlurmProvider`: This provider allows you to schedule resources via the Slurm scheduler. +3. `parsl.providers.CondorProvider`: This provider allows you to schedule resources via the Condor scheduler. +4. `parsl.providers.GridEngineProvider`: This provider allows you to schedule resources via the GridEngine scheduler. +5. `parsl.providers.TorqueProvider`: This provider allows you to schedule resources via the Torque scheduler. +6. `parsl.providers.AWSProvider`: This provider allows you to provision and manage cloud nodes from Amazon Web Services. +7. `parsl.providers.GoogleCloudProvider`: This provider allows you to provision and manage cloud nodes from Google Cloud. +8. `parsl.providers.KubernetesProvider`: This provider allows you to provision and manage containers on a Kubernetes cluster. +9. `parsl.providers.LSFProvider`: This provider allows you to schedule resources via IBM's LSF scheduler. + + + +Executors +--------- + +Parsl programs vary widely in terms of their +execution requirements. Individual Apps may run for milliseconds +or days, and available parallelism can vary between none for +sequential programs to millions for "pleasingly parallel" programs. +Parsl executors, as the name suggests, execute Apps on one or more +target execution resources such as multi-core workstations, clouds, +or supercomputers. As it appears infeasible to implement a single +execution strategy that will meet so many diverse requirements on +such varied platforms, Parsl provides a modular executor interface +and a collection of executors that are tuned for common execution +patterns. + +Parsl executors extend the Executor class offered by Python's +concurrent.futures library, which allows Parsl to use +existing solutions in the Python Standard Library (e.g., ThreadPoolExecutor) +and from other packages such as Work Queue. Parsl +extends the concurrent.futures executor interface to support +additional capabilities such as automatic scaling of execution resources, +monitoring, deferred initialization, and methods to set working +directories. +All executors share a common execution kernel that is responsible +for deserializing the task (i.e., the App and its input arguments) +and executing the task in a sandboxed Python environment. + +Parsl currently supports the following executors: + +1. `parsl.executors.ThreadPoolExecutor`: This executor supports multi-thread execution on local resources. + +2. `parsl.executors.HighThroughputExecutor`: This executor implements hierarchical scheduling and batching using a pilot job model to deliver high throughput task execution on up to 4000 Nodes. + +3. `parsl.executors.WorkQueueExecutor`: This executor integrates `Work Queue `_ as an execution backend. Work Queue scales to tens of thousands of cores and implements reliable execution of tasks with dynamic resource sizing. + +4. `parsl.executors.taskvine.TaskVineExecutor`: This executor uses `TaskVine `_ as the execution backend. TaskVine scales up to tens of thousands of cores and actively uses local storage on compute nodes to offer a diverse array of performance-oriented features, including: smart caching and sharing common large files between tasks and compute nodes, reliable execution of tasks, dynamic resource sizing, automatic Python environment detection and sharing. +These executors cover a broad range of execution requirements. As with other Parsl components, there is a standard interface (ParslExecutor) that can be implemented to add support for other executors. + +.. note:: + Refer to :ref:`configuration-section` for information on how to configure these executors. + + +Launchers +--------- + +Many LRMs offer mechanisms for spawning applications across nodes +inside a single job and for specifying the +resources and task placement information needed to execute that +application at launch time. Common mechanisms include +`srun `_ (for Slurm), +`aprun `_ (for Crays), and `mpirun `_ (for MPI). +Thus, to run Parsl programs on such systems, we typically want first to +request a large number of nodes and then to *launch* "pilot job" or +**worker** processes using the system launchers. +Parsl's Launcher abstraction enables Parsl programs +to use these system-specific launcher systems to start workers across +cores and nodes. + +Parsl currently supports the following set of launchers: + +1. `parsl.launchers.SrunLauncher`: Srun based launcher for Slurm based systems. +2. `parsl.launchers.AprunLauncher`: Aprun based launcher for Crays. +3. `parsl.launchers.SrunMPILauncher`: Launcher for launching MPI applications with Srun. +4. `parsl.launchers.GnuParallelLauncher`: Launcher using GNU parallel to launch workers across nodes and cores. +5. `parsl.launchers.MpiExecLauncher`: Uses Mpiexec to launch. +6. `parsl.launchers.SimpleLauncher`: The launcher default to a single worker launch. +7. `parsl.launchers.SingleNodeLauncher`: This launcher launches ``workers_per_node`` count workers on a single node. + +Additionally, the launcher interface can be used to implement specialized behaviors +in custom environments (for example, to +launch node processes inside containers with customized environments). +For example, the following launcher uses Srun to launch ``worker-wrapper``, passing the +command to be run as parameters to ``worker-wrapper``. It is the responsibility of ``worker-wrapper`` +to launch the command it is given inside the appropriate environment. + +.. code:: python + + class MyShifterSRunLauncher: + def __init__(self): + self.srun_launcher = SrunLauncher() + + def __call__(self, command, tasks_per_node, nodes_per_block): + new_command="worker-wrapper {}".format(command) + return self.srun_launcher(new_command, tasks_per_node, nodes_per_block) + +Blocks +------ + +One challenge when making use of heterogeneous +execution resource types is the need to provide a uniform representation of +resources. Consider that single requests on clouds return individual +nodes, clusters and supercomputers provide batches of nodes, grids +provide cores, and workstations provide a single multicore node + +Parsl defines a resource abstraction called a *block* as the most basic unit +of resources to be acquired from a provider. A block contains one +or more nodes and maps to the different provider abstractions. In +a cluster, a block corresponds to a single allocation request to a +scheduler. In a cloud, a block corresponds to a single API request +for one or more instances. +Parsl can then execute *tasks* (instances of apps) +within and across (e.g., for MPI jobs) nodes within a block. +Blocks are also used as the basis for +elasticity on batch scheduling systems (see Elasticity below). +Three different examples of block configurations are shown below. + +1. A single block comprised of a node executing one task: + + .. image:: ../../images/N1_T1.png + :scale: 75% + +2. A single block with one node executing several tasks. This configuration is + most suitable for single threaded apps running on multicore target systems. + The number of tasks executed concurrently is proportional to the number of cores available on the system. + + .. image:: ../../images/N1_T4.png + :scale: 75% + +3. A block comprised of several nodes and executing several tasks, where a task can span multiple nodes. This configuration + is generally used by MPI applications. Starting a task requires using a specific + MPI launcher that is supported on the target system (e.g., aprun, srun, mpirun, mpiexec). + The `MPI Apps `_ documentation page describes how to configure Parsl for this case. + + .. image:: ../../images/N4_T2.png + +The configuration options for specifying the shape of each block are: + +1. ``workers_per_node``: Number of workers started per node, which corresponds to the number of tasks that can execute concurrently on a node. +2. ``nodes_per_block``: Number of nodes requested per block. + + + +Multi-executor +-------------- + +Parsl supports the use of one or more executors as specified in the configuration. +In this situation, individual apps may indicate which executors they are able to use. + +The common scenarios for this feature are: + +* A workflow has an initial simulation stage that runs on the compute heavy + nodes of an HPC system followed by an analysis and visualization stage that + is better suited for GPU nodes. +* A workflow follows a repeated fan-out, fan-in model where the long running + fan-out tasks are computed on a cluster and the quick fan-in computation is + better suited for execution using threads on a login node. +* A workflow includes apps that wait and evaluate the results of a + computation to determine whether the app should be relaunched. + Only apps running on threads may launch other apps. Often, simulations + have stochastic behavior and may terminate before completion. + In such cases, having a wrapper app that checks the exit code + and determines whether or not the app has completed successfully can + be used to automatically re-execute the app (possibly from a + checkpoint) until successful completion. + + +The following code snippet shows how apps can specify suitable executors in the app decorator. + +.. code-block:: python + + #(CPU heavy app) (CPU heavy app) (CPU heavy app) <--- Run on compute queue + # | | | + # (data) (data) (data) + # \ | / + # (Analysis and visualization phase) <--- Run on GPU node + + # A mock molecular dynamics simulation app + @bash_app(executors=["Theta.Phi"]) + def MD_Sim(arg, outputs=()): + return "MD_simulate {} -o {}".format(arg, outputs[0]) + + # Visualize results from the mock MD simulation app + @bash_app(executors=["Cooley.GPU"]) + def visualize(inputs=(), outputs=()): + bash_array = " ".join(inputs) + return "viz {} -o {}".format(bash_array, outputs[0]) + diff --git a/docs/userguide/configuration/heterogeneous.rst b/docs/userguide/configuration/heterogeneous.rst new file mode 100644 index 0000000000..f004f68fbf --- /dev/null +++ b/docs/userguide/configuration/heterogeneous.rst @@ -0,0 +1,106 @@ +Heterogeneous resources +----------------------- + +In some cases, it can be difficult to specify the resource requirements for running a workflow. +For example, if the compute nodes a site provides are not uniform, there is no "correct" resource configuration; +the amount of parallelism depends on which node (large or small) each job runs on. +In addition, the software and filesystem setup can vary from node to node. +A Condor cluster may not provide shared filesystem access at all, +and may include nodes with a variety of Python versions and available libraries. + +The :class:`parsl.executors.WorkQueueExecutor` provides several features to work with heterogeneous resources. +By default, Parsl only runs one app at a time on each worker node. +However, it is possible to specify the requirements for a particular app, +and Work Queue will automatically run as many parallel instances as possible on each node. +Work Queue automatically detects the amount of cores, memory, and other resources available on each execution node. +To activate this feature, add a resource specification to your apps. A resource specification is a dictionary with +the following three keys: ``cores`` (an integer corresponding to the number of cores required by the task), +``memory`` (an integer corresponding to the task's memory requirement in MB), and ``disk`` (an integer corresponding to +the task's disk requirement in MB), passed to an app via the special keyword argument ``parsl_resource_specification``. The specification can be set for all app invocations via a default, for example: + + .. code-block:: python + + @python_app + def compute(x, parsl_resource_specification={'cores': 1, 'memory': 1000, 'disk': 1000}): + return x*2 + + +or updated when the app is invoked: + + .. code-block:: python + + spec = {'cores': 1, 'memory': 500, 'disk': 500} + future = compute(x, parsl_resource_specification=spec) + +This ``parsl_resource_specification`` special keyword argument will inform Work Queue about the resources this app requires. +When placing instances of ``compute(x)``, Work Queue will run as many parallel instances as possible based on each worker node's available resources. + +If an app's resource requirements are not known in advance, +Work Queue has an auto-labeling feature that measures the actual resource usage of your apps and automatically chooses resource labels for you. +With auto-labeling, it is not necessary to provide ``parsl_resource_specification``; +Work Queue collects stats in the background and updates resource labels as your workflow runs. +To activate this feature, add the following flags to your executor config: + + .. code-block:: python + + config = Config( + executors=[ + WorkQueueExecutor( + # ...other options go here + autolabel=True, + autocategory=True + ) + ] + ) + +The ``autolabel`` flag tells Work Queue to automatically generate resource labels. +By default, these labels are shared across all apps in your workflow. +The ``autocategory`` flag puts each app into a different category, +so that Work Queue will choose separate resource requirements for each app. +This is important if e.g. some of your apps use a single core and some apps require multiple cores. +Unless you know that all apps have uniform resource requirements, +you should turn on ``autocategory`` when using ``autolabel``. + +The Work Queue executor can also help deal with sites that have non-uniform software environments across nodes. +Parsl assumes that the Parsl program and the compute nodes all use the same Python version. +In addition, any packages your apps import must be available on compute nodes. +If no shared filesystem is available or if node configuration varies, +this can lead to difficult-to-trace execution problems. + +If your Parsl program is running in a Conda environment, +the Work Queue executor can automatically scan the imports in your apps, +create a self-contained software package, +transfer the software package to worker nodes, +and run your code inside the packaged and uniform environment. +First, make sure that the Conda environment is active and you have the required packages installed (via either ``pip`` or ``conda``): + +- ``python`` +- ``parsl`` +- ``ndcctools`` +- ``conda-pack`` + +Then add the following to your config: + + .. code-block:: python + + config = Config( + executors=[ + WorkQueueExecutor( + # ...other options go here + pack=True + ) + ] + ) + +.. note:: + There will be a noticeable delay the first time Work Queue sees an app; + it is creating and packaging a complete Python environment. + This packaged environment is cached, so subsequent app invocations should be much faster. + +Using this approach, it is possible to run Parsl applications on nodes that don't have Python available at all. +The packaged environment includes a Python interpreter, +and Work Queue does not require Python to run. + +.. note:: + The automatic packaging feature only supports packages installed via ``pip`` or ``conda``. + Importing from other locations (e.g. via ``$PYTHONPATH``) or importing other modules in the same directory is not supported. \ No newline at end of file diff --git a/docs/userguide/aws_image.png b/docs/userguide/configuration/img/aws_image.png similarity index 100% rename from docs/userguide/aws_image.png rename to docs/userguide/configuration/img/aws_image.png diff --git a/docs/userguide/parsl_parallelism.gif b/docs/userguide/configuration/img/parsl_parallelism.gif similarity index 100% rename from docs/userguide/parsl_parallelism.gif rename to docs/userguide/configuration/img/parsl_parallelism.gif diff --git a/docs/userguide/parsl_scaling.gif b/docs/userguide/configuration/img/parsl_scaling.gif similarity index 100% rename from docs/userguide/parsl_scaling.gif rename to docs/userguide/configuration/img/parsl_scaling.gif diff --git a/docs/userguide/configuration/index.rst b/docs/userguide/configuration/index.rst new file mode 100644 index 0000000000..5db6aca918 --- /dev/null +++ b/docs/userguide/configuration/index.rst @@ -0,0 +1,88 @@ +.. _configuration-section: + +Configuring Parsl +================= + +Parsl separates program logic from execution configuration, enabling +programs to be developed entirely independently from their execution +environment. Configuration is described by a Python object (:class:`~parsl.config.Config`) +so that developers can +introspect permissible options, validate settings, and retrieve/edit +configurations dynamically during execution. A configuration object specifies +details of the provider, executors, allocation size, +queues, durations, and data management options. + +The following example shows a basic configuration object (:class:`~parsl.config.Config`) for the Frontera +supercomputer at TACC. +This config uses the `parsl.executors.HighThroughputExecutor` to submit +tasks from a login node. It requests an allocation of +128 nodes, deploying 1 worker for each of the 56 cores per node, from the normal partition. +To limit network connections to just the internal network the config specifies the address +used by the infiniband interface with ``address_by_interface('ib0')`` + +.. code-block:: python + + from parsl.config import Config + from parsl.providers import SlurmProvider + from parsl.executors import HighThroughputExecutor + from parsl.launchers import SrunLauncher + from parsl.addresses import address_by_interface + + config = Config( + executors=[ + HighThroughputExecutor( + label="frontera_htex", + address=address_by_interface('ib0'), + max_workers_per_node=56, + provider=SlurmProvider( + nodes_per_block=128, + init_blocks=1, + partition='normal', + launcher=SrunLauncher(), + ), + ) + ], + ) + + +Use the ``Config`` object to start Parsl's data flow kernel with the ``parsl.load`` method : + +.. code-block:: python + + from parsl.configs.htex_local import config + import parsl + + with parsl.load(config): + +The ``load`` statement can happen after Apps are defined but must occur before tasks are started. +Loading the Config object within context manager like ``with`` is recommended +for implicit cleaning of DFK on exiting the context manager. + +The :class:`~parsl.config.Config` object may not be used again after loaded. +Consider a configuration function if the application will shut down and re-launch the DFK. + +.. code-block:: python + + from parsl.config import Config + import parsl + + def make_config() -> Config: + return Config(...) + + with parsl.load(make_config()): + # Your workflow here + parsl.clear() # Stops Parsl + with parsl.load(make_config()): # Re-launches with a fresh configuration + # Your workflow here + + +.. toctree:: + :maxdepth: 2 + + execution + elasticity + pinning + data + heterogeneous + encryption + examples diff --git a/docs/userguide/configuration/pinning.rst b/docs/userguide/configuration/pinning.rst new file mode 100644 index 0000000000..d34f1f9030 --- /dev/null +++ b/docs/userguide/configuration/pinning.rst @@ -0,0 +1,125 @@ +Resource pinning +================ + +Resource pinning reduces contention between multiple workers using the same CPU cores or accelerators. + +Multi-Threaded Applications +--------------------------- + +Workflows which launch multiple workers on a single node which perform multi-threaded tasks (e.g., NumPy, Tensorflow operations) may run into thread contention issues. +Each worker may try to use the same hardware threads, which leads to performance penalties. +Use the ``cpu_affinity`` feature of the :class:`~parsl.executors.HighThroughputExecutor` to assign workers to specific threads. Users can pin threads to +workers either with a strategy method or an explicit list. + +The strategy methods will auto assign all detected hardware threads to workers. +Allowed strategies that can be assigned to ``cpu_affinity`` are ``block``, ``block-reverse``, and ``alternating``. +The ``block`` method pins threads to workers in sequential order (ex: 4 threads are grouped (0, 1) and (2, 3) on two workers); +``block-reverse`` pins threads in reverse sequential order (ex: (3, 2) and (1, 0)); and ``alternating`` alternates threads among workers (ex: (0, 2) and (1, 3)). + +Select the best blocking strategy for processor's cache hierarchy (choose ``alternating`` if in doubt) to ensure workers to not compete for cores. + +.. code-block:: python + + local_config = Config( + executors=[ + HighThroughputExecutor( + label="htex_Local", + worker_debug=True, + cpu_affinity='alternating', + provider=LocalProvider( + init_blocks=1, + max_blocks=1, + ), + ) + ], + strategy='none', + ) + +Users can also use ``cpu_affinity`` to assign explicitly threads to workers with a string that has the format of +``cpu_affinity="list:::"``. + +Each worker's threads can be specified as a comma separated list or a hyphenated range: +``thread1,thread2,thread3`` +or +``thread_start-thread_end``. + +An example for 12 workers on a node with 208 threads is: + +.. code-block:: python + + cpu_affinity="list:0-7,104-111:8-15,112-119:16-23,120-127:24-31,128-135:32-39,136-143:40-47,144-151:52-59,156-163:60-67,164-171:68-75,172-179:76-83,180-187:84-91,188-195:92-99,196-203" + +This example assigns 16 threads each to 12 workers. Note that in this example there are threads that are skipped. +If a thread is not explicitly assigned to a worker, it will be left idle. +The number of thread "ranks" (colon separated thread lists/ranges) must match the total number of workers on the node; otherwise an exception will be raised. + + + +Thread affinity is accomplished in two ways. +Each worker first sets the affinity for the Python process using `the affinity mask `_, +which may not be available on all operating systems. +It then sets environment variables to control +`OpenMP thread affinity `_ +so that any subprocesses launched by a worker which use OpenMP know which processors are valid. +These include ``OMP_NUM_THREADS``, ``GOMP_COMP_AFFINITY``, and ``KMP_THREAD_AFFINITY``. + +Accelerators +------------ + +Many modern clusters provide multiple accelerators per compute node, yet many applications are best suited to using a +single accelerator per task. Parsl supports pinning each worker to different accelerators using +``available_accelerators`` option of the :class:`~parsl.executors.HighThroughputExecutor`. Provide either the number of +executors (Parsl will assume they are named in integers starting from zero) or a list of the names of the accelerators +available on the node. Parsl will limit the number of workers it launches to the number of accelerators specified, +in other words, you cannot have more workers per node than there are accelerators. By default, Parsl will launch +as many workers as the accelerators specified via ``available_accelerators``. + +.. code-block:: python + + local_config = Config( + executors=[ + HighThroughputExecutor( + label="htex_Local", + worker_debug=True, + available_accelerators=2, + provider=LocalProvider( + init_blocks=1, + max_blocks=1, + ), + ) + ], + strategy='none', + ) + +It is possible to bind multiple/specific accelerators to each worker by specifying a list of comma separated strings +each specifying accelerators. In the context of binding to NVIDIA GPUs, this works by setting ``CUDA_VISIBLE_DEVICES`` +on each worker to a specific string in the list supplied to ``available_accelerators``. + +Here's an example: + +.. code-block:: python + + # The following config is trimmed for clarity + local_config = Config( + executors=[ + HighThroughputExecutor( + # Starts 2 workers per node, each bound to 2 GPUs + available_accelerators=["0,1", "2,3"], + + # Start a single worker bound to all 4 GPUs + # available_accelerators=["0,1,2,3"] + ) + ], + ) + +GPU Oversubscription +"""""""""""""""""""" + +For hardware that uses Nvidia devices, Parsl allows for the oversubscription of workers to GPUS. This is intended to +make use of Nvidia's `Multi-Process Service (MPS) `_ available on many of their +GPUs that allows users to run multiple concurrent processes on a single GPU. The user needs to set in the +``worker_init`` commands to start MPS on every node in the block (this is machine dependent). The +``available_accelerators`` option should then be set to the total number of GPU partitions run on a single node in the +block. For example, for a node with 4 Nvidia GPUs, to create 8 workers per GPU, set ``available_accelerators=32``. +GPUs will be assigned to workers in ascending order in contiguous blocks. In the example, workers 0-7 will be placed +on GPU 0, workers 8-15 on GPU 1, workers 16-23 on GPU 2, and workers 24-31 on GPU 3. diff --git a/docs/userguide/configuring.rst b/docs/userguide/configuring.rst index 88d4456a26..1b0f2be022 100644 --- a/docs/userguide/configuring.rst +++ b/docs/userguide/configuring.rst @@ -1,658 +1,9 @@ -.. _configuration-section: +:orphan: -Configuration -============= +.. meta:: + :content http-equiv="refresh": 0;url=configuration/index.html -Parsl separates program logic from execution configuration, enabling -programs to be developed entirely independently from their execution -environment. Configuration is described by a Python object (:class:`~parsl.config.Config`) -so that developers can -introspect permissible options, validate settings, and retrieve/edit -configurations dynamically during execution. A configuration object specifies -details of the provider, executors, allocation size, -queues, durations, and data management options. - -The following example shows a basic configuration object (:class:`~parsl.config.Config`) for the Frontera -supercomputer at TACC. -This config uses the `parsl.executors.HighThroughputExecutor` to submit -tasks from a login node. It requests an allocation of -128 nodes, deploying 1 worker for each of the 56 cores per node, from the normal partition. -To limit network connections to just the internal network the config specifies the address -used by the infiniband interface with ``address_by_interface('ib0')`` - -.. code-block:: python - - from parsl.config import Config - from parsl.providers import SlurmProvider - from parsl.executors import HighThroughputExecutor - from parsl.launchers import SrunLauncher - from parsl.addresses import address_by_interface - - config = Config( - executors=[ - HighThroughputExecutor( - label="frontera_htex", - address=address_by_interface('ib0'), - max_workers_per_node=56, - provider=SlurmProvider( - nodes_per_block=128, - init_blocks=1, - partition='normal', - launcher=SrunLauncher(), - ), - ) - ], - ) - -.. contents:: Configuration How-To and Examples: - - -Creating and Using Config Objects ---------------------------------- - -:class:`~parsl.config.Config` objects are loaded to define the "Data Flow Kernel" (DFK) that will manage tasks. -All Parsl applications start by creating or importing a configuration then calling the load function. - -.. code-block:: python - - from parsl.configs.htex_local import config - import parsl - - with parsl.load(config): - -The ``load`` statement can happen after Apps are defined but must occur before tasks are started. -Loading the Config object within context manager like ``with`` is recommended -for implicit cleaning of DFK on exiting the context manager - -The :class:`~parsl.config.Config` object may not be used again after loaded. -Consider a configuration function if the application will shut down and re-launch the DFK. - -.. code-block:: python - - from parsl.config import Config - import parsl - - def make_config() -> Config: - return Config(...) - - with parsl.load(make_config()): - # Your workflow here - parsl.clear() # Stops Parsl - with parsl.load(make_config()): # Re-launches with a fresh configuration - # Your workflow here - - -How to Configure ----------------- - -.. note:: - All configuration examples below must be customized for the user's - allocation, Python environment, file system, etc. - - -The configuration specifies what, and how, resources are to be used for executing -the Parsl program and its apps. -It is important to carefully consider the needs of the Parsl program and its apps, -and the characteristics of the compute resources, to determine an ideal configuration. -Aspects to consider include: -1) where the Parsl apps will execute; -2) how many nodes will be used to execute the apps, and how long the apps will run; -3) should Parsl request multiple nodes in an individual scheduler job; and -4) where will the main Parsl program run and how will it communicate with the apps. - -Stepping through the following question should help formulate a suitable configuration object. - -1. Where should apps be executed? - -+---------------------+-----------------------------------------------+----------------------------------------+ -| Target | Executor | Provider | -+=====================+===============================================+========================================+ -| Laptop/Workstation | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.LocalProvider` | -| | * `parsl.executors.ThreadPoolExecutor` | | -| | * `parsl.executors.WorkQueueExecutor` | | -| | * `parsl.executors.taskvine.TaskVineExecutor` | | -+---------------------+-----------------------------------------------+----------------------------------------+ -| Amazon Web Services | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.AWSProvider` | -+---------------------+-----------------------------------------------+----------------------------------------+ -| Google Cloud | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.GoogleCloudProvider` | -+---------------------+-----------------------------------------------+----------------------------------------+ -| Slurm based system | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.SlurmProvider` | -| | * `parsl.executors.WorkQueueExecutor` | | -| | * `parsl.executors.taskvine.TaskVineExecutor` | | -+---------------------+-----------------------------------------------+----------------------------------------+ -| Torque/PBS based | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.TorqueProvider` | -| system | * `parsl.executors.WorkQueueExecutor` | | -+---------------------+-----------------------------------------------+----------------------------------------+ -| GridEngine based | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.GridEngineProvider` | -| system | * `parsl.executors.WorkQueueExecutor` | | -+---------------------+-----------------------------------------------+----------------------------------------+ -| Condor based | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.CondorProvider` | -| cluster or grid | * `parsl.executors.WorkQueueExecutor` | | -| | * `parsl.executors.taskvine.TaskVineExecutor` | | -+---------------------+-----------------------------------------------+----------------------------------------+ -| Kubernetes cluster | * `parsl.executors.HighThroughputExecutor` | `parsl.providers.KubernetesProvider` | -+---------------------+-----------------------------------------------+----------------------------------------+ - - -2. How many nodes will be used to execute the apps? What task durations are necessary to achieve good performance? - - -+--------------------------------------------+----------------------+-------------------------------------+ -| Executor | Number of Nodes [*]_ | Task duration for good performance | -+============================================+======================+=====================================+ -| `parsl.executors.ThreadPoolExecutor` | 1 (Only local) | Any | -+--------------------------------------------+----------------------+-------------------------------------+ -| `parsl.executors.HighThroughputExecutor` | <=2000 | Task duration(s)/#nodes >= 0.01 | -| | | longer tasks needed at higher scale | -+--------------------------------------------+----------------------+-------------------------------------+ -| `parsl.executors.WorkQueueExecutor` | <=1000 [*]_ | 10s+ | -+--------------------------------------------+----------------------+-------------------------------------+ -| `parsl.executors.taskvine.TaskVineExecutor`| <=1000 [*]_ | 10s+ | -+--------------------------------------------+----------------------+-------------------------------------+ - - -.. [*] Assuming 32 workers per node. If there are fewer workers launched - per node, a larger number of nodes could be supported. - -.. [*] The maximum number of nodes tested for the `parsl.executors.WorkQueueExecutor` is 10,000 GPU cores and - 20,000 CPU cores. - -.. [*] The maximum number of nodes tested for the `parsl.executors.taskvine.TaskVineExecutor` is - 10,000 GPU cores and 20,000 CPU cores. - -3. Should Parsl request multiple nodes in an individual scheduler job? -(Here the term block is equivalent to a single scheduler job.) - -+--------------------------------------------------------------------------------------------+ -| ``nodes_per_block = 1`` | -+---------------------+--------------------------+-------------------------------------------+ -| Provider | Executor choice | Suitable Launchers | -+=====================+==========================+===========================================+ -| Systems that don't | Any | * `parsl.launchers.SingleNodeLauncher` | -| use Aprun | | * `parsl.launchers.SimpleLauncher` | -+---------------------+--------------------------+-------------------------------------------+ -| Aprun based systems | Any | * `parsl.launchers.AprunLauncher` | -+---------------------+--------------------------+-------------------------------------------+ - -+---------------------------------------------------------------------------------------------------------------------+ -| ``nodes_per_block > 1`` | -+-------------------------------------+--------------------------+----------------------------------------------------+ -| Provider | Executor choice | Suitable Launchers | -+=====================================+==========================+====================================================+ -| `parsl.providers.TorqueProvider` | Any | * `parsl.launchers.AprunLauncher` | -| | | * `parsl.launchers.MpiExecLauncher` | -+-------------------------------------+--------------------------+----------------------------------------------------+ -| `parsl.providers.SlurmProvider` | Any | * `parsl.launchers.SrunLauncher` if native slurm | -| | | * `parsl.launchers.AprunLauncher`, otherwise | -+-------------------------------------+--------------------------+----------------------------------------------------+ - -.. note:: If using a Cray system, you most likely need to use the `parsl.launchers.AprunLauncher` to launch workers unless you - are on a **native Slurm** system like :ref:`configuring_nersc_cori` - - -Heterogeneous Resources ------------------------ - -In some cases, it can be difficult to specify the resource requirements for running a workflow. -For example, if the compute nodes a site provides are not uniform, there is no "correct" resource configuration; -the amount of parallelism depends on which node (large or small) each job runs on. -In addition, the software and filesystem setup can vary from node to node. -A Condor cluster may not provide shared filesystem access at all, -and may include nodes with a variety of Python versions and available libraries. - -The `parsl.executors.WorkQueueExecutor` provides several features to work with heterogeneous resources. -By default, Parsl only runs one app at a time on each worker node. -However, it is possible to specify the requirements for a particular app, -and Work Queue will automatically run as many parallel instances as possible on each node. -Work Queue automatically detects the amount of cores, memory, and other resources available on each execution node. -To activate this feature, add a resource specification to your apps. A resource specification is a dictionary with -the following three keys: ``cores`` (an integer corresponding to the number of cores required by the task), -``memory`` (an integer corresponding to the task's memory requirement in MB), and ``disk`` (an integer corresponding to -the task's disk requirement in MB), passed to an app via the special keyword argument ``parsl_resource_specification``. The specification can be set for all app invocations via a default, for example: - - .. code-block:: python - - @python_app - def compute(x, parsl_resource_specification={'cores': 1, 'memory': 1000, 'disk': 1000}): - return x*2 - - -or updated when the app is invoked: - - .. code-block:: python - - spec = {'cores': 1, 'memory': 500, 'disk': 500} - future = compute(x, parsl_resource_specification=spec) - -This ``parsl_resource_specification`` special keyword argument will inform Work Queue about the resources this app requires. -When placing instances of ``compute(x)``, Work Queue will run as many parallel instances as possible based on each worker node's available resources. - -If an app's resource requirements are not known in advance, -Work Queue has an auto-labeling feature that measures the actual resource usage of your apps and automatically chooses resource labels for you. -With auto-labeling, it is not necessary to provide ``parsl_resource_specification``; -Work Queue collects stats in the background and updates resource labels as your workflow runs. -To activate this feature, add the following flags to your executor config: - - .. code-block:: python - - config = Config( - executors=[ - WorkQueueExecutor( - # ...other options go here - autolabel=True, - autocategory=True - ) - ] - ) - -The ``autolabel`` flag tells Work Queue to automatically generate resource labels. -By default, these labels are shared across all apps in your workflow. -The ``autocategory`` flag puts each app into a different category, -so that Work Queue will choose separate resource requirements for each app. -This is important if e.g. some of your apps use a single core and some apps require multiple cores. -Unless you know that all apps have uniform resource requirements, -you should turn on ``autocategory`` when using ``autolabel``. - -The Work Queue executor can also help deal with sites that have non-uniform software environments across nodes. -Parsl assumes that the Parsl program and the compute nodes all use the same Python version. -In addition, any packages your apps import must be available on compute nodes. -If no shared filesystem is available or if node configuration varies, -this can lead to difficult-to-trace execution problems. - -If your Parsl program is running in a Conda environment, -the Work Queue executor can automatically scan the imports in your apps, -create a self-contained software package, -transfer the software package to worker nodes, -and run your code inside the packaged and uniform environment. -First, make sure that the Conda environment is active and you have the required packages installed (via either ``pip`` or ``conda``): - -- ``python`` -- ``parsl`` -- ``ndcctools`` -- ``conda-pack`` - -Then add the following to your config: - - .. code-block:: python - - config = Config( - executors=[ - WorkQueueExecutor( - # ...other options go here - pack=True - ) - ] - ) - -.. note:: - There will be a noticeable delay the first time Work Queue sees an app; - it is creating and packaging a complete Python environment. - This packaged environment is cached, so subsequent app invocations should be much faster. - -Using this approach, it is possible to run Parsl applications on nodes that don't have Python available at all. -The packaged environment includes a Python interpreter, -and Work Queue does not require Python to run. - -.. note:: - The automatic packaging feature only supports packages installed via ``pip`` or ``conda``. - Importing from other locations (e.g. via ``$PYTHONPATH``) or importing other modules in the same directory is not supported. - - -Accelerators ------------- - -Many modern clusters provide multiple accelerators per compute note, yet many applications are best suited to using a -single accelerator per task. Parsl supports pinning each worker to different accelerators using -``available_accelerators`` option of the :class:`~parsl.executors.HighThroughputExecutor`. Provide either the number of -executors (Parsl will assume they are named in integers starting from zero) or a list of the names of the accelerators -available on the node. Parsl will limit the number of workers it launches to the number of accelerators specified, -in other words, you cannot have more workers per node than there are accelerators. By default, Parsl will launch -as many workers as the accelerators specified via ``available_accelerators``. - -.. code-block:: python - - local_config = Config( - executors=[ - HighThroughputExecutor( - label="htex_Local", - worker_debug=True, - available_accelerators=2, - provider=LocalProvider( - init_blocks=1, - max_blocks=1, - ), - ) - ], - strategy='none', - ) - -It is possible to bind multiple/specific accelerators to each worker by specifying a list of comma separated strings -each specifying accelerators. In the context of binding to NVIDIA GPUs, this works by setting ``CUDA_VISIBLE_DEVICES`` -on each worker to a specific string in the list supplied to ``available_accelerators``. - -Here's an example: - -.. code-block:: python - - # The following config is trimmed for clarity - local_config = Config( - executors=[ - HighThroughputExecutor( - # Starts 2 workers per node, each bound to 2 GPUs - available_accelerators=["0,1", "2,3"], - - # Start a single worker bound to all 4 GPUs - # available_accelerators=["0,1,2,3"] - ) - ], - ) - -GPU Oversubscription -"""""""""""""""""""" - -For hardware that uses Nvidia devices, Parsl allows for the oversubscription of workers to GPUS. This is intended to -make use of Nvidia's `Multi-Process Service (MPS) `_ available on many of their -GPUs that allows users to run multiple concurrent processes on a single GPU. The user needs to set in the -``worker_init`` commands to start MPS on every node in the block (this is machine dependent). The -``available_accelerators`` option should then be set to the total number of GPU partitions run on a single node in the -block. For example, for a node with 4 Nvidia GPUs, to create 8 workers per GPU, set ``available_accelerators=32``. -GPUs will be assigned to workers in ascending order in contiguous blocks. In the example, workers 0-7 will be placed -on GPU 0, workers 8-15 on GPU 1, workers 16-23 on GPU 2, and workers 24-31 on GPU 3. - -Multi-Threaded Applications ---------------------------- - -Workflows which launch multiple workers on a single node which perform multi-threaded tasks (e.g., NumPy, Tensorflow operations) may run into thread contention issues. -Each worker may try to use the same hardware threads, which leads to performance penalties. -Use the ``cpu_affinity`` feature of the :class:`~parsl.executors.HighThroughputExecutor` to assign workers to specific threads. Users can pin threads to -workers either with a strategy method or an explicit list. - -The strategy methods will auto assign all detected hardware threads to workers. -Allowed strategies that can be assigned to ``cpu_affinity`` are ``block``, ``block-reverse``, and ``alternating``. -The ``block`` method pins threads to workers in sequential order (ex: 4 threads are grouped (0, 1) and (2, 3) on two workers); -``block-reverse`` pins threads in reverse sequential order (ex: (3, 2) and (1, 0)); and ``alternating`` alternates threads among workers (ex: (0, 2) and (1, 3)). - -Select the best blocking strategy for processor's cache hierarchy (choose ``alternating`` if in doubt) to ensure workers to not compete for cores. - -.. code-block:: python - - local_config = Config( - executors=[ - HighThroughputExecutor( - label="htex_Local", - worker_debug=True, - cpu_affinity='alternating', - provider=LocalProvider( - init_blocks=1, - max_blocks=1, - ), - ) - ], - strategy='none', - ) - -Users can also use ``cpu_affinity`` to assign explicitly threads to workers with a string that has the format of -``cpu_affinity="list:::"``. - -Each worker's threads can be specified as a comma separated list or a hyphenated range: -``thread1,thread2,thread3`` -or -``thread_start-thread_end``. - -An example for 12 workers on a node with 208 threads is: - -.. code-block:: python - - cpu_affinity="list:0-7,104-111:8-15,112-119:16-23,120-127:24-31,128-135:32-39,136-143:40-47,144-151:52-59,156-163:60-67,164-171:68-75,172-179:76-83,180-187:84-91,188-195:92-99,196-203" - -This example assigns 16 threads each to 12 workers. Note that in this example there are threads that are skipped. -If a thread is not explicitly assigned to a worker, it will be left idle. -The number of thread "ranks" (colon separated thread lists/ranges) must match the total number of workers on the node; otherwise an exception will be raised. - - - -Thread affinity is accomplished in two ways. -Each worker first sets the affinity for the Python process using `the affinity mask `_, -which may not be available on all operating systems. -It then sets environment variables to control -`OpenMP thread affinity `_ -so that any subprocesses launched by a worker which use OpenMP know which processors are valid. -These include ``OMP_NUM_THREADS``, ``GOMP_COMP_AFFINITY``, and ``KMP_THREAD_AFFINITY``. - -Ad-Hoc Clusters ---------------- - -Parsl's support of ad-hoc clusters of compute nodes without a scheduler -is deprecated. - -See -`issue #3515 `_ -for further discussion. - -Amazon Web Services -------------------- - -.. image:: ./aws_image.png - -.. note:: - To use AWS with Parsl, install Parsl with AWS dependencies via ``python3 -m pip install 'parsl[aws]'`` - -Amazon Web Services is a commercial cloud service which allows users to rent a range of computers and other computing services. -The following snippet shows how Parsl can be configured to provision nodes from the Elastic Compute Cloud (EC2) service. -The first time this configuration is used, Parsl will configure a Virtual Private Cloud and other networking and security infrastructure that will be -re-used in subsequent executions. The configuration uses the `parsl.providers.AWSProvider` to connect to AWS. - -.. literalinclude:: ../../parsl/configs/ec2.py - - -ASPIRE 1 (NSCC) ---------------- - -.. image:: https://www.nscc.sg/wp-content/uploads/2017/04/ASPIRE1Img.png - -The following snippet shows an example configuration for accessing NSCC's **ASPIRE 1** supercomputer. This example uses the `parsl.executors.HighThroughputExecutor` executor and connects to ASPIRE1's PBSPro scheduler. It also shows how ``scheduler_options`` parameter could be used for scheduling array jobs in PBSPro. - -.. literalinclude:: ../../parsl/configs/ASPIRE1.py - - - - -Illinois Campus Cluster (UIUC) ------------------------------- - -.. image:: https://campuscluster.illinois.edu/wp-content/uploads/2018/02/ND2_3633-sm.jpg - -The following snippet shows an example configuration for executing on the Illinois Campus Cluster. -The configuration assumes the user is running on a login node and uses the `parsl.providers.SlurmProvider` to interface -with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. - -.. literalinclude:: ../../parsl/configs/illinoiscluster.py - -Bridges (PSC) -------------- - -.. image:: https://insidehpc.com/wp-content/uploads/2016/08/Bridges_FB1b.jpg - -The following snippet shows an example configuration for executing on the Bridges supercomputer at the Pittsburgh Supercomputing Center. -The configuration assumes the user is running on a login node and uses the `parsl.providers.SlurmProvider` to interface -with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. - -.. literalinclude:: ../../parsl/configs/bridges.py - - - -CC-IN2P3 +Redirect -------- -.. image:: https://cc.in2p3.fr/wp-content/uploads/2017/03/bandeau_accueil.jpg - -The snippet below shows an example configuration for executing from a login node on IN2P3's Computing Centre. -The configuration uses the `parsl.providers.LocalProvider` to run on a login node primarily to avoid GSISSH, which Parsl does not support. -This system uses Grid Engine which Parsl interfaces with using the `parsl.providers.GridEngineProvider`. - -.. literalinclude:: ../../parsl/configs/cc_in2p3.py - - -CCL (Notre Dame, TaskVine) --------------------------- - -.. image:: https://ccl.cse.nd.edu/software/taskvine/taskvine-logo.png - -To utilize TaskVine with Parsl, please install the full CCTools software package within an appropriate Anaconda or Miniconda environment -(instructions for installing Miniconda can be found `in the Conda install guide `_): - -.. code-block:: bash - - $ conda create -y --name python= conda-pack - $ conda activate - $ conda install -y -c conda-forge ndcctools parsl - -This creates a Conda environment on your machine with all the necessary tools and setup needed to utilize TaskVine with the Parsl library. - -The following snippet shows an example configuration for using the Parsl/TaskVine executor to run applications on the local machine. -This examples uses the `parsl.executors.taskvine.TaskVineExecutor` to schedule tasks, and a local worker will be started automatically. -For more information on using TaskVine, including configurations for remote execution, visit the -`TaskVine/Parsl documentation online `_. - -.. literalinclude:: ../../parsl/configs/vineex_local.py - -TaskVine's predecessor, WorkQueue, may continue to be used with Parsl. -For more information on using WorkQueue visit the `CCTools documentation online `_. - -Expanse (SDSC) --------------- - -.. image:: https://www.hpcwire.com/wp-content/uploads/2019/07/SDSC-Expanse-graphic-cropped.jpg - -The following snippet shows an example configuration for executing remotely on San Diego Supercomputer -Center's **Expanse** supercomputer. The example is designed to be executed on the login nodes, using the -`parsl.providers.SlurmProvider` to interface with the Slurm scheduler used by Comet and the `parsl.launchers.SrunLauncher` to launch workers. - -.. literalinclude:: ../../parsl/configs/expanse.py - - -Improv (Argonne LCRC) ---------------------- - -.. image:: https://www.lcrc.anl.gov/sites/default/files/styles/965_wide/public/2023-12/20231214_114057.jpg?itok=A-Rz5pP9 - -**Improv** is a PBS Pro based supercomputer at Argonne's Laboratory Computing Resource -Center (LCRC). The following snippet is an example configuration that uses `parsl.providers.PBSProProvider` -and `parsl.launchers.MpiRunLauncher` to run on multinode jobs. - -.. literalinclude:: ../../parsl/configs/improv.py - - -.. _configuring_nersc_cori: - -Perlmutter (NERSC) ------------------- - -NERSC provides documentation on `how to use Parsl on Perlmutter `_. -Perlmutter is a Slurm based HPC system and parsl uses `parsl.providers.SlurmProvider` with `parsl.launchers.SrunLauncher` -to launch tasks onto this machine. - - -Frontera (TACC) ---------------- - -.. image:: https://frontera-portal.tacc.utexas.edu/media/filer_public/2c/fb/2cfbf6ab-818d-42c8-b4d5-9b39eb9d0a05/frontera-banner-home.jpg - -Deployed in June 2019, Frontera is the 5th most powerful supercomputer in the world. Frontera replaces the NSF Blue Waters system at NCSA -and is the first deployment in the National Science Foundation's petascale computing program. The configuration below assumes that the user is -running on a login node and uses the `parsl.providers.SlurmProvider` to interface with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. - -.. literalinclude:: ../../parsl/configs/frontera.py - - -Kubernetes Clusters -------------------- - -.. image:: https://d1.awsstatic.com/PAC/kuberneteslogo.eabc6359f48c8e30b7a138c18177f3fd39338e05.png - -Kubernetes is an open-source system for container management, such as automating deployment and scaling of containers. -The snippet below shows an example configuration for deploying pods as workers on a Kubernetes cluster. -The KubernetesProvider exploits the Python Kubernetes API, which assumes that you have kube config in ``~/.kube/config``. - -.. literalinclude:: ../../parsl/configs/kubernetes.py - - -Midway (RCC, UChicago) ----------------------- - -.. image:: https://rcc.uchicago.edu/sites/rcc.uchicago.edu/files/styles/slideshow-image/public/uploads/images/slideshows/20140430_RCC_8978.jpg?itok=BmRuJ-wq - -This Midway cluster is a campus cluster hosted by the Research Computing Center at the University of Chicago. -The snippet below shows an example configuration for executing remotely on Midway. -The configuration assumes the user is running on a login node and uses the `parsl.providers.SlurmProvider` to interface -with the scheduler, and uses the `parsl.launchers.SrunLauncher` to launch workers. - -.. literalinclude:: ../../parsl/configs/midway.py - - -Open Science Grid ------------------ - -.. image:: https://www.renci.org/wp-content/uploads/2008/10/osg_logo.png - -The Open Science Grid (OSG) is a national, distributed computing Grid spanning over 100 individual sites to provide tens of thousands of CPU cores. -The snippet below shows an example configuration for executing remotely on OSG. You will need to have a valid project name on the OSG. -The configuration uses the `parsl.providers.CondorProvider` to interface with the scheduler. - -.. literalinclude:: ../../parsl/configs/osg.py - - -Polaris (ALCF) --------------- - -.. image:: https://www.alcf.anl.gov/sites/default/files/styles/965x543/public/2022-07/33181D_086_ALCF%20Polaris%20Crop.jpg?itok=HVAHsZtt - :width: 75% - -ALCF provides documentation on `how to use Parsl on Polaris `_. -Polaris uses `parsl.providers.PBSProProvider` and `parsl.launchers.MpiExecLauncher` to launch tasks onto the HPC system. - - - -Stampede2 (TACC) ----------------- - -.. image:: https://www.tacc.utexas.edu/documents/1084364/1413880/stampede2-0717.jpg/ - -The following snippet shows an example configuration for accessing TACC's **Stampede2** supercomputer. This example uses theHighThroughput executor and connects to Stampede2's Slurm scheduler. - -.. literalinclude:: ../../parsl/configs/stampede2.py - - -Summit (ORNL) -------------- - -.. image:: https://www.olcf.ornl.gov/wp-content/uploads/2018/06/Summit_Exaop-1500x844.jpg - -The following snippet shows an example configuration for executing from the login node on Summit, the leadership class supercomputer hosted at the Oak Ridge National Laboratory. -The example uses the `parsl.providers.LSFProvider` to provision compute nodes from the LSF cluster scheduler and the `parsl.launchers.JsrunLauncher` to launch workers across the compute nodes. - -.. literalinclude:: ../../parsl/configs/summit.py - - -TOSS3 (LLNL) ------------- - -.. image:: https://hpc.llnl.gov/sites/default/files/Magma--2020-LLNL.jpg - -The following snippet shows an example configuration for executing on one of LLNL's **TOSS3** -machines, such as Quartz, Ruby, Topaz, Jade, or Magma. This example uses the `parsl.executors.FluxExecutor` -and connects to Slurm using the `parsl.providers.SlurmProvider`. This configuration assumes that the script -is being executed on the login nodes of one of the machines. - -.. literalinclude:: ../../parsl/configs/toss3_llnl.py - - -Further help ------------- - -For help constructing a configuration, you can click on class names such as :class:`~parsl.config.Config` or :class:`~parsl.executors.HighThroughputExecutor` to see the associated class documentation. The same documentation can be accessed interactively at the python command line via, for example: - -.. code-block:: python - - from parsl.config import Config - help(Config) +This page has been `moved `_ diff --git a/docs/userguide/data.rst b/docs/userguide/data.rst index 9350a6d96f..4626f0ed38 100644 --- a/docs/userguide/data.rst +++ b/docs/userguide/data.rst @@ -1,445 +1,9 @@ -.. _label-data: +:orphan: -Passing Python objects -====================== +.. meta:: + :content http-equiv="refresh": 0;url=configuration/data.html -Parsl apps can communicate via standard Python function parameter passing -and return statements. The following example shows how a Python string -can be passed to, and returned from, a Parsl app. +Redirect +-------- -.. code-block:: python - - @python_app - def example(name): - return 'hello {0}'.format(name) - - r = example('bob') - print(r.result()) - -Parsl uses the dill and pickle libraries to serialize Python objects -into a sequence of bytes that can be passed over a network from the submitting -machine to executing workers. - -Thus, Parsl apps can receive and return standard Python data types -such as booleans, integers, tuples, lists, and dictionaries. However, not -all objects can be serialized with these methods (e.g., closures, generators, -and system objects), and so those objects cannot be used with all executors. - -Parsl will raise a `SerializationError` if it encounters an object that it cannot -serialize. This applies to objects passed as arguments to an app, as well as objects -returned from an app. See :ref:`label_serialization_error`. - - -Staging data files -================== - -Parsl apps can take and return data files. A file may be passed as an input -argument to an app, or returned from an app after execution. Parsl -provides support to automatically transfer (stage) files between -the main Parsl program, worker nodes, and external data storage systems. - -Input files can be passed as regular arguments, or a list of them may be -specified in the special ``inputs`` keyword argument to an app invocation. - -Inside an app, the ``filepath`` attribute of a `File` can be read to determine -where on the execution-side file system the input file has been placed. - -Output `File` objects must also be passed in at app invocation, through the -outputs parameter. In this case, the `File` object specifies where Parsl -should place output after execution. - -Inside an app, the ``filepath`` attribute of an output -`File` provides the path at which the corresponding output file should be -placed so that Parsl can find it after execution. - -If the output from an app is to be used as the input to a subsequent app, -then a `DataFuture` that represents whether the output file has been created -must be extracted from the first app's AppFuture, and that must be passed -to the second app. This causes app -executions to be properly ordered, in the same way that passing AppFutures -to subsequent apps causes execution ordering based on an app returning. - -In a Parsl program, file handling is split into two pieces: files are named in an -execution-location independent manner using :py:class:`~parsl.data_provider.files.File` -objects, and executors are configured to stage those files in to and out of -execution locations using instances of the :py:class:`~parsl.data_provider.staging.Staging` -interface. - - -Parsl files ------------ - -Parsl uses a custom :py:class:`~parsl.data_provider.files.File` to provide a -location-independent way of referencing and accessing files. -Parsl files are defined by specifying the URL *scheme* and a path to the file. -Thus a file may represent an absolute path on the submit-side file system -or a URL to an external file. - -The scheme defines the protocol via which the file may be accessed. -Parsl supports the following schemes: file, ftp, http, https, and globus. -If no scheme is specified Parsl will default to the file scheme. - -The following example shows creation of two files with different -schemes: a locally-accessible data.txt file and an HTTPS-accessible -README file. - -.. code-block:: python - - File('file://home/parsl/data.txt') - File('https://github.com/Parsl/parsl/blob/master/README.rst') - - -Parsl automatically translates the file's location relative to the -environment in which it is accessed (e.g., the Parsl program or an app). -The following example shows how a file can be accessed in the app -irrespective of where that app executes. - -.. code-block:: python - - @python_app - def print_file(inputs=()): - with open(inputs[0].filepath, 'r') as inp: - content = inp.read() - return(content) - - # create an remote Parsl file - f = File('https://github.com/Parsl/parsl/blob/master/README.rst') - - # call the print_file app with the Parsl file - r = print_file(inputs=[f]) - r.result() - -As described below, the method by which this files are transferred -depends on the scheme and the staging providers specified in the Parsl -configuration. - -Staging providers ------------------ - -Parsl is able to transparently stage files between at-rest locations and -execution locations by specifying a list of -:py:class:`~parsl.data_provider.staging.Staging` instances for an executor. -These staging instances define how to transfer files in and out of an execution -location. This list should be supplied as the ``storage_access`` -parameter to an executor when it is constructed. - -Parsl includes several staging providers for moving files using the -schemes defined above. By default, Parsl executors are created with -three common staging providers: -the NoOpFileStaging provider for local and shared file systems -and the HTTP(S) and FTP staging providers for transferring -files to and from remote storage locations. The following -example shows how to explicitly set the default staging providers. - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - from parsl.data_provider.data_manager import default_staging - - config = Config( - executors=[ - HighThroughputExecutor( - storage_access=default_staging, - # equivalent to the following - # storage_access=[NoOpFileStaging(), FTPSeparateTaskStaging(), HTTPSeparateTaskStaging()], - ) - ] - ) - - -Parsl further differentiates when staging occurs relative to -the app invocation that requires or produces files. -Staging either occurs with the executing task (*in-task staging*) -or as a separate task (*separate task staging*) before app execution. -In-task staging -uses a wrapper that is executed around the Parsl task and thus -occurs on the resource on which the task is executed. Separate -task staging inserts a new Parsl task in the graph and associates -a dependency between the staging task and the task that depends -on that file. Separate task staging may occur on either the submit-side -(e.g., when using Globus) or on the execution-side (e.g., HTTPS, FTP). - - -NoOpFileStaging for Local/Shared File Systems -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -The NoOpFileStaging provider assumes that files specified either -with a path or with the ``file`` URL scheme are available both -on the submit and execution side. This occurs, for example, when there is a -shared file system. In this case, files will not moved, and the -File object simply presents the same file path to the Parsl program -and any executing tasks. - -Files defined as follows will be handled by the NoOpFileStaging provider. - -.. code-block:: python - - File('file://home/parsl/data.txt') - File('/home/parsl/data.txt') - - -The NoOpFileStaging provider is enabled by default on all -executors. It can be explicitly set as the only -staging provider as follows. - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - from parsl.data_provider.file_noop import NoOpFileStaging - - config = Config( - executors=[ - HighThroughputExecutor( - storage_access=[NoOpFileStaging()] - ) - ] - ) - - -FTP, HTTP, HTTPS: separate task staging -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Files named with the ``ftp``, ``http`` or ``https`` URL scheme will be -staged in using HTTP GET or anonymous FTP commands. These commands -will be executed as a separate -Parsl task that will complete before the corresponding app -executes. These providers cannot be used to stage out output files. - -The following example defines a file accessible on a remote FTP server. - -.. code-block:: python - - File('ftp://www.iana.org/pub/mirror/rirstats/arin/ARIN-STATS-FORMAT-CHANGE.txt') - -When such a file object is passed as an input to an app, Parsl will download the file to whatever location is selected for the app to execute. -The following example illustrates how the remote file is implicitly downloaded from an FTP server and then converted. Note that the app does not need to know the location of the downloaded file on the remote computer, as Parsl abstracts this translation. - -.. code-block:: python - - @python_app - def convert(inputs=(), outputs=()): - with open(inputs[0].filepath, 'r') as inp: - content = inp.read() - with open(outputs[0].filepath, 'w') as out: - out.write(content.upper()) - - # create an remote Parsl file - inp = File('ftp://www.iana.org/pub/mirror/rirstats/arin/ARIN-STATS-FORMAT-CHANGE.txt') - - # create a local Parsl file - out = File('file:///tmp/ARIN-STATS-FORMAT-CHANGE.txt') - - # call the convert app with the Parsl file - f = convert(inputs=[inp], outputs=[out]) - f.result() - -HTTP and FTP separate task staging providers can be configured as follows. - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - from parsl.data_provider.http import HTTPSeparateTaskStaging - from parsl.data_provider.ftp import FTPSeparateTaskStaging - - config = Config( - executors=[ - HighThroughputExecutor( - storage_access=[HTTPSeparateTaskStaging(), FTPSeparateTaskStaging()] - ) - ] - ) - -FTP, HTTP, HTTPS: in-task staging -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -These staging providers are intended for use on executors that do not have -a file system shared between each executor node. - -These providers will use the same HTTP GET/anonymous FTP as the separate -task staging providers described above, but will do so in a wrapper around -individual app invocations, which guarantees that they will stage files to -a file system visible to the app. - -A downside of this staging approach is that the staging tasks are less visible -to Parsl, as they are not performed as separate Parsl tasks. - -In-task staging providers can be configured as follows. - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - from parsl.data_provider.http import HTTPInTaskStaging - from parsl.data_provider.ftp import FTPInTaskStaging - - config = Config( - executors=[ - HighThroughputExecutor( - storage_access=[HTTPInTaskStaging(), FTPInTaskStaging()] - ) - ] - ) - - -Globus -^^^^^^ - -The ``Globus`` staging provider is used to transfer files that can be accessed -using Globus. A guide to using Globus is available `here -`_). - -A file using the Globus scheme must specify the UUID of the Globus -endpoint and a path to the file on the endpoint, for example: - -.. code-block:: python - - File('globus://037f054a-15cf-11e8-b611-0ac6873fc732/unsorted.txt') - -Note: a Globus endpoint's UUID can be found in the Globus `Manage Endpoints `_ page. - -There must also be a Globus endpoint available with access to a -execute-side file system, because Globus file transfers happen -between two Globus endpoints. - -Globus Configuration -"""""""""""""""""""" - -In order to manage where files are staged, users must configure the default ``working_dir`` on a remote location. This information is specified in the :class:`~parsl.executors.base.ParslExecutor` via the ``working_dir`` parameter in the :class:`~parsl.config.Config` instance. For example: - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - - config = Config( - executors=[ - HighThroughputExecutor( - working_dir="/home/user/data" - ) - ] - ) - -Parsl requires knowledge of the Globus endpoint that is associated with an executor. This is done by specifying the ``endpoint_name`` (the UUID of the Globus endpoint that is associated with the system) in the configuration. - -In some cases, for example when using a Globus `shared endpoint `_ or when a Globus endpoint is mounted on a supercomputer, the path seen by Globus is not the same as the local path seen by Parsl. In this case the configuration may optionally specify a mapping between the ``endpoint_path`` (the common root path seen in Globus), and the ``local_path`` (the common root path on the local file system), as in the following. In most cases, ``endpoint_path`` and ``local_path`` are the same and do not need to be specified. - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - from parsl.data_provider.globus import GlobusStaging - from parsl.data_provider.data_manager import default_staging - - config = Config( - executors=[ - HighThroughputExecutor( - working_dir="/home/user/parsl_script", - storage_access=default_staging + [GlobusStaging( - endpoint_uuid="7d2dc622-2edb-11e8-b8be-0ac6873fc732", - endpoint_path="/", - local_path="/home/user" - )] - ) - ] - ) - - -Globus Authorization -"""""""""""""""""""" - -In order to transfer files with Globus, the user must first authenticate. -The first time that Globus is used with Parsl on a computer, the program -will prompt the user to follow an authentication and authorization -procedure involving a web browser. Users can authorize out of band by -running the parsl-globus-auth utility. This is useful, for example, -when running a Parsl program in a batch system where it will be unattended. - -.. code-block:: bash - - $ parsl-globus-auth - Parsl Globus command-line authorizer - If authorization to Globus is necessary, the library will prompt you now. - Otherwise it will do nothing - Authorization complete - -rsync -^^^^^ - -The ``rsync`` utility can be used to transfer files in the ``file`` scheme in configurations where -workers cannot access the submit-side file system directly, such as when executing -on an AWS EC2 instance or on a cluster without a shared file system. -However, the submit-side file system must be exposed using rsync. - -rsync Configuration -""""""""""""""""""" - -``rsync`` must be installed on both the submit and worker side. It can usually be installed -by using the operating system package manager: for example, by ``apt-get install rsync``. - -An `RSyncStaging` option must then be added to the Parsl configuration file, as in the following. -The parameter to RSyncStaging should describe the prefix to be passed to each rsync -command to connect from workers to the submit-side host. This will often be the username -and public IP address of the submitting system. - -.. code-block:: python - - from parsl.data_provider.rsync import RSyncStaging - - config = Config( - executors=[ - HighThroughputExecutor( - storage_access=[HTTPInTaskStaging(), FTPInTaskStaging(), RSyncStaging("benc@" + public_ip)], - ... - ) - ) - -rsync Authorization -""""""""""""""""""" - -The rsync staging provider delegates all authentication and authorization to the -underlying ``rsync`` command. This command must be correctly authorized to connect back to -the submit-side system. The form of this authorization will depend on the systems in -question. - -The following example installs an ssh key from the submit-side file system and turns off host key -checking, in the ``worker_init`` initialization of an EC2 instance. The ssh key must have -sufficient privileges to run ``rsync`` over ssh on the submit-side system. - -.. code-block:: python - - with open("rsync-callback-ssh", "r") as f: - private_key = f.read() - - ssh_init = """ - mkdir .ssh - chmod go-rwx .ssh - - cat > .ssh/id_rsa < .ssh/config <`_ diff --git a/docs/userguide/exceptions.rst b/docs/userguide/exceptions.rst index d18fbe704d..71af9d52d7 100644 --- a/docs/userguide/exceptions.rst +++ b/docs/userguide/exceptions.rst @@ -1,171 +1,9 @@ -.. _label-exceptions: +:orphan: -Error handling -============== +.. meta:: + :content http-equiv="refresh": 0;url=workflows/exceptions.html -Parsl provides various mechanisms to add resiliency and robustness to programs. +Redirect +-------- -Exceptions ----------- - -Parsl is designed to capture, track, and handle various errors occurring -during execution, including those related to the program, apps, execution -environment, and Parsl itself. -It also provides functionality to appropriately respond to failures during -execution. - -Failures might occur for various reasons: - -1. A task failed during execution. -2. A task failed to launch, for example, because an input dependency was not met. -3. There was a formatting error while formatting the command-line string in Bash apps. -4. A task completed execution but failed to produce one or more of its specified - outputs. -5. Task exceeded the specified walltime. - -Since Parsl tasks are executed asynchronously and remotely, it can be difficult to determine -when errors have occurred and to appropriately handle them in a Parsl program. - -For errors occurring in Python code, Parsl captures Python exceptions and returns -them to the main Parsl program. For non-Python errors, for example when a node -or worker fails, Parsl imposes a timeout, and considers a task to have failed -if it has not heard from the task by that timeout. Parsl also considers a task to have failed -if it does not meet the contract stated by the user during invocation, such as failing -to produce the stated output files. - -Parsl communicates these errors by associating Python exceptions with task futures. -These exceptions are raised only when a result is called on the future -of a failed task. For example: - -.. code-block:: python - - @python_app - def bad_divide(x): - return 6 / x - - # Call bad divide with 0, to cause a divide by zero exception - doubled_x = bad_divide(0) - - # Catch and handle the exception. - try: - doubled_x.result() - except ZeroDivisionError as e: - print('Oops! You tried to divide by 0.') - except Exception as e: - print('Oops! Something really bad happened.') - - -Retries -------- - -Often errors in distributed/parallel environments are transient. -In these cases, retrying failed tasks can be a simple way -of overcoming transient (e.g., machine failure, -network failure) and intermittent failures. -When ``retries`` are enabled (and set to an integer > 0), Parsl will automatically -re-launch tasks that have failed until the retry limit is reached. -By default, retries are disabled and exceptions will be communicated -to the Parsl program. - -The following example shows how the number of retries can be set to 2: - -.. code-block:: python - - import parsl - from parsl.configs.htex_local import config - - config.retries = 2 - - parsl.load(config) - -More specific retry handling can be specified using retry handlers, documented -below. - - -Lazy fail ---------- - -Parsl implements a lazy failure model through which a workload will continue -to execute in the case that some tasks fail. That is, the program will not -halt as soon as it encounters a failure, rather it will continue to execute -unaffected apps. - -The following example shows how lazy failures affect execution. In this -case, task C fails and therefore tasks E and F that depend on results from -C cannot be executed; however, Parsl will continue to execute tasks B and D -as they are unaffected by task C's failure. - -.. code-block:: - - Here's a workflow graph, where - (X) is runnable, - [X] is completed, - (X*) is failed. - (!X) is dependency failed - - (A) [A] (A) - / \ / \ / \ - (B) (C) [B] (C*) [B] (C*) - | | => | | => | | - (D) (E) (D) (E) [D] (!E) - \ / \ / \ / - (F) (F) (!F) - - time -----> - - -Retry handlers --------------- - -The basic parsl retry mechanism keeps a count of the number of times a task -has been (re)tried, and will continue retrying that task until the configured -retry limit is reached. - -Retry handlers generalize this to allow more expressive retry handling: -parsl keeps a retry cost for a task, and the task will be retried until the -configured retry limit is reached. Instead of the cost being 1 for each -failure, user-supplied code can examine the failure and compute a custom -cost. - -This allows user knowledge about failures to influence the retry mechanism: -an exception which is almost definitely a non-recoverable failure (for example, -due to bad parameters) can be given a high retry cost (so that it will not -be retried many times, or at all), and exceptions which are likely to be -transient (for example, where a worker node has died) can be given a low -retry cost so they will be retried many times. - -A retry handler can be specified in the parsl configuration like this: - - -.. code-block:: python - - Config( - retries=2, - retry_handler=example_retry_handler - ) - - -``example_retry_handler`` should be a function defined by the user that will -compute the retry cost for a particular failure, given some information about -the failure. - -For example, the following handler will give a cost of 1 to all exceptions, -except when a bash app exits with unix exitcode 9, in which case the cost will -be 100. This will have the effect that retries will happen as normal for most -errors, but the bash app can indicate that there is little point in retrying -by exiting with exitcode 9. - -.. code-block:: python - - def example_retry_handler(exception, task_record): - if isinstance(exception, BashExitFailure) and exception.exitcode == 9: - return 100 - else - return 1 - -The retry handler is given two parameters: the exception from execution, and -the parsl internal task_record. The task record contains details such as the -app name, parameters and executor. - -If a retry handler raises an exception itself, then the task will be aborted -and no further tries will be attempted. +This page has been `moved `_ diff --git a/docs/userguide/execution.rst b/docs/userguide/execution.rst index 832985c164..20346bfe35 100644 --- a/docs/userguide/execution.rst +++ b/docs/userguide/execution.rst @@ -1,389 +1,9 @@ -.. _label-execution: +:orphan: +.. meta:: + :content http-equiv="refresh": 0;url=configuration/execution.html -Execution -========= +Redirect +-------- -Contemporary computing environments may include a wide range of computational platforms or **execution providers**, from laptops and PCs to various clusters, supercomputers, and cloud computing platforms. Different execution providers may require or allow for the use of different **execution models**, such as threads (for efficient parallel execution on a multicore processor), processes, and pilot jobs for running many small tasks on a large parallel system. - -Parsl is designed to abstract these low-level details so that an identical Parsl program can run unchanged on different platforms or across multiple platforms. -To this end, Parsl uses a configuration file to specify which execution provider(s) and execution model(s) to use. -Parsl provides a high level abstraction, called a *block*, for providing a uniform description of a compute resource irrespective of the specific execution provider. - -.. note:: - Refer to :ref:`configuration-section` for information on how to configure the various components described - below for specific scenarios. - -Execution providers -------------------- - -Clouds, supercomputers, and local PCs offer vastly different modes of access. -To overcome these differences, and present a single uniform interface, -Parsl implements a simple provider abstraction. This -abstraction is key to Parsl's ability to enable scripts to be moved -between resources. The provider interface exposes three core actions: submit a -job for execution (e.g., sbatch for the Slurm resource manager), -retrieve the status of an allocation (e.g., squeue), and cancel a running -job (e.g., scancel). Parsl implements providers for local execution -(fork), for various cloud platforms using cloud-specific APIs, and -for clusters and supercomputers that use a Local Resource Manager -(LRM) to manage access to resources, such as Slurm and HTCondor. - -Each provider implementation may allow users to specify additional parameters for further configuration. Parameters are generally mapped to LRM submission script or cloud API options. -Examples of LRM-specific options are partition, wall clock time, -scheduler options (e.g., #SBATCH arguments for Slurm), and worker -initialization commands (e.g., loading a conda environment). Cloud -parameters include access keys, instance type, and spot bid price - -Parsl currently supports the following providers: - -1. `parsl.providers.LocalProvider`: The provider allows you to run locally on your laptop or workstation. -2. `parsl.providers.SlurmProvider`: This provider allows you to schedule resources via the Slurm scheduler. -3. `parsl.providers.CondorProvider`: This provider allows you to schedule resources via the Condor scheduler. -4. `parsl.providers.GridEngineProvider`: This provider allows you to schedule resources via the GridEngine scheduler. -5. `parsl.providers.TorqueProvider`: This provider allows you to schedule resources via the Torque scheduler. -6. `parsl.providers.AWSProvider`: This provider allows you to provision and manage cloud nodes from Amazon Web Services. -7. `parsl.providers.GoogleCloudProvider`: This provider allows you to provision and manage cloud nodes from Google Cloud. -8. `parsl.providers.KubernetesProvider`: This provider allows you to provision and manage containers on a Kubernetes cluster. -9. `parsl.providers.LSFProvider`: This provider allows you to schedule resources via IBM's LSF scheduler. - - - -Executors ---------- - -Parsl programs vary widely in terms of their -execution requirements. Individual Apps may run for milliseconds -or days, and available parallelism can vary between none for -sequential programs to millions for "pleasingly parallel" programs. -Parsl executors, as the name suggests, execute Apps on one or more -target execution resources such as multi-core workstations, clouds, -or supercomputers. As it appears infeasible to implement a single -execution strategy that will meet so many diverse requirements on -such varied platforms, Parsl provides a modular executor interface -and a collection of executors that are tuned for common execution -patterns. - -Parsl executors extend the Executor class offered by Python's -concurrent.futures library, which allows Parsl to use -existing solutions in the Python Standard Library (e.g., ThreadPoolExecutor) -and from other packages such as Work Queue. Parsl -extends the concurrent.futures executor interface to support -additional capabilities such as automatic scaling of execution resources, -monitoring, deferred initialization, and methods to set working -directories. -All executors share a common execution kernel that is responsible -for deserializing the task (i.e., the App and its input arguments) -and executing the task in a sandboxed Python environment. - -Parsl currently supports the following executors: - -1. `parsl.executors.ThreadPoolExecutor`: This executor supports multi-thread execution on local resources. - -2. `parsl.executors.HighThroughputExecutor`: This executor implements hierarchical scheduling and batching using a pilot job model to deliver high throughput task execution on up to 4000 Nodes. - -3. `parsl.executors.WorkQueueExecutor`: This executor integrates `Work Queue `_ as an execution backend. Work Queue scales to tens of thousands of cores and implements reliable execution of tasks with dynamic resource sizing. - -4. `parsl.executors.taskvine.TaskVineExecutor`: This executor uses `TaskVine `_ as the execution backend. TaskVine scales up to tens of thousands of cores and actively uses local storage on compute nodes to offer a diverse array of performance-oriented features, including: smart caching and sharing common large files between tasks and compute nodes, reliable execution of tasks, dynamic resource sizing, automatic Python environment detection and sharing. -These executors cover a broad range of execution requirements. As with other Parsl components, there is a standard interface (ParslExecutor) that can be implemented to add support for other executors. - -.. note:: - Refer to :ref:`configuration-section` for information on how to configure these executors. - - -Launchers ---------- - -Many LRMs offer mechanisms for spawning applications across nodes -inside a single job and for specifying the -resources and task placement information needed to execute that -application at launch time. Common mechanisms include -`srun `_ (for Slurm), -`aprun `_ (for Crays), and `mpirun `_ (for MPI). -Thus, to run Parsl programs on such systems, we typically want first to -request a large number of nodes and then to *launch* "pilot job" or -**worker** processes using the system launchers. -Parsl's Launcher abstraction enables Parsl programs -to use these system-specific launcher systems to start workers across -cores and nodes. - -Parsl currently supports the following set of launchers: - -1. `parsl.launchers.SrunLauncher`: Srun based launcher for Slurm based systems. -2. `parsl.launchers.AprunLauncher`: Aprun based launcher for Crays. -3. `parsl.launchers.SrunMPILauncher`: Launcher for launching MPI applications with Srun. -4. `parsl.launchers.GnuParallelLauncher`: Launcher using GNU parallel to launch workers across nodes and cores. -5. `parsl.launchers.MpiExecLauncher`: Uses Mpiexec to launch. -6. `parsl.launchers.SimpleLauncher`: The launcher default to a single worker launch. -7. `parsl.launchers.SingleNodeLauncher`: This launcher launches ``workers_per_node`` count workers on a single node. - -Additionally, the launcher interface can be used to implement specialized behaviors -in custom environments (for example, to -launch node processes inside containers with customized environments). -For example, the following launcher uses Srun to launch ``worker-wrapper``, passing the -command to be run as parameters to ``worker-wrapper``. It is the responsibility of ``worker-wrapper`` -to launch the command it is given inside the appropriate environment. - -.. code:: python - - class MyShifterSRunLauncher: - def __init__(self): - self.srun_launcher = SrunLauncher() - - def __call__(self, command, tasks_per_node, nodes_per_block): - new_command="worker-wrapper {}".format(command) - return self.srun_launcher(new_command, tasks_per_node, nodes_per_block) - -Blocks ------- - -One challenge when making use of heterogeneous -execution resource types is the need to provide a uniform representation of -resources. Consider that single requests on clouds return individual -nodes, clusters and supercomputers provide batches of nodes, grids -provide cores, and workstations provide a single multicore node - -Parsl defines a resource abstraction called a *block* as the most basic unit -of resources to be acquired from a provider. A block contains one -or more nodes and maps to the different provider abstractions. In -a cluster, a block corresponds to a single allocation request to a -scheduler. In a cloud, a block corresponds to a single API request -for one or more instances. -Parsl can then execute *tasks* (instances of apps) -within and across (e.g., for MPI jobs) nodes within a block. -Blocks are also used as the basis for -elasticity on batch scheduling systems (see Elasticity below). -Three different examples of block configurations are shown below. - -1. A single block comprised of a node executing one task: - - .. image:: ../images/N1_T1.png - :scale: 75% - -2. A single block with one node executing several tasks. This configuration is - most suitable for single threaded apps running on multicore target systems. - The number of tasks executed concurrently is proportional to the number of cores available on the system. - - .. image:: ../images/N1_T4.png - :scale: 75% - -3. A block comprised of several nodes and executing several tasks, where a task can span multiple nodes. This configuration - is generally used by MPI applications. Starting a task requires using a specific - MPI launcher that is supported on the target system (e.g., aprun, srun, mpirun, mpiexec). - The `MPI Apps `_ documentation page describes how to configure Parsl for this case. - - .. image:: ../images/N4_T2.png - -The configuration options for specifying the shape of each block are: - -1. ``workers_per_node``: Number of workers started per node, which corresponds to the number of tasks that can execute concurrently on a node. -2. ``nodes_per_block``: Number of nodes requested per block. - -.. _label-elasticity: - -Elasticity ----------- - -Workload resource requirements often vary over time. -For example, in the map-reduce paradigm the map phase may require more -resources than the reduce phase. In general, reserving sufficient -resources for the widest parallelism will result in underutilization -during periods of lower load; conversely, reserving minimal resources -for the thinnest parallelism will lead to optimal utilization -but also extended execution time. -Even simple bag-of-task applications may have tasks of different durations, leading to trailing -tasks with a thin workload. - -To address dynamic workload requirements, -Parsl implements a cloud-like elasticity model in which resource -blocks are provisioned/deprovisioned in response to workload pressure. -Given the general nature of the implementation, -Parsl can provide elastic execution on clouds, clusters, -and supercomputers. Of course, in an HPC setting, elasticity may -be complicated by queue delays. - -Parsl's elasticity model includes a flow control system -that monitors outstanding tasks and available compute capacity. -This flow control monitor determines when to trigger scaling (in or out) -events to match workload needs. - -The animated diagram below shows how blocks are elastically -managed within an executor. The Parsl configuration for an executor -defines the minimum, maximum, and initial number of blocks to be used. - -.. image:: parsl_scaling.gif - -The configuration options for specifying elasticity bounds are: - -1. ``min_blocks``: Minimum number of blocks to maintain per executor. -2. ``init_blocks``: Initial number of blocks to provision at initialization of workflow. -3. ``max_blocks``: Maximum number of blocks that can be active per executor. - - - -Parallelism -^^^^^^^^^^^ - -Parsl provides a user-managed model for controlling elasticity. -In addition to setting the minimum -and maximum number of blocks to be provisioned, users can also define -the desired level of parallelism by setting a parameter (*p*). Parallelism -is expressed as the ratio of task execution capacity to the sum of running tasks -and available tasks (tasks with their dependencies met, but waiting for execution). -A parallelism value of 1 represents aggressive scaling where the maximum resources -needed are used (i.e., max_blocks); parallelism close to 0 represents the opposite situation in which -as few resources as possible (i.e., min_blocks) are used. By selecting a fraction between 0 and 1, -the provisioning aggressiveness can be controlled. - -For example: - -- When p = 0: Use the fewest resources possible. If there is no workload then no blocks will be provisioned, otherwise the fewest blocks specified (e.g., min_blocks, or 1 if min_blocks is set to 0) will be provisioned. - -.. code:: python - - if active_tasks == 0: - blocks = min_blocks - else: - blocks = max(min_blocks, 1) - -- When p = 1: Use as many resources as possible. Provision sufficient nodes to execute all running and available tasks concurrently up to the max_blocks specified. - -.. code-block:: python - - blocks = min(max_blocks, - ceil((running_tasks + available_tasks) / (workers_per_node * nodes_per_block)) - -- When p = 1/2: Queue up to 2 tasks per worker before requesting a new block. - - -Configuration -^^^^^^^^^^^^^ - -The example below shows how elasticity and parallelism can be configured. Here, a `parsl.executors.HighThroughputExecutor` -is used with a minimum of 1 block and a maximum of 2 blocks, where each block may host -up to 2 workers per node. Thus this setup is capable of servicing 2 tasks concurrently. -Parallelism of 0.5 means that when more than 2 * the total task capacity (i.e., 4 tasks) are queued a new -block will be requested. An example :class:`~parsl.config.Config` is: - -.. code:: python - - from parsl.config import Config - from libsubmit.providers.local.local import Local - from parsl.executors import HighThroughputExecutor - - config = Config( - executors=[ - HighThroughputExecutor( - label='local_htex', - workers_per_node=2, - provider=Local( - min_blocks=1, - init_blocks=1, - max_blocks=2, - nodes_per_block=1, - parallelism=0.5 - ) - ) - ] - ) - -The animated diagram below illustrates the behavior of this executor. -In the diagram, the tasks are allocated to the first block, until -5 tasks are submitted. At this stage, as more than double the available -task capacity is used, Parsl provisions a new block for executing the remaining -tasks. - -.. image:: parsl_parallelism.gif - - -Multi-executor --------------- - -Parsl supports the use of one or more executors as specified in the configuration. -In this situation, individual apps may indicate which executors they are able to use. - -The common scenarios for this feature are: - -* A workflow has an initial simulation stage that runs on the compute heavy - nodes of an HPC system followed by an analysis and visualization stage that - is better suited for GPU nodes. -* A workflow follows a repeated fan-out, fan-in model where the long running - fan-out tasks are computed on a cluster and the quick fan-in computation is - better suited for execution using threads on a login node. -* A workflow includes apps that wait and evaluate the results of a - computation to determine whether the app should be relaunched. - Only apps running on threads may launch other apps. Often, simulations - have stochastic behavior and may terminate before completion. - In such cases, having a wrapper app that checks the exit code - and determines whether or not the app has completed successfully can - be used to automatically re-execute the app (possibly from a - checkpoint) until successful completion. - - -The following code snippet shows how apps can specify suitable executors in the app decorator. - -.. code-block:: python - - #(CPU heavy app) (CPU heavy app) (CPU heavy app) <--- Run on compute queue - # | | | - # (data) (data) (data) - # \ | / - # (Analysis and visualization phase) <--- Run on GPU node - - # A mock molecular dynamics simulation app - @bash_app(executors=["Theta.Phi"]) - def MD_Sim(arg, outputs=()): - return "MD_simulate {} -o {}".format(arg, outputs[0]) - - # Visualize results from the mock MD simulation app - @bash_app(executors=["Cooley.GPU"]) - def visualize(inputs=(), outputs=()): - bash_array = " ".join(inputs) - return "viz {} -o {}".format(bash_array, outputs[0]) - - -Encryption ----------- - -Users can enable encryption for the ``HighThroughputExecutor`` by setting its ``encrypted`` -initialization argument to ``True``. - -For example, - -.. code-block:: python - - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - - config = Config( - executors=[ - HighThroughputExecutor( - encrypted=True - ) - ] - ) - -Under the hood, we use `CurveZMQ `_ to encrypt all communication channels -between the executor and related nodes. - -Encryption performance -^^^^^^^^^^^^^^^^^^^^^^ - -CurveZMQ depends on `libzmq `_ and `libsodium `_, -which `pyzmq `_ (a Parsl dependency) includes as part of its -installation via ``pip``. This installation path should work on most systems, but users have -reported significant performance degradation as a result. - -If you experience a significant performance hit after enabling encryption, we recommend installing -``pyzmq`` with conda: - -.. code-block:: bash - - conda install conda-forge::pyzmq - -Alternatively, you can `install libsodium `_, then -`install libzmq `_, then build ``pyzmq`` from source: - -.. code-block:: bash - - pip3 install parsl --no-binary pyzmq +This page has been `moved `_ diff --git a/docs/userguide/futures.rst b/docs/userguide/futures.rst index 13d22a211b..1a4f40e79e 100644 --- a/docs/userguide/futures.rst +++ b/docs/userguide/futures.rst @@ -1,165 +1,9 @@ -.. _label-futures: +:orphan: -Futures -======= +.. meta:: + :content http-equiv="refresh": 0;url=execution/futures.html -When an ordinary Python function is invoked in a Python program, the Python interpreter waits for the function to complete execution -before proceeding to the next statement. -But if a function is expected to execute for a long period of time, it may be preferable not to wait for -its completion but instead to proceed immediately with executing subsequent statements. -The function can then execute concurrently with that other computation. +Redirect +-------- -Concurrency can be used to enhance performance when independent activities -can execute on different cores or nodes in parallel. The following -code fragment demonstrates this idea, showing that overall execution time -may be reduced if the two function calls are executed concurrently. - -.. code-block:: python - - v1 = expensive_function(1) - v2 = expensive_function(2) - result = v1 + v2 - -However, concurrency also introduces a need for **synchronization**. -In the example, it is not possible to compute the sum of ``v1`` and ``v2`` -until both function calls have completed. -Synchronization provides a way of blocking execution of one activity -(here, the statement ``result = v1 + v2``) until other activities -(here, the two calls to ``expensive_function()``) have completed. - -Parsl supports concurrency and synchronization as follows. -Whenever a Parsl program calls a Parsl app (a function annotated with a Parsl -app decorator, see :ref:`apps`), -Parsl will create a new ``task`` and immediately return a -`future `_ in lieu of that function's result(s). -The program will then continue immediately to the next statement in the program. -At some point, for example when the task's dependencies are met and there -is available computing capacity, Parsl will execute the task. Upon -completion, Parsl will set the value of the future to contain the task's -output. - -A future can be used to track the status of an asynchronous task. -For example, after creation, the future may be interrogated to determine -the task's status (e.g., running, failed, completed), access results, -and capture exceptions. Further, futures may be used for synchronization, -enabling the calling Python program to block until the future -has completed execution. - -Parsl provides two types of futures: `AppFuture` and `DataFuture`. -While related, they enable subtly different parallel patterns. - -AppFutures ----------- - -AppFutures are the basic building block upon which Parsl programs are built. Every invocation of a Parsl app returns an AppFuture that may be used to monitor and manage the task's execution. -AppFutures are inherited from Python's `concurrent library `_. -They provide three key capabilities: - -1. An AppFuture's ``result()`` function can be used to wait for an app to complete, and then access any result(s). -This function is blocking: it returns only when the app completes or fails. -The following code fragment implements an example similar to the ``expensive_function()`` example above. -Here, the ``sleep_double`` app simply doubles the input value. The program invokes -the ``sleep_double`` app twice, and returns futures in place of results. The example -shows how the future's ``result()`` function can be used to wait for the results from the -two ``sleep_double`` app invocations to be computed. - -.. code-block:: python - - @python_app - def sleep_double(x): - import time - time.sleep(2) # Sleep for 2 seconds - return x*2 - - # Start two concurrent sleep_double apps. doubled_x1 and doubled_x2 are AppFutures - doubled_x1 = sleep_double(10) - doubled_x2 = sleep_double(5) - - # The result() function will block until each of the corresponding app calls have completed - print(doubled_x1.result() + doubled_x2.result()) - -2. An AppFuture's ``done()`` function can be used to check the status of an app, *without blocking*. -The following example shows that calling the future's ``done()`` function will not stop execution of the main Python program. - -.. code-block:: python - - @python_app - def double(x): - return x*2 - - # doubled_x is an AppFuture - doubled_x = double(10) - - # Check status of doubled_x, this will print True if the result is available, else False - print(doubled_x.done()) - -3. An AppFuture provides a safe way to handle exceptions and errors while asynchronously executing -apps. The example shows how exceptions can be captured in the same way as a standard Python program -when calling the future's ``result()`` function. - -.. code-block:: python - - @python_app - def bad_divide(x): - return 6/x - - # Call bad divide with 0, to cause a divide by zero exception - doubled_x = bad_divide(0) - - # Catch and handle the exception. - try: - doubled_x.result() - except ZeroDivisionError as ze: - print('Oops! You tried to divide by 0') - except Exception as e: - print('Oops! Something really bad happened') - - -In addition to being able to capture exceptions raised by a specific app, Parsl also raises ``DependencyErrors`` when apps are unable to execute due to failures in prior dependent apps. -That is, an app that is dependent upon the successful completion of another app will fail with a dependency error if any of the apps on which it depends fail. - - -DataFutures ------------ - -While an AppFuture represents the execution of an asynchronous app, -a DataFuture represents a file to be produced by that app. -Parsl's dataflow model requires such a construct so that it can determine -when dependent apps, apps that that are to consume a file produced by another app, -can start execution. - -When calling an app that produces files as outputs, Parsl requires that a list of output files be specified (as a list of `File` objects passed in via the ``outputs`` keyword argument). Parsl will return a DataFuture for each output file as part AppFuture when the app is executed. -These DataFutures are accessible in the AppFuture's ``outputs`` attribute. - -Each DataFuture will complete when the App has finished executing, -and the corresponding file has been created (and if specified, staged out). - -When a DataFuture is passed as an argument to a subsequent app invocation, -that subsequent app will not begin execution until the DataFuture is -completed. The input argument will then be replaced with an appropriate -File object. - -The following code snippet shows how DataFutures are used. In this -example, the call to the echo Bash app specifies that the results -should be written to an output file ("hello1.txt"). The main -program inspects the status of the output file (via the future's -``outputs`` attribute) and then blocks waiting for the file to -be created (``hello.outputs[0].result()``). - -.. code-block:: python - - # This app echoes the input string to the first file specified in the - # outputs list - @bash_app - def echo(message, outputs=()): - return 'echo {} &> {}'.format(message, outputs[0]) - - # Call echo specifying the output file - hello = echo('Hello World!', outputs=[File('hello1.txt')]) - - # The AppFuture's outputs attribute is a list of DataFutures - print(hello.outputs) - - # Print the contents of the output DataFuture when complete - with open(hello.outputs[0].result().filepath, 'r') as f: - print(f.read()) +This page has been `moved `_ diff --git a/docs/userguide/index.rst b/docs/userguide/index.rst index 12254cd6e2..a626b42a2f 100644 --- a/docs/userguide/index.rst +++ b/docs/userguide/index.rst @@ -1,23 +1,17 @@ User guide ========== +Parsl applications are composed of **Apps** that define tasks to be executed, +**Configuration** objects which define resources available for executing tasks, +and **Workflow** scripts that weave tasks together into parallel workflows. + +Start with an `overview of Parsl `_ before learning about each component. + .. toctree:: - :maxdepth: 5 + :maxdepth: 2 overview - apps - futures - data - execution - mpi_apps - exceptions - checkpoints - configuring - monitoring - workflow - modularizing - lifted_ops - joins - usage_tracking - plugins - parsl_perf + apps/index + configuration/index + workflows/index + advanced/index diff --git a/docs/userguide/joins.rst b/docs/userguide/joins.rst index defb0ad012..4cc741f931 100644 --- a/docs/userguide/joins.rst +++ b/docs/userguide/joins.rst @@ -1,257 +1,9 @@ -.. _label-joinapp: +:orphan: -Join Apps -========= +.. meta:: + :content http-equiv="refresh": 0;url=apps/joins.html -Join apps, defined with the ``@join_app`` decorator, are a form of app that can -launch other pieces of a workflow: for example a Parsl sub-workflow, or a task -that runs in some other system. +Redirect +-------- -Parsl sub-workflows -------------------- - -One reason for launching Parsl apps from inside a join app, rather than -directly in the main workflow code, is because the definitions of those tasks -are not known well enough at the start of the workflow. - -For example, a workflow might run an expensive step to detect some objects -in an image, and then on each object, run a further expensive step. Because -the number of objects is not known at the start of the workflow, but instead -only after an expensive step has completed, the subsequent tasks cannot be -defined until after that step has completed. - -In simple cases, the main workflow script can be stopped using -``Future.result()`` and join apps are not necessary, but in more complicated -cases, that approach can severely limit concurrency. - -Join apps allow more naunced dependencies to be expressed that can help with: - -* increased concurrency - helping with strong scaling -* more focused error propagation - allowing more of an ultimately failing workflow to complete -* more useful monitoring information - -Using Futures from other components ------------------------------------ - -Sometimes, a workflow might need to incorporate tasks from other systems that -run asynchronously but do not need a Parsl worker allocated for their entire -run. An example of this is delegating some work into Globus Compute: work can -be given to Globus Compute, but Parsl does not need to keep a worker allocated -to that task while it runs. Instead, Parsl can be told to wait for the ``Future`` -returned by Globus Compute to complete. - -Usage ------ - -A `join_app` looks quite like a `python_app`, but should return one or more -``Future`` objects, rather than a value. Once the Python code has run, the -app will wait for those Futures to complete without occuping a Parsl worker, -and when those Futures complete, their contents will be the return value -of the `join_app`. - -For example: - -.. code-block:: python - - @python_app - def some_app(): - return 3 - - @join_app - def example(): - x: Future = some_app() - return x # note that x is a Future, not a value - - assert example.result() == 3 - -Example of a Parsl sub-workflow -------------------------------- - -This example workflow shows a preprocessing step, followed by -a middle stage that is chosen by the result of the pre-processing step -(either option 1 or option 2) followed by a know post-processing step. - -.. code-block:: python - - @python_app - def pre_process(): - return 3 - - @python_app - def option_one(x): - return x*2 - - @python_app - def option_two(x): - return (-x) * 2 - - @join_app - def process(x): - if x > 0: - return option_one(x) - else: - return option_two(x) - - @python_app - def post_process(x): - return str(x) - - assert post_process(process(pre_process()))).result() == "6" - -* Why can't process be a regular python function? - -``process`` needs to inspect the value of ``x`` to make a decision about -what app to launch. So it needs to defer execution until after the -pre-processing stage has completed. In Parsl, the way to defer that is -using apps: even though ``process`` is invoked at the start of the workflow, -it will execute later on, when the Future returned by ``pre_process`` has a -value. - -* Why can't process be a @python_app? - -A Python app, if run in a `parsl.executors.ThreadPoolExecutor`, can launch -more parsl apps; so a ``python_app`` implementation of process() would be able -to inspect x and choose and invoke the appropriate ``option_{one, two}``. - -From launching the ``option_{one, two}`` app, the app body python code would -get a ``Future[int]`` - a ``Future`` that will eventually contain ``int``. - -But, we want to invoke ``post_process`` at submission time near the start of -workflow so that Parsl knows about as many tasks as possible. But we don't -want it to execute until the value of the chosen ``option_{one, two}`` app -is known. - -If we don't have join apps, how can we do this? - -We could make process wait for ``option_{one, two}`` to complete, before -returning, like this: - -.. code-block:: python - - @python_app - def process(x): - if x > 0: - f = option_one(x) - else: - f = option_two(x) - return f.result() - -but this will block the worker running ``process`` until ``option_{one, two}`` -has completed. If there aren't enough workers to run ``option_{one, two}`` this -can even deadlock. (principle: apps should not wait on completion of other -apps and should always allow parsl to handle this through dependencies) - -We could make process return the ``Future`` to the main workflow thread: - -.. code-block:: python - - @python_app - def process(x): - if x > 0: - f = option_one(x) - else: - f = option_two(x) - return f # f is a Future[int] - - # process(3) is a Future[Future[int]] - - -What comes out of invoking ``process(x)`` now is a nested ``Future[Future[int]]`` -- it's a promise that eventually process will give you a promise (from -``option_one, two}``) that will eventually give you an int. - -We can't pass that future into post_process... because post_process wants the -final int, and that future will complete before the int is ready, and that -(outer) future will have as its value the inner future (which won't be complete yet). - -So we could wait for the result in the main workflow thread: - -.. code-block:: python - - f_outer = process(pre_process()) # Future[Future[int]] - f_inner = f_outer.result # Future[int] - result = post_process(f_inner) - # result == "6" - -But this now blocks the main workflow thread. If we really only need to run -these three lines, that's fine, but what about if we are in a for loop that -sets up 1000 parametrised iterations: - -.. code-block:: python - - for x in [1..1000]: - f_outer = process(pre_process(x)) # Future[Future[int]] - f_inner = f_outer.result() # Future[int] - result = post_process(f_inner) - -The ``for`` loop can only iterate after pre_processing is done for each -iteration - it is unnecessarily serialised by the ``.result()`` call, -so that pre_processing cannot run in parallel. - -So, the rule about not calling ``.result()`` applies in the main workflow thread -too. - -What join apps add is the ability for parsl to unwrap that Future[Future[int]] into a -Future[int] in a "sensible" way (eg it doesn't need to block a worker). - - -.. _label-join-globus-compute: - -Example of invoking a Futures-driven task from another system -------------------------------------------------------------- - - -This example shows launching some activity in another system, without -occupying a Parsl worker while that activity happens: in this example, work is -delegated to Globus Compute, which performs the work elsewhere. When the work -is completed, Globus Compute will put the result into the future that it -returns, and then (because the Parsl app is a ``@join_app``), that result will -be used as the result of the Parsl app. - -As above, the motivation for doing this inside an app, rather than in the -top level is that sufficient information to launch the Globus Compute task -might not be available at start of the workflow. - -This workflow will run a first stage, ``const_five``, on a Parsl worker, -then using the result of that stage, pass the result as a parameter to a -Globus Compute task, getting a ``Future`` from that submission. Then, the -results of the Globus Compute task will be passed onto a second Parsl -local task, ``times_two``. - -.. code-block:: python - - import parsl - from globus_compute_sdk import Executor - - tutorial_endpoint_uuid = '4b116d3c-1703-4f8f-9f6f-39921e5864df' - gce = Executor(endpoint_id=tutorial_endpoint_uuid) - - def increment_in_funcx(n): - return n+1 - - @parsl.join_app - def increment_in_parsl(n): - future = gce.submit(increment_in_funcx, n) - return future - - @parsl.python_app - def times_two(n): - return n*2 - - @parsl.python_app - def const_five(): - return 5 - - parsl.load() - - workflow = times_two(increment_in_parsl(const_five())) - - r = workflow.result() - - assert r == (5+1)*2 - - -Terminology ------------ - -The term "join" comes from use of monads in functional programming, especially Haskell. +This page has been `moved `_ diff --git a/docs/userguide/lifted_ops.rst b/docs/userguide/lifted_ops.rst index 6e258b9b62..ed28f55d56 100644 --- a/docs/userguide/lifted_ops.rst +++ b/docs/userguide/lifted_ops.rst @@ -1,56 +1,9 @@ -.. _label-liftedops: +:orphan: -Lifted operators -================ +.. meta:: + :content http-equiv="refresh": 0;url=workflows/lifted_ops.html -Parsl allows some operators (``[]`` and ``.``) to be used on an AppFuture in -a way that makes sense with those operators on the eventually returned -result. +Redirect +-------- -Lifted [] operator ------------------- - -When an app returns a complex structure such as a ``dict`` or a ``list``, -it is sometimes useful to pass an element of that structure to a subsequent -task, without waiting for that subsequent task to complete. - -To help with this, Parsl allows the ``[]`` operator to be used on an -`AppFuture`. This operator will return another `AppFuture` that will -complete after the initial future, with the result of ``[]`` on the value -of the initial future. - -The end result is that this assertion will hold: - -.. code-block:: python - - fut = my_app() - assert fut['x'].result() == fut.result()[x] - -but more concurrency will be available, as execution of the main workflow -code will not stop to wait for ``result()`` to complete on the initial -future. - -`AppFuture` does not implement other methods commonly associated with -dicts and lists, such as ``len``, because those methods should return a -specific type of result immediately, and that is not possible when the -results are not available until the future. - -If a key does not exist in the returned result, then the exception will -appear in the Future returned by ``[]``, rather than at the point that -the ``[]`` operator is applied. This is because the valid values that can -be used are not known until the underlying result is available. - -Lifted . operator ------------------ - -The ``.`` operator works similarly to ``[]`` described above: - -.. code-block:: python - - fut = my_app - assert fut.x.result() == fut.result().x - -Attributes beginning with ``_`` are not lifted as this usually indicates an -attribute that is used for internal purposes, and to try to avoid mixing -protocols (such as iteration in for loops) defined on AppFutures vs protocols -defined on the underlying result object. +This page has been `moved `_ diff --git a/docs/userguide/modularizing.rst b/docs/userguide/modularizing.rst index 143a4ebcd8..69ed82482c 100644 --- a/docs/userguide/modularizing.rst +++ b/docs/userguide/modularizing.rst @@ -1,109 +1,9 @@ -.. _codebases: +:orphan: -Structuring Parsl programs --------------------------- +.. meta:: + :content http-equiv="refresh": 0;url=advanced/modularizing.html -While convenient to build simple Parsl programs as a single Python file, -splitting a Parsl programs into multiple files and a Python module -has significant benefits, including: +Redirect +-------- - 1. Better readability - 2. Logical separation of components (e.g., apps, config, and control logic) - 3. Ease of reuse of components - -Large applications that use Parsl often divide into several core components: - -.. contents:: - :local: - :depth: 2 - -The following sections use an example where each component is in a separate file: - -.. code-block:: - - examples/logic.py - examples/app.py - examples/config.py - examples/__init__.py - run.py - pyproject.toml - -Run the application by first installing the Python library and then executing the "run.py" script. - -.. code-block:: bash - - pip install . # Install module so it can be imported by workers - python run.py - - -Core application logic -====================== - -The core application logic should be developed without any deference to Parsl. -Implement capabilities, write unit tests, and prepare documentation -in which ever way works best for the problem at hand. - -Parallelization with Parsl will be easy if the software already follows best practices. - -The example defines a function to convert a single integer into binary. - -.. literalinclude:: examples/library/logic.py - :caption: library/logic.py - -Workflow functions -================== - -Tasks within a workflow may require unique combinations of core functions. -Functions to be run in parallel must also meet :ref:`specific requirements ` -that may complicate writing the core logic effectively. -As such, separating functions to be used as Apps is often beneficial. - -The example includes a function to convert many integers into binary. - -Key points to note: - -- It is not necessary to have import statements inside the function. - Parsl will serialize this function by reference, as described in :ref:`functions-from-modules`. - -- The function is not yet marked as a Parsl PythonApp. - Keeping Parsl out of the function definitions simplifies testing - because you will not need to run Parsl when testing the code. - -- *Advanced*: Consider including Parsl decorators in the library if using complex workflow patterns, - such as :ref:`join apps ` or functions which take :ref:`special arguments `. - -.. literalinclude:: examples/library/app.py - :caption: library/app.py - - -Parsl configuration functions -============================= - -Create Parsl configurations specific to your application needs as functions. -While not necessary, including the Parsl configuration functions inside the module -ensures they can be imported into other scripts easily. - -Generating Parsl :class:`~parsl.config.Config` objects from a function -makes it possible to change the configuration without editing the module. - -The example function provides a configuration suited for a single node. - -.. literalinclude:: examples/library/config.py - :caption: library/config.py - -Orchestration Scripts -===================== - -The last file defines the workflow itself. - -Such orchestration scripts, at minimum, perform at least four tasks: - -1. *Load execution options* using a tool like :mod:`argparse`. -2. *Prepare workflow functions for execution* by creating :class:`~parsl.app.python.PythonApp` wrappers over each function. -3. *Create configuration then start Parsl* with the :meth:`parsl.load` function. -4. *Launch tasks and retrieve results* depending on the needs of the application. - -An example run script is as follows - -.. literalinclude:: examples/run.py - :caption: run.py +This page has been `moved `_ diff --git a/docs/userguide/monitoring.rst b/docs/userguide/monitoring.rst index 02b3177ca7..138127ec90 100644 --- a/docs/userguide/monitoring.rst +++ b/docs/userguide/monitoring.rst @@ -1,121 +1,9 @@ -Monitoring -========== +:orphan: -Parsl includes a monitoring system to capture task state as well as resource -usage over time. The Parsl monitoring system aims to provide detailed -information and diagnostic capabilities to help track the state of your -programs, down to the individual apps that are executed on remote machines. +.. meta:: + :content http-equiv="refresh": 0;url=advanced/monitoring.html -The monitoring system records information to an SQLite database while a -workflow runs. This information can then be visualised in a web dashboard -using the ``parsl-visualize`` tool, or queried using SQL using regular -SQLite tools. - - -Monitoring configuration ------------------------- - -Parsl monitoring is only supported with the `parsl.executors.HighThroughputExecutor`. - -The following example shows how to enable monitoring in the Parsl -configuration. Here the `parsl.monitoring.MonitoringHub` is specified to use port -55055 to receive monitoring messages from workers every 10 seconds. - -.. code-block:: python - - import parsl - from parsl.monitoring.monitoring import MonitoringHub - from parsl.config import Config - from parsl.executors import HighThroughputExecutor - from parsl.addresses import address_by_hostname - - import logging - - config = Config( - executors=[ - HighThroughputExecutor( - label="local_htex", - cores_per_worker=1, - max_workers_per_node=4, - address=address_by_hostname(), - ) - ], - monitoring=MonitoringHub( - hub_address=address_by_hostname(), - monitoring_debug=False, - resource_monitoring_interval=10, - ), - strategy='none' - ) - - -Visualization -------------- - -To run the web dashboard utility ``parsl-visualize`` you first need to install -its dependencies: - - $ pip install 'parsl[monitoring,visualization]' - -To view the web dashboard while or after a Parsl program has executed, run -the ``parsl-visualize`` utility:: - - $ parsl-visualize - -By default, this command expects that the default ``monitoring.db`` database is used -in the runinfo directory. Other databases can be loaded by passing -the database URI on the command line. For example, if the full path -to the database is ``/tmp/my_monitoring.db``, run:: - - $ parsl-visualize sqlite:////tmp/my_monitoring.db - -By default, the visualization web server listens on ``127.0.0.1:8080``. If the web server is deployed on a machine with a web browser, the dashboard can be accessed in the browser at ``127.0.0.1:8080``. If the web server is deployed on a remote machine, such as the login node of a cluster, you will need to use an ssh tunnel from your local machine to the cluster:: - - $ ssh -L 50000:127.0.0.1:8080 username@cluster_address - -This command will bind your local machine's port 50000 to the remote cluster's port 8080. -The dashboard can then be accessed via the local machine's browser at ``127.0.0.1:50000``. - -.. warning:: Alternatively you can deploy the visualization server on a public interface. However, first check that this is allowed by the cluster's security policy. The following example shows how to deploy the web server on a public port (i.e., open to Internet via ``public_IP:55555``):: - - $ parsl-visualize --listen 0.0.0.0 --port 55555 - - -Workflows Page -^^^^^^^^^^^^^^ - -The workflows page lists all Parsl workflows that have been executed with monitoring enabled -with the selected database. -It provides a high level summary of workflow state as shown below: - -.. image:: ../images/mon_workflows_page.png - -Throughout the dashboard, all blue elements are clickable. For example, clicking a specific worklow -name from the table takes you to the Workflow Summary page described in the next section. - -Workflow Summary -^^^^^^^^^^^^^^^^ - -The workflow summary page captures the run level details of a workflow, including start and end times -as well as task summary statistics. The workflow summary section is followed by the *App Summary* that lists -the various apps and invocation count for each. - -.. image:: ../images/mon_workflow_summary.png - - -The workflow summary also presents three different views of the workflow: - -* Workflow DAG - with apps differentiated by colors: This visualization is useful to visually inspect the dependency - structure of the workflow. Hovering over the nodes in the DAG shows a tooltip for the app represented by the node and it's task ID. - -.. image:: ../images/mon_task_app_grouping.png - -* Workflow DAG - with task states differentiated by colors: This visualization is useful to identify what tasks have been completed, failed, or are currently pending. - -.. image:: ../images/mon_task_state_grouping.png - -* Workflow resource usage: This visualization provides resource usage information at the workflow level. - For example, cumulative CPU/Memory utilization across workers over time. - -.. image:: ../images/mon_resource_summary.png +Redirect +-------- +This page has been `moved `_ diff --git a/docs/userguide/mpi_apps.rst b/docs/userguide/mpi_apps.rst index 82123123b6..3f758190ee 100644 --- a/docs/userguide/mpi_apps.rst +++ b/docs/userguide/mpi_apps.rst @@ -1,153 +1,9 @@ -MPI and Multi-node Apps -======================= +:orphan: -The :class:`~parsl.executors.MPIExecutor` supports running MPI applications or other computations which can -run on multiple compute nodes. +.. meta:: + :content http-equiv="refresh": 0;url=apps/mpi_apps.html -Background ----------- +Redirect +-------- -MPI applications run multiple copies of a program that complete a single task by -coordinating using messages passed within or across nodes. - -Starting MPI application requires invoking a "launcher" code (e.g., ``mpiexec``) -with options that define how the copies of a program should be distributed. - -The launcher includes options that control how copies of the program are distributed -across the nodes (e.g., how many copies per node) and -how each copy is configured (e.g., which CPU cores it can use). - -The options for launchers vary between MPI implementations and compute clusters. - -Configuring ``MPIExecutor`` ---------------------------- - -The :class:`~parsl.executors.MPIExecutor` is a wrapper over -:class:`~parsl.executors.high_throughput.executor.HighThroughputExecutor` -which eliminates options that are irrelevant for MPI applications. - -Define a configuration for :class:`~parsl.executors.MPIExecutor` by - -1. Setting ``max_workers_per_block`` to the maximum number of tasks to run per block of compute nodes. - This value is typically the number of nodes per block divided by the number of nodes per task. -2. Setting ``mpi_launcher`` to the launcher used for your application. -3. Specifying a provider that matches your cluster and use the :class:`~parsl.launchers.SimpleLauncher`, - which will ensure that no Parsl processes are placed on the compute nodes. - -An example for ALCF's Polaris supercomputer that will run 3 MPI tasks of 2 nodes each at the same time: - -.. code-block:: python - - config = Config( - executors=[ - MPIExecutor( - address=address_by_interface('bond0'), - max_workers_per_block=3, # Assuming 2 nodes per task - provider=PBSProProvider( - account="parsl", - worker_init=f"""module load miniconda; source activate /lus/eagle/projects/parsl/env""", - walltime="1:00:00", - queue="debug", - scheduler_options="#PBS -l filesystems=home:eagle:grand", - launcher=SimpleLauncher(), - select_options="ngpus=4", - nodes_per_block=6, - max_blocks=1, - cpus_per_node=64, - ), - ), - ] - ) - - -.. warning:: - Please note that ``Provider`` options that specify per-task or per-node resources, for example, - ``SlurmProvider(cores_per_node=N, ...)`` should not be used with :class:`~parsl.executors.high_throughput.MPIExecutor`. - Parsl primarily uses a pilot job model and assumptions from that context do not translate to the MPI context. For - more info refer to : - `github issue #3006 `_ - -Writing an MPI App ------------------- - -:class:`~parsl.executors.high_throughput.MPIExecutor` can execute both Python or Bash Apps which invoke an MPI application. - -Create the app by first defining a function which includes ``parsl_resource_specification`` keyword argument. -The resource specification is a dictionary which defines the number of nodes and ranks used by the application: - -.. code-block:: python - - resource_specification = { - 'num_nodes': , # Number of nodes required for the application instance - 'ranks_per_node': , # Number of ranks / application elements to be launched per node - 'num_ranks': , # Number of ranks in total - } - -Then, replace the call to the MPI launcher with ``$PARSL_MPI_PREFIX``. -``$PARSL_MPI_PREFIX`` references an environmental variable which will be replaced with -the correct MPI launcher configured for the resource list provided when calling the function -and with options that map the task to nodes which Parsl knows to be available. - -The function can be a Bash app - -.. code-block:: python - - @bash_app - def lammps_mpi_application(infile: File, parsl_resource_specification: Dict): - # PARSL_MPI_PREFIX will resolve to `mpiexec -n 4 -ppn 2 -hosts NODE001,NODE002` - return f"$PARSL_MPI_PREFIX lmp_mpi -in {infile.filepath}" - - -or a Python app: - -.. code-block:: python - - @python_app - def lammps_mpi_application(infile: File, parsl_resource_specification: Dict): - from subprocess import run - with open('stdout.lmp', 'w') as fp, open('stderr.lmp', 'w') as fe: - proc = run(['$PARSL_MPI_PREFIX', '-i', 'in.lmp'], stdout=fp, stderr=fe) - return proc.returncode - - -Run either App by calling with its arguments and a resource specification which defines how to execute it - -.. code-block:: python - - # Resources in terms of nodes and how ranks are to be distributed are set on a per app - # basis via the resource_spec dictionary. - resource_spec = { - "num_nodes": 2, - "ranks_per_node": 2, - "num_ranks": 4, - } - future = lammps_mpi_application(File('in.file'), parsl_resource_specification=resource_spec) - -Advanced: More Environment Variables -++++++++++++++++++++++++++++++++++++ - -Parsl Apps which run using :class:`~parsl.executors.high_throughput.MPIExecutor` -can make their own MPI invocation using other environment variables. - -These other variables include versions of the launch command for different launchers - -- ``PARSL_MPIEXEC_PREFIX``: mpiexec launch command which works for a large number of batch systems especially PBS systems -- ``PARSL_SRUN_PREFIX``: srun launch command for Slurm based clusters -- ``PARSL_APRUN_PREFIX``: aprun launch command prefix for some Cray machines - -And the information used by Parsl when assembling the launcher commands: - -- ``PARSL_NUM_RANKS``: Total number of ranks to use for the MPI application -- ``PARSL_NUM_NODES``: Number of nodes to use for the calculation -- ``PARSL_MPI_NODELIST``: List of assigned nodes separated by commas (Eg, NODE1,NODE2) -- ``PARSL_RANKS_PER_NODE``: Number of ranks per node - -Limitations -+++++++++++ - -Support for MPI tasks in HTEX is limited. It is designed for running many multi-node MPI applications within a single -batch job. - -#. MPI tasks may not span across nodes from more than one block. -#. Parsl does not correctly determine the number of execution slots per block (`Issue #1647 `_) -#. The executor uses a Python process per task, which can use a lot of memory (`Issue #2264 `_) \ No newline at end of file +This page has been `moved `_ diff --git a/docs/userguide/overview.rst b/docs/userguide/overview.rst index 073cc202e6..a83d9875e5 100644 --- a/docs/userguide/overview.rst +++ b/docs/userguide/overview.rst @@ -1,8 +1,8 @@ Overview ======== -Parsl is designed to enable straightforward parallelism and orchestration of asynchronous -tasks into dataflow-based workflows, in Python. Parsl manages the concurrent execution of +Parsl is designed to enable straightforward parallelism and orchestration of asynchronous +tasks into dataflow-based workflows, in Python. Parsl manages the concurrent execution of these tasks across various computation resources, from laptops to supercomputers, scheduling each task only when its dependencies (e.g., input data dependencies) are met. @@ -28,7 +28,7 @@ Parsl and Concurrency Any call to a Parsl app creates a new task that executes concurrently with the main program and any other task(s) that are currently executing. Different tasks may execute on the same nodes or on different nodes, and on the same or -different computers. +different computers. The Parsl execution model thus differs from the Python native execution model, which is inherently sequential. A Python program that does not contain Parsl @@ -42,13 +42,13 @@ main program resumes only after the function returns. .. image:: ../images/overview/python-concurrency.png :scale: 70 - :align: center + :align: center In contrast, the Parsl execution model is inherently concurrent. Whenever a program calls an app, a separate thread of execution is created, and the main program continues without pausing. Thus in the example shown in the figure below. There is initially a single task: the main program (black). The first -call to ``double`` creates a second task (red) and the second call to ``double`` +call to ``double`` creates a second task (red) and the second call to ``double`` creates a third task (orange). The second and third task terminate as the function that they execute returns. (The dashed lines represent the start and finish of the tasks). The calling program will only block (wait) when it is @@ -70,28 +70,28 @@ Parsl and Execution We have now seen that Parsl tasks are executed concurrently alongside the main Python program and other Parsl tasks. We now turn to the question of how and where are those tasks executed. Given the range of computers on which parallel -programs may be executed, Parsl allows tasks to be executed using different -executors (:py:class:`parsl.executors`). Executors are responsible for taking a queue of tasks and executing +programs may be executed, Parsl allows tasks to be executed using different +executors (:py:class:`parsl.executors`). Executors are responsible for taking a queue of tasks and executing them on local or remote resources. -We briefly describe two of Parsl's most commonly used executors. +We briefly describe two of Parsl's most commonly used executors. Other executors are described in :ref:`label-execution`. -The `parsl.executors.HighThroughputExecutor` (HTEX) implements a *pilot job model* that enables -fine-grain task execution using across one or more provisioned nodes. -HTEX can be used on a single node (e.g., a laptop) and will make use of +The `parsl.executors.HighThroughputExecutor` (HTEX) implements a *pilot job model* that enables +fine-grain task execution using across one or more provisioned nodes. +HTEX can be used on a single node (e.g., a laptop) and will make use of multiple processes for concurrent execution. -As shown in the following figure, HTEX uses Parsl's provider abstraction (:py:class:`parsl.providers`) to -communicate with a resource manager (e.g., batch scheduler or cloud API) to +As shown in the following figure, HTEX uses Parsl's provider abstraction (:py:class:`parsl.providers`) to +communicate with a resource manager (e.g., batch scheduler or cloud API) to provision a set of nodes (e.g., Parsl will use Slurm’s sbatch command to request -nodes on a Slurm cluster) for the duration of execution. -HTEX deploys a lightweight worker agent on the nodes which subsequently connects -back to the main Parsl process. Parsl tasks are then sent from the main program -to the connected workers for execution and the results are sent back via the -same mechanism. This approach has a number of advantages over other methods: -it avoids long job scheduler queue delays by acquiring one set of resources -for the entire program and it allows for scheduling of many tasks on individual -nodes. +nodes on a Slurm cluster) for the duration of execution. +HTEX deploys a lightweight worker agent on the nodes which subsequently connects +back to the main Parsl process. Parsl tasks are then sent from the main program +to the connected workers for execution and the results are sent back via the +same mechanism. This approach has a number of advantages over other methods: +it avoids long job scheduler queue delays by acquiring one set of resources +for the entire program and it allows for scheduling of many tasks on individual +nodes. .. image:: ../images/overview/htex-model.png @@ -101,13 +101,13 @@ nodes. back to the main Parsl process. Thus, you should verify that there is network connectivity between the workers and the Parsl process and ensure that the correct network address is used by the workers. Parsl provides a helper - function to automatically detect network addresses + function to automatically detect network addresses (`parsl.addresses.address_by_query`). -The `parsl.executors.ThreadPoolExecutor` allows tasks to be executed on a pool of locally -accessible threads. As execution occurs on the same computer, on a pool of -threads forked from the main program, the tasks share memory with one another +The `parsl.executors.ThreadPoolExecutor` allows tasks to be executed on a pool of locally +accessible threads. As execution occurs on the same computer, on a pool of +threads forked from the main program, the tasks share memory with one another (this is discussed further in the following sections). @@ -115,14 +115,14 @@ Parsl and Communication ----------------------- Parsl tasks typically need to communicate in order to perform useful work. Parsl provides for two forms of communication: by parameter passing -and by file passing. +and by file passing. As described in the next section, Parsl programs may also communicate by -interacting with shared filesystems and services its environment. +interacting with shared filesystems and services its environment. Parameter Passing ^^^^^^^^^^^^^^^^^ -The figure above illustrates communication via parameter passing. +The figure above illustrates communication via parameter passing. The call ``double(3)`` to the app ``double`` in the main program creates a new task and passes the parameter value, 3, to that new task. When the task completes execution, its return value, 6, is returned to the main program. Similarly, the @@ -133,27 +133,27 @@ passed to/from tasks. File Passing ^^^^^^^^^^^^ -Parsl supports communication via files in both Bash apps and Python apps. -Files may be used in place of parameter passing for many reasons, such as for -apps are designed to support files, when data to be exchanged are large, -or when data cannot be easily serialized into Python objects. -As Parsl tasks may be executed on remote nodes, without shared file systems, -Parsl offers a Parsl :py:class:`parsl.data_provider.files.File` construct for location-independent reference +Parsl supports communication via files in both Bash apps and Python apps. +Files may be used in place of parameter passing for many reasons, such as for +apps are designed to support files, when data to be exchanged are large, +or when data cannot be easily serialized into Python objects. +As Parsl tasks may be executed on remote nodes, without shared file systems, +Parsl offers a Parsl :py:class:`parsl.data_provider.files.File` construct for location-independent reference to files. Parsl will translate file objects to worker-accessible paths when executing dependent apps. Parsl is also able to transfer files in, out, and between Parsl -apps using one of several methods (e.g., FTP, HTTP(S), Globus and rsync). -To accommodate the asynchronous nature of file transfer, Parsl treats +apps using one of several methods (e.g., FTP, HTTP(S), Globus and rsync). +To accommodate the asynchronous nature of file transfer, Parsl treats data movement like a Parsl app, adding a dependency to the execution graph -and waiting for transfers to complete before executing dependent apps. +and waiting for transfers to complete before executing dependent apps. More information is provided in :ref:`label-data`). Futures ^^^^^^^ -Communication via parameter and file passing also serves a second purpose, namely +Communication via parameter and file passing also serves a second purpose, namely synchronization. As we discuss in more detail in :ref:`label-futures`, a call to an -app returns a special object called a future that has a special unassigned -state until such time as the app returns, at which time it takes the return +app returns a special object called a future that has a special unassigned +state until such time as the app returns, at which time it takes the return value. (In the example program, two futures are thus created, d1 and d2.) The AppFuture function result() blocks until the future to which it is applied takes a value. Thus the print statement in the main program blocks until both child @@ -168,16 +168,16 @@ and when those values are available, is active again. The Parsl Environment --------------------- Regular Python and Parsl-enhanced Python differ in terms of the environment in -which code executes. We use the term *environment* here to refer to the -variables and modules (the *memory environment*), the file system(s) -(the *file system environment*), and the services (the *service environment*) +which code executes. We use the term *environment* here to refer to the +variables and modules (the *memory environment*), the file system(s) +(the *file system environment*), and the services (the *service environment*) that are accessible to a function. -An important question when it comes to understanding the behavior of Parsl -programs is the environment in which this new task executes: does it have the -same or different memory, file system, or service environment as its parent -task or any other task? The answer, depends on the executor used, and (in the -case of the file system environment) where the task executes. +An important question when it comes to understanding the behavior of Parsl +programs is the environment in which this new task executes: does it have the +same or different memory, file system, or service environment as its parent +task or any other task? The answer, depends on the executor used, and (in the +case of the file system environment) where the task executes. Below we describe behavior for the most commonly used `parsl.executors.HighThroughputExecutor` which is representative of all Parsl executors except the `parsl.executors.ThreadPoolExecutor`. @@ -186,13 +186,13 @@ which is representative of all Parsl executors except the `parsl.executors.Threa it allows tasks to share memory. Memory environment -^^^^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^^^ -In Python, the variables and modules that are accessible to a function are defined -by Python scoping rules, by which a function has access to both variables defined -within the function (*local* variables) and those defined outside the function -(*global* variables). Thus in the following code, the print statement in the -print_answer function accesses the global variable "answer", and we see as output +In Python, the variables and modules that are accessible to a function are defined +by Python scoping rules, by which a function has access to both variables defined +within the function (*local* variables) and those defined outside the function +(*global* variables). Thus in the following code, the print statement in the +print_answer function accesses the global variable "answer", and we see as output "the answer is 42." .. code-block:: python @@ -206,17 +206,17 @@ print_answer function accesses the global variable "answer", and we see as outpu In Parsl (except when using the `parsl.executors.ThreadPoolExecutor`) a Parsl app is executed -in a distinct environment that only has access to local variables associated -with the app function. Thus, if the program above is executed with say the +in a distinct environment that only has access to local variables associated +with the app function. Thus, if the program above is executed with say the `parsl.executors.HighThroughputExecutor`, will print "the answer is 0" rather than "the answer -is 42," because the print statement in provide_answer does not have access to +is 42," because the print statement in provide_answer does not have access to the global variable that has been assigned the value 42. The program will run without errors when using the `parsl.executors.ThreadPoolExecutor`. -Similarly, the same scoping rules apply to import statements, and thus -the following program will run without errors with the `parsl.executors.ThreadPoolExecutor`, -but raise errors when run with any other executor, because the return statement -in ``ambiguous_double`` refers to a variable (factor) and a module (random) that are +Similarly, the same scoping rules apply to import statements, and thus +the following program will run without errors with the `parsl.executors.ThreadPoolExecutor`, +but raise errors when run with any other executor, because the return statement +in ``ambiguous_double`` refers to a variable (factor) and a module (random) that are not known to the function. .. code-block:: python @@ -229,9 +229,9 @@ not known to the function. return x * random.random() * factor print(ambiguous_double(42)) - -To allow this program to run correctly with all Parsl executors, the random + +To allow this program to run correctly with all Parsl executors, the random library must be imported within the app, and the factor variable must be passed as an argument, as follows. @@ -248,12 +248,12 @@ passed as an argument, as follows. print(good_double(factor, 42)) -File system environment +File system environment ^^^^^^^^^^^^^^^^^^^^^^^ -In a regular Python program the environment that is accessible to a Python -program also includes the file system(s) of the computer on which it is -executing. +In a regular Python program the environment that is accessible to a Python +program also includes the file system(s) of the computer on which it is +executing. Thus in the following code, a value written to a file "answer.txt" in the current directory can be retrieved by reading the same file, and the print statement outputs "the answer is 42." @@ -272,15 +272,15 @@ statement outputs "the answer is 42." The question of which file system environment is accessible to a Parsl app -depends on where the app executes. If two tasks run on nodes that share a -file system, then those tasks (e.g., tasks A and B in the figure below, -but not task C) share a file system environment. Thus the program above will -output "the answer is 42" if the parent task and the child task run on +depends on where the app executes. If two tasks run on nodes that share a +file system, then those tasks (e.g., tasks A and B in the figure below, +but not task C) share a file system environment. Thus the program above will +output "the answer is 42" if the parent task and the child task run on nodes 1 and 2, but not if they run on nodes 2 and 3. .. image:: ../images/overview/filesystem.png :scale: 70 - :align: center + :align: center Service Environment ^^^^^^^^^^^^^^^^^^^ @@ -292,7 +292,7 @@ service. These services are accessible to any task. Environment Summary ^^^^^^^^^^^^^^^^^^^ -As we summarize in the table, if tasks execute with the `parsl.executors.ThreadPoolExecutor`, +As we summarize in the table, if tasks execute with the `parsl.executors.ThreadPoolExecutor`, they share the memory and file system environment of the parent task. If they execute with any other executor, they have a separate memory environment, and may or may not share their file system environment with other tasks, depending @@ -302,7 +302,7 @@ services. +--------------------+--------------------+--------------------+---------------------------+------------------+ | | Share memory | Share file system | Share file system | Share service | | | environment with | environment with | environment with other | environment | -| | parent/other tasks | parent | tasks | with other tasks | +| | parent/other tasks | parent | tasks | with other tasks | +====================+====================+====================+===========================+==================+ +--------------------+--------------------+--------------------+---------------------------+------------------+ | Python | Yes | Yes | N/A | N/A | diff --git a/docs/userguide/parsl_perf.rst b/docs/userguide/parsl_perf.rst index 2ea1adb00f..88c4f1a20c 100644 --- a/docs/userguide/parsl_perf.rst +++ b/docs/userguide/parsl_perf.rst @@ -1,53 +1,9 @@ -.. _label-parsl-perf: +:orphan: -Measuring performance with parsl-perf -===================================== +.. meta:: + :content http-equiv="refresh": 0;url=advanced/parsl_perf.html -``parsl-perf`` is tool for making basic performance measurements of Parsl -configurations. +Redirect +-------- -It runs increasingly large numbers of no-op apps until a batch takes -(by default) 120 seconds, giving a measurement of tasks per second. - -This can give a basic measurement of some of the overheads in task -execution. - -``parsl-perf`` must be invoked with a configuration file, which is a Python -file containing a variable ``config`` which contains a `Config` object, or -a function ``fresh_config`` which returns a `Config` object. The -``fresh_config`` format is the same as used with the pytest test suite. - -To specify a ``parsl_resource_specification`` for tasks, add a ``--resources`` -argument. - -To change the target runtime from the default of 120 seconds, add a -``--time`` parameter. - -For example: - -.. code-block:: bash - - - $ python -m parsl.benchmark.perf --config parsl/tests/configs/workqueue_ex.py --resources '{"cores":1, "memory":0, "disk":0}' - ==== Iteration 1 ==== - Will run 10 tasks to target 120 seconds runtime - Submitting tasks / invoking apps - warning: using plain-text when communicating with workers. - warning: use encryption with a key and cert when creating the manager. - All 10 tasks submitted ... waiting for completion - Submission took 0.008 seconds = 1248.676 tasks/second - Runtime: actual 3.668s vs target 120s - Tasks per second: 2.726 - - [...] - - ==== Iteration 4 ==== - Will run 57640 tasks to target 120 seconds runtime - Submitting tasks / invoking apps - All 57640 tasks submitted ... waiting for completion - Submission took 34.839 seconds = 1654.487 tasks/second - Runtime: actual 364.387s vs target 120s - Tasks per second: 158.184 - Cleaning up DFK - The end - +This page has been `moved `_ diff --git a/docs/userguide/plugins.rst b/docs/userguide/plugins.rst index cd9244960c..2f4cdbcb60 100644 --- a/docs/userguide/plugins.rst +++ b/docs/userguide/plugins.rst @@ -1,106 +1,9 @@ -Plugins -======= +:orphan: -Parsl has several places where code can be plugged in. Parsl usually provides -several implementations that use each plugin point. +.. meta:: + :content http-equiv="refresh": 0;url=advanced/plugins.html -This page gives a brief summary of those places and why you might want -to use them, with links to the API guide. +Redirect +-------- -Executors ---------- -When the parsl dataflow kernel is ready for a task to run, it passes that -task to an `ParslExecutor`. The executor is then responsible for running the task's -Python code and returning the result. This is the abstraction that allows one -executor to run code on the local submitting host, while another executor can -run the same code on a large supercomputer. - - -Providers and Launchers ------------------------ -Some executors are based on blocks of workers (for example the -`parsl.executors.HighThroughputExecutor`: the submit side requires a -batch system (eg slurm, kubernetes) to start worker processes, which then -execute tasks. - -The particular way in which a system makes those workers start is implemented -by providers and launchers. - -An `ExecutionProvider` allows a command line to be submitted as a request to the -underlying batch system to be run inside an allocation of nodes. - -A `Launcher` modifies that command line when run inside the allocation to -add on any wrappers that are needed to launch the command (eg srun inside -slurm). Providers and launchers are usually paired together for a particular -system type. - -File staging ------------- -Parsl can copy input files from an arbitrary URL into a task's working -environment, and copy output files from a task's working environment to -an arbitrary URL. A small set of data staging providers is installed by default, -for ``file://`` ``http://`` and ``ftp://`` URLs. More data staging providers can -be added in the workflow configuration, in the ``storage`` parameter of the -relevant `ParslExecutor`. Each provider should subclass the `Staging` class. - - -Default stdout/stderr name generation -------------------------------------- -Parsl can choose names for your bash apps stdout and stderr streams -automatically, with the parsl.AUTO_LOGNAME parameter. The choice of path is -made by a function which can be configured with the ``std_autopath`` -parameter of Parsl `Config`. By default, ``DataFlowKernel.default_std_autopath`` -will be used. - - -Memoization/checkpointing -------------------------- - -When parsl memoizes/checkpoints an app parameter, it does so by computing a -hash of that parameter that should be the same if that parameter is the same -on subsequent invocations. This isn't straightforward to do for arbitrary -objects, so parsl implements a checkpointing hash function for a few common -types, and raises an exception on unknown types: - -.. code-block:: - - ValueError("unknown type for memoization ...") - -You can plug in your own type-specific hash code for additional types that -you need and understand using `id_for_memo`. - - -Invoking other asynchronous components --------------------------------------- - -Parsl code can invoke other asynchronous components which return Futures, and -integrate those Futures into the task graph: Parsl apps can be given any -`concurrent.futures.Future` as a dependency, even if those futures do not come -from invoking a Parsl app. This includes as the return value of a -``join_app``. - -An specific example of this is integrating Globus Compute tasks into a Parsl -task graph. See :ref:`label-join-globus-compute` - -Dependency resolution ---------------------- - -When Parsl examines the arguments to an app, it uses a `DependencyResolver`. -The default `DependencyResolver` will cause Parsl to wait for -``concurrent.futures.Future`` instances (including `AppFuture` and -`DataFuture`), and pass through other arguments without waiting. - -This behaviour is pluggable: Parsl comes with another dependency resolver, -`DEEP_DEPENDENCY_RESOLVER` which knows about futures contained with structures -such as tuples, lists, sets and dicts. - -This plugin interface might be used to interface other task-like or future-like -objects to the Parsl dependency mechanism, by describing how they can be -interpreted as a Future. - -Removed interfaces ------------------- - -Parsl had a deprecated ``Channel`` abstraction. See -`issue 3515 `_ -for further discussion on its removal. +This page has been `moved `_ diff --git a/docs/userguide/usage_tracking.rst b/docs/userguide/usage_tracking.rst index da8ac9b79d..1b891ce0e9 100644 --- a/docs/userguide/usage_tracking.rst +++ b/docs/userguide/usage_tracking.rst @@ -1,171 +1,9 @@ -.. _label-usage-tracking: +:orphan: -Usage Statistics Collection -=========================== +.. meta:: + :content http-equiv="refresh": 0;url=advanced/usage_tracking.html -Parsl uses an **Opt-in** model for usage tracking, allowing users to decide if they wish to participate. Usage statistics are crucial for improving software reliability and help focus development and maintenance efforts on the most used components of Parsl. The collected data is used solely for enhancements and reporting and is not shared in its raw form outside of the Parsl team. - -Why are we doing this? ----------------------- - -The Parsl development team relies on funding from government agencies. To sustain this funding and advocate for continued support, it is essential to show that the research community benefits from these investments. - -By opting in to share usage data, you actively support the ongoing development and maintenance of Parsl. (See:ref:`What is sent? ` below). - -Opt-In Model ------------- - -We use an **opt-in model** for usage tracking to respect user privacy and provide full control over shared information. We hope that developers and researchers will choose to send us this information. The reason is that we need this data - it is a requirement for funding. - -Choose the data you share with Usage Tracking Levels. - -**Usage Tracking Levels:** - -* **Level 1:** Only basic information such as Python version, Parsl version, and platform name (Linux, MacOS, etc.) -* **Level 2:** Level 1 information and configuration information including provider, executor, and launcher names. -* **Level 3:** Level 2 information and workflow execution details, including the number of applications run, failures, and execution time. - -By enabling usage tracking, you support Parsl's development. - -**To opt-in, set** ``usage_tracking`` **to the desired level (1, 2, or 3) in the configuration object** (``parsl.config.Config``) **.** - -Example: - -.. code-block:: python3 - - config = Config( - executors=[ - HighThroughputExecutor( - ... - ) - ], - usage_tracking=3 - ) - -.. _what-is-sent: - -What is sent? -------------- - -The data collected depends on the tracking level selected: - -* **Level 1:** Only basic information such as Python version, Parsl version, and platform name (Linux, MacOS, etc.) -* **Level 2:** Level 1 information and configuration information including provider, executor, and launcher names. -* **Level 3:** Level 2 information and workflow execution details, including the number of applications run, failures, and execution time. - -**Example Messages:** - -- At launch: - - .. code-block:: json - - { - "correlator":"6bc7484e-5693-48b2-b6c0-5889a73f7f4e", - "parsl_v":"1.3.0-dev", - "python_v":"3.12.2", - "platform.system":"Darwin", - "tracking_level":3, - "components":[ - { - "c":"parsl.config.Config", - "executors_len":1, - "dependency_resolver":false - }, - "parsl.executors.threads.ThreadPoolExecutor" - ], - "start":1727156153 - } - -- On closure (Tracking Level 3 only): - - .. code-block:: json - - { - "correlator":"6bc7484e-5693-48b2-b6c0-5889a73f7f4e", - "execution_time":31, - "components":[ - { - "c":"parsl.dataflow.dflow.DataFlowKernel", - "app_count":3, - "app_fails":0 - }, - { - "c":"parsl.config.Config", - "executors_len":1, - "dependency_resolver":false - }, - "parsl.executors.threads.ThreadPoolExecutor" - ], - "end":1727156156 - } - -**All messages sent are logged in the** ``parsl.log`` **file, ensuring complete transparency.** - -How is the data sent? ---------------------- - -Data is sent using **UDP** to minimize the impact on workflow performance. While this may result in some data loss, it significantly reduces the chances of usage tracking affecting the software's operation. - -The data is processed through AWS CloudWatch to generate a monitoring dashboard, providing valuable insights into usage patterns. - -When is the data sent? ----------------------- - -Data is sent twice per run: - -1. At the start of the script. -2. Upon script completion (for Tracking Level 3). - -What will the data be used for? -------------------------------- - -The data will help the Parsl team understand Parsl usage and make development and maintenance decisions, including: - -* Focus development and maintenance on the most-used components of Parsl. -* Determine which Python versions to continue supporting. -* Track the age of Parsl installations. -* Assess how long it takes for most users to adopt new changes. -* Track usage statistics to report to funders. - -Usage Statistics Dashboard --------------------------- - -The collected data is aggregated and displayed on a publicly accessible dashboard. This dashboard provides an overview of how Parsl is being used across different environments and includes metrics such as: - -* Total workflows executed over time -* Most-used Python and Parsl versions -* Most common platforms and executors and more - -`Find the dashboard here `_ - -Leaderboard ------------ - -**Opting in to usage tracking also allows you to participate in the Parsl Leaderboard. -To participate in the leaderboard, you can deanonymize yourself using the** ``project_name`` **parameter in the parsl configuration object** (``parsl.config.Config``) **.** - -`Find the Parsl Leaderboard here `_ - -Example: - -.. code-block:: python3 - - config = Config( - executors=[ - HighThroughputExecutor( - ... - ) - ], - usage_tracking=3, - project_name="my-test-project" - ) - -Every run of parsl with usage tracking **Level 1** or **Level 2** earns you **1 point**. And every run with usage tracking **Level 3**, earns you **2 points**. - -Feedback +Redirect -------- -Please send us your feedback at parsl@googlegroups.com. Feedback from our user communities will be -useful in determining our path forward with usage tracking in the future. - -**Please consider turning on usage tracking to support the continued development of Parsl.** +This page has been `moved `_ diff --git a/docs/userguide/workflow.rst b/docs/userguide/workflow.rst index 2a0a2c8c28..139dd79175 100644 --- a/docs/userguide/workflow.rst +++ b/docs/userguide/workflow.rst @@ -1,243 +1,9 @@ -.. _label-workflow: +:orphan: -Example parallel patterns -========================= +.. meta:: + :content http-equiv="refresh": 0;url=workflows/workflow.html -Parsl can be used to implement a wide range of parallel programming patterns, from bag of tasks -through to nested workflows. Parsl implicitly assembles a dataflow -dependency graph based on the data shared between apps. -The flexibility of this model allows for the implementation of a wide range -of parallel programming and workflow patterns. +Redirect +-------- -Parsl is also designed to address broad execution requirements, from programs -that run many short tasks to those that run a few long tasks. - -Below we illustrate a range of parallel programming and workflow patterns. It is important -to note that this set of examples is by no means comprehensive. - - -Bag of Tasks ------------- -Parsl can be used to execute a large bag of tasks. In this case, Parsl -assembles the set of tasks (represented as Parsl apps) and manages their concurrent -execution on available resources. - -.. code-block:: python - - from parsl import python_app - - parsl.load() - - # Map function that returns double the input integer - @python_app - def app_random(): - import random - return random.random() - - results = [] - for i in range(0, 10): - x = app_random() - results.append(x) - - for r in results: - print(r.result()) - - -Sequential workflows --------------------- - -Sequential workflows can be created by passing an AppFuture from one task to another. For example, in the following program the ``generate`` app (a Python app) generates a random number that is consumed by the ``save`` app (a Bash app), which writes it to a file. Because ``save`` cannot execute until it receives the ``message`` produced by ``generate``, the two apps execute in sequence. - -.. code-block:: python - - from parsl import python_app - - parsl.load() - - # Generate a random number - @python_app - def generate(limit): - from random import randint - """Generate a random integer and return it""" - return randint(1, limit) - - # Write a message to a file - @bash_app - def save(message, outputs=()): - return 'echo {} &> {}'.format(message, outputs[0]) - - message = generate(10) - - saved = save(message, outputs=['output.txt']) - - with open(saved.outputs[0].result(), 'r') as f: - print(f.read()) - - -Parallel workflows ------------------- - -Parallel execution occurs automatically in Parsl, respecting dependencies among app executions. In the following example, three instances of the ``wait_sleep_double`` app are created. The first two execute concurrently, as they have no dependencies; the third must wait until the first two complete and thus the ``doubled_x`` and ``doubled_y`` futures have values. Note that this sequencing occurs even though ``wait_sleep_double`` does not in fact use its second and third arguments. - -.. code-block:: python - - from parsl import python_app - - parsl.load() - - @python_app - def wait_sleep_double(x, foo_1, foo_2): - import time - time.sleep(2) # Sleep for 2 seconds - return x*2 - - # Launch two apps, which will execute in parallel, since they do not have to - # wait on any futures - doubled_x = wait_sleep_double(10, None, None) - doubled_y = wait_sleep_double(10, None, None) - - # The third app depends on the first two: - # doubled_x doubled_y (2 s) - # \ / - # doublex_z (2 s) - doubled_z = wait_sleep_double(10, doubled_x, doubled_y) - - # doubled_z will be done in ~4s - print(doubled_z.result()) - - -Parallel workflows with loops ------------------------------ - -A common approach to executing Parsl apps in parallel is via loops. The following example uses a loop to create many random numbers in parallel. - -.. code-block:: python - - from parsl import python_app - - parsl.load() - - @python_app - def generate(limit): - """Generate a random integer and return it""" - from random import randint - return randint(1, limit) - - rand_nums = [] - for i in range(1,5): - rand_nums.append(generate(i)) - - # Wait for all apps to finish and collect the results - outputs = [r.result() for r in rand_nums] - -The :class:`~parsl.concurrent.ParslPoolExecutor` simplifies this pattern using the same interface as -`Python's native Executors `_. - -.. code-block:: python - - from parsl.concurrent import ParslPoolExecutor - from parsl.configs.htex_local import config - - # NOTE: Functions used by the ParslPoolExecutor do _not_ use decorators - def generate(limit): - """Generate a random integer and return it""" - from random import randint - return randint(1, limit) - - - with ParslPoolExecutor(config) as pool: - outputs = pool.map(generate, range(1, 5)) - - -In the preceding example, the execution of different tasks is coordinated by passing Python objects from producers to consumers. -In other cases, it can be convenient to pass data in files, as in the following reformulation. Here, a set of files, each with a random number, is created by the ``generate`` app. These files are then concatenated into a single file, which is subsequently used to compute the sum of all numbers. - -.. code-block:: python - - from parsl import python_app, bash_app - - parsl.load() - - @bash_app - def generate(outputs=()): - return 'echo $(( RANDOM % (10 - 5 + 1 ) + 5 )) &> {}'.format(outputs[0]) - - @bash_app - def concat(inputs=(), outputs=(), stdout='stdout.txt', stderr='stderr.txt'): - return 'cat {0} >> {1}'.format(' '.join(inputs), outputs[0]) - - @python_app - def total(inputs=()): - total = 0 - with open(inputs[0].filepath, 'r') as f: - for l in f: - total += int(l) - return total - - # Create 5 files with random numbers - output_files = [] - for i in range (5): - output_files.append(generate(outputs=['random-%s.txt' % i])) - - # Concatenate the files into a single file - cc = concat(inputs=[i.outputs[0] for i in output_files], outputs=['all.txt']) - - # Calculate the average of the random numbers - totals = total(inputs=[cc.outputs[0]]) - - print(totals.result()) - - -MapReduce ---------- -MapReduce is a common pattern used in data analytics. It is composed of a map phase -that filters values and a reduce phase that aggregates values. -The following example demonstrates how Parsl can be used to specify a MapReduce computation -in which the map phase doubles a set of input integers and the reduce phase computes -the sum of those results. - -.. code-block:: python - - from parsl import python_app - - parsl.load() - - # Map function that returns double the input integer - @python_app - def app_double(x): - return x*2 - - # Reduce function that returns the sum of a list - @python_app - def app_sum(inputs=()): - return sum(inputs) - - # Create a list of integers - items = range(0,4) - - # Map phase: apply the double *app* function to each item in list - mapped_results = [] - for i in items: - x = app_double(i) - mapped_results.append(x) - - # Reduce phase: apply the sum *app* function to the set of results - total = app_sum(inputs=mapped_results) - - print(total.result()) - -The program first defines two Parsl apps, ``app_double`` and ``app_sum``. -It then makes calls to the ``app_double`` app with a set of input -values. It then passes the results from ``app_double`` to the ``app_sum`` app -to aggregate values into a single result. -These tasks execute concurrently, synchronized by the ``mapped_results`` variable. -The following figure shows the resulting task graph. - -.. image:: ../images/MapReduce.png - -Caching expensive initialisation between tasks ----------------------------------------------- - -Many tasks in workflows require a expensive "initialization" steps that, once performed, can be used across successive invocations for that task. For example, you may want to reuse a machine learning model for multiple interface tasks and avoid loading it onto GPUs more than once. - -`This ExaWorks tutorial `_ gives examples of how to do this. +This page has been `moved `_ diff --git a/docs/userguide/workflows/checkpoints.rst b/docs/userguide/workflows/checkpoints.rst new file mode 100644 index 0000000000..8867107b7a --- /dev/null +++ b/docs/userguide/workflows/checkpoints.rst @@ -0,0 +1,299 @@ +.. _label-memos: + +Memoization and checkpointing +----------------------------- + +When an app is invoked several times with the same parameters, Parsl can +reuse the result from the first invocation without executing the app again. + +This can save time and computational resources. + +This is done in two ways: + +* Firstly, *app caching* will allow reuse of results within the same run. + +* Building on top of that, *checkpointing* will store results on the filesystem + and reuse those results in later runs. + +.. _label-appcaching: + +App caching +=========== + + +There are many situations in which a program may be re-executed +over time. Often, large fragments of the program will not have changed +and therefore, re-execution of apps will waste valuable time and +computation resources. Parsl's app caching solves this problem by +storing results from apps that have successfully completed +so that they can be re-used. + +App caching is enabled by setting the ``cache`` +argument in the :func:`~parsl.app.app.python_app` or :func:`~parsl.app.app.bash_app` +decorator to ``True`` (by default it is ``False``). + +.. code-block:: python + + @bash_app(cache=True) + def hello (msg, stdout=None): + return 'echo {}'.format(msg) + +App caching can be globally disabled by setting ``app_cache=False`` +in the :class:`~parsl.config.Config`. + +App caching can be particularly useful when developing interactive programs such as when +using a Jupyter notebook. In this case, cells containing apps are often re-executed +during development. Using app caching will ensure that only modified apps are re-executed. + + +App equivalence +^^^^^^^^^^^^^^^ + +Parsl determines app equivalence using the name of the app function: +if two apps have the same name, then they are equivalent under this +relation. + +Changes inside the app, or by functions called by an app will not invalidate +cached values. + +There are lots of other ways functions might be compared for equivalence, +and `parsl.dataflow.memoization.id_for_memo` provides a hook to plug in +alternate application-specific implementations. + + +Invocation equivalence +^^^^^^^^^^^^^^^^^^^^^^ + +Two app invocations are determined to be equivalent if their +input arguments are identical. + +In simple cases, this follows obvious rules: + +.. code-block:: python + + # these two app invocations are the same and the second invocation will + # reuse any cached input from the first invocation + x = 7 + f(x).result() + + y = 7 + f(y).result() + + +Internally, equivalence is determined by hashing the input arguments, and +comparing the hash to hashes from previous app executions. + +This approach can only be applied to data types for which a deterministic hash +can be computed. + +By default Parsl can compute sensible hashes for basic data types: +str, int, float, None, as well as more some complex types: +functions, and dictionaries and lists containing hashable types. + +Attempting to cache apps invoked with other, non-hashable, data types will +lead to an exception at invocation. + +In that case, mechanisms to hash new types can be registered by a program by +implementing the `parsl.dataflow.memoization.id_for_memo` function for +the new type. + +Ignoring arguments +^^^^^^^^^^^^^^^^^^ + +On occasion one may wish to ignore particular arguments when determining +app invocation equivalence - for example, when generating log file +names automatically based on time or run information. +Parsl allows developers to list the arguments to be ignored +in the ``ignore_for_cache`` app decorator parameter: + +.. code-block:: python + + @bash_app(cache=True, ignore_for_cache=['stdout']) + def hello (msg, stdout=None): + return 'echo {}'.format(msg) + + +Caveats +^^^^^^^ + +It is important to consider several important issues when using app caching: + +- Determinism: App caching is generally useful only when the apps are deterministic. + If the outputs may be different for identical inputs, app caching will obscure + this non-deterministic behavior. For instance, caching an app that returns + a random number will result in every invocation returning the same result. + +- Timing: If several identical calls to an app are made concurrently having + not yet cached a result, many instances of the app will be launched. + Once one invocation completes and the result is cached + all subsequent calls will return immediately with the cached result. + +- Performance: If app caching is enabled, there may be some performance + overhead especially if a large number of short duration tasks are launched rapidly. + This overhead has not been quantified. + +.. _label-checkpointing: + +Checkpointing +============= + +Large-scale Parsl programs are likely to encounter errors due to node failures, +application or environment errors, and myriad other issues. Parsl offers an +application-level checkpointing model to improve resilience, fault tolerance, and +efficiency. + +.. note:: + Checkpointing builds on top of app caching, and so app caching must be + enabled. If app caching is disabled in the config ``Config.app_cache``, checkpointing will + not work. + +Parsl follows an incremental checkpointing model, where each checkpoint file contains +all results that have been updated since the last checkpoint. + +When a Parsl program loads a checkpoint file and is executed, it will use +checkpointed results for any apps that have been previously executed. +Like app caching, checkpoints +use the hash of the app and the invocation input parameters to identify previously computed +results. If multiple checkpoints exist for an app (with the same hash) +the most recent entry will be used. + +Parsl provides four checkpointing modes: + +1. ``task_exit``: a checkpoint is created each time an app completes or fails + (after retries if enabled). This mode minimizes the risk of losing information + from completed tasks. + + .. code-block:: python + + from parsl.configs.local_threads import config + config.checkpoint_mode = 'task_exit' + +2. ``periodic``: a checkpoint is created periodically using a user-specified + checkpointing interval. Results will be saved to the checkpoint file for + all tasks that have completed during this period. + + .. code-block:: python + + from parsl.configs.local_threads import config + config.checkpoint_mode = 'periodic' + config.checkpoint_period = "01:00:00" + +3. ``dfk_exit``: checkpoints are created when Parsl is + about to exit. This reduces the risk of losing results due to + premature program termination from exceptions, terminate signals, etc. However + it is still possible that information might be lost if the program is + terminated abruptly (machine failure, SIGKILL, etc.) + + .. code-block:: python + + from parsl.configs.local_threads import config + config.checkpoint_mode = 'dfk_exit' + +4. ``manual``: in addition to these automated checkpointing modes, it is also possible + to manually initiate a checkpoint by calling ``DataFlowKernel.checkpoint()`` in the + Parsl program code. + + .. code-block:: python + + import parsl + from parsl.configs.local_threads import config + dfk = parsl.load(config) + .... + dfk.checkpoint() + +In all cases the checkpoint file is written out to the ``runinfo/RUN_ID/checkpoint/`` directory. + +.. Note:: Checkpoint modes ``periodic``, ``dfk_exit``, and ``manual`` can interfere with garbage collection. + In these modes task information will be retained after completion, until checkpointing events are triggered. + + +Creating a checkpoint +^^^^^^^^^^^^^^^^^^^^^ + +Automated checkpointing must be explicitly enabled in the Parsl configuration. +There is no need to modify a Parsl program as checkpointing will occur transparently. +In the following example, checkpointing is enabled at task exit. The results of +each invocation of the ``slow_double`` app will be stored in the checkpoint file. + +.. code-block:: python + + import parsl + from parsl.app.app import python_app + from parsl.configs.local_threads import config + + config.checkpoint_mode = 'task_exit' + + parsl.load(config) + + @python_app(cache=True) + def slow_double(x): + import time + time.sleep(5) + return x * 2 + + d = [] + for i in range(5): + d.append(slow_double(i)) + + print([d[i].result() for i in range(5)]) + +Alternatively, manual checkpointing can be used to explictly specify when the checkpoint +file should be saved. The following example shows how manual checkpointing can be used. +Here, the ``dfk.checkpoint()`` function will save the results of the prior invocations +of the ``slow_double`` app. + +.. code-block:: python + + import parsl + from parsl import python_app + from parsl.configs.local_threads import config + + dfk = parsl.load(config) + + @python_app(cache=True) + def slow_double(x, sleep_dur=1): + import time + time.sleep(sleep_dur) + return x * 2 + + N = 5 # Number of calls to slow_double + d = [] # List to store the futures + for i in range(0, N): + d.append(slow_double(i)) + + # Wait for the results + [i.result() for i in d] + + cpt_dir = dfk.checkpoint() + print(cpt_dir) # Prints the checkpoint dir + + +Resuming from a checkpoint +^^^^^^^^^^^^^^^^^^^^^^^^^^ + +When resuming a program from a checkpoint Parsl allows the user to select +which checkpoint file(s) to use. +Checkpoint files are stored in the ``runinfo/RUNID/checkpoint`` directory. + +The example below shows how to resume using all available checkpoints. +Here, the program re-executes the same calls to the ``slow_double`` app +as above and instead of waiting for results to be computed, the values +from the checkpoint file are are immediately returned. + +.. code-block:: python + + import parsl + from parsl.tests.configs.local_threads import config + from parsl.utils import get_all_checkpoints + + config.checkpoint_files = get_all_checkpoints() + + parsl.load(config) + + # Rerun the same workflow + d = [] + for i in range(5): + d.append(slow_double(i)) + + # wait for results + print([d[i].result() for i in range(5)]) diff --git a/docs/userguide/workflows/exceptions.rst b/docs/userguide/workflows/exceptions.rst new file mode 100644 index 0000000000..d18fbe704d --- /dev/null +++ b/docs/userguide/workflows/exceptions.rst @@ -0,0 +1,171 @@ +.. _label-exceptions: + +Error handling +============== + +Parsl provides various mechanisms to add resiliency and robustness to programs. + +Exceptions +---------- + +Parsl is designed to capture, track, and handle various errors occurring +during execution, including those related to the program, apps, execution +environment, and Parsl itself. +It also provides functionality to appropriately respond to failures during +execution. + +Failures might occur for various reasons: + +1. A task failed during execution. +2. A task failed to launch, for example, because an input dependency was not met. +3. There was a formatting error while formatting the command-line string in Bash apps. +4. A task completed execution but failed to produce one or more of its specified + outputs. +5. Task exceeded the specified walltime. + +Since Parsl tasks are executed asynchronously and remotely, it can be difficult to determine +when errors have occurred and to appropriately handle them in a Parsl program. + +For errors occurring in Python code, Parsl captures Python exceptions and returns +them to the main Parsl program. For non-Python errors, for example when a node +or worker fails, Parsl imposes a timeout, and considers a task to have failed +if it has not heard from the task by that timeout. Parsl also considers a task to have failed +if it does not meet the contract stated by the user during invocation, such as failing +to produce the stated output files. + +Parsl communicates these errors by associating Python exceptions with task futures. +These exceptions are raised only when a result is called on the future +of a failed task. For example: + +.. code-block:: python + + @python_app + def bad_divide(x): + return 6 / x + + # Call bad divide with 0, to cause a divide by zero exception + doubled_x = bad_divide(0) + + # Catch and handle the exception. + try: + doubled_x.result() + except ZeroDivisionError as e: + print('Oops! You tried to divide by 0.') + except Exception as e: + print('Oops! Something really bad happened.') + + +Retries +------- + +Often errors in distributed/parallel environments are transient. +In these cases, retrying failed tasks can be a simple way +of overcoming transient (e.g., machine failure, +network failure) and intermittent failures. +When ``retries`` are enabled (and set to an integer > 0), Parsl will automatically +re-launch tasks that have failed until the retry limit is reached. +By default, retries are disabled and exceptions will be communicated +to the Parsl program. + +The following example shows how the number of retries can be set to 2: + +.. code-block:: python + + import parsl + from parsl.configs.htex_local import config + + config.retries = 2 + + parsl.load(config) + +More specific retry handling can be specified using retry handlers, documented +below. + + +Lazy fail +--------- + +Parsl implements a lazy failure model through which a workload will continue +to execute in the case that some tasks fail. That is, the program will not +halt as soon as it encounters a failure, rather it will continue to execute +unaffected apps. + +The following example shows how lazy failures affect execution. In this +case, task C fails and therefore tasks E and F that depend on results from +C cannot be executed; however, Parsl will continue to execute tasks B and D +as they are unaffected by task C's failure. + +.. code-block:: + + Here's a workflow graph, where + (X) is runnable, + [X] is completed, + (X*) is failed. + (!X) is dependency failed + + (A) [A] (A) + / \ / \ / \ + (B) (C) [B] (C*) [B] (C*) + | | => | | => | | + (D) (E) (D) (E) [D] (!E) + \ / \ / \ / + (F) (F) (!F) + + time -----> + + +Retry handlers +-------------- + +The basic parsl retry mechanism keeps a count of the number of times a task +has been (re)tried, and will continue retrying that task until the configured +retry limit is reached. + +Retry handlers generalize this to allow more expressive retry handling: +parsl keeps a retry cost for a task, and the task will be retried until the +configured retry limit is reached. Instead of the cost being 1 for each +failure, user-supplied code can examine the failure and compute a custom +cost. + +This allows user knowledge about failures to influence the retry mechanism: +an exception which is almost definitely a non-recoverable failure (for example, +due to bad parameters) can be given a high retry cost (so that it will not +be retried many times, or at all), and exceptions which are likely to be +transient (for example, where a worker node has died) can be given a low +retry cost so they will be retried many times. + +A retry handler can be specified in the parsl configuration like this: + + +.. code-block:: python + + Config( + retries=2, + retry_handler=example_retry_handler + ) + + +``example_retry_handler`` should be a function defined by the user that will +compute the retry cost for a particular failure, given some information about +the failure. + +For example, the following handler will give a cost of 1 to all exceptions, +except when a bash app exits with unix exitcode 9, in which case the cost will +be 100. This will have the effect that retries will happen as normal for most +errors, but the bash app can indicate that there is little point in retrying +by exiting with exitcode 9. + +.. code-block:: python + + def example_retry_handler(exception, task_record): + if isinstance(exception, BashExitFailure) and exception.exitcode == 9: + return 100 + else + return 1 + +The retry handler is given two parameters: the exception from execution, and +the parsl internal task_record. The task record contains details such as the +app name, parameters and executor. + +If a retry handler raises an exception itself, then the task will be aborted +and no further tries will be attempted. diff --git a/docs/userguide/workflows/futures.rst b/docs/userguide/workflows/futures.rst new file mode 100644 index 0000000000..13d22a211b --- /dev/null +++ b/docs/userguide/workflows/futures.rst @@ -0,0 +1,165 @@ +.. _label-futures: + +Futures +======= + +When an ordinary Python function is invoked in a Python program, the Python interpreter waits for the function to complete execution +before proceeding to the next statement. +But if a function is expected to execute for a long period of time, it may be preferable not to wait for +its completion but instead to proceed immediately with executing subsequent statements. +The function can then execute concurrently with that other computation. + +Concurrency can be used to enhance performance when independent activities +can execute on different cores or nodes in parallel. The following +code fragment demonstrates this idea, showing that overall execution time +may be reduced if the two function calls are executed concurrently. + +.. code-block:: python + + v1 = expensive_function(1) + v2 = expensive_function(2) + result = v1 + v2 + +However, concurrency also introduces a need for **synchronization**. +In the example, it is not possible to compute the sum of ``v1`` and ``v2`` +until both function calls have completed. +Synchronization provides a way of blocking execution of one activity +(here, the statement ``result = v1 + v2``) until other activities +(here, the two calls to ``expensive_function()``) have completed. + +Parsl supports concurrency and synchronization as follows. +Whenever a Parsl program calls a Parsl app (a function annotated with a Parsl +app decorator, see :ref:`apps`), +Parsl will create a new ``task`` and immediately return a +`future `_ in lieu of that function's result(s). +The program will then continue immediately to the next statement in the program. +At some point, for example when the task's dependencies are met and there +is available computing capacity, Parsl will execute the task. Upon +completion, Parsl will set the value of the future to contain the task's +output. + +A future can be used to track the status of an asynchronous task. +For example, after creation, the future may be interrogated to determine +the task's status (e.g., running, failed, completed), access results, +and capture exceptions. Further, futures may be used for synchronization, +enabling the calling Python program to block until the future +has completed execution. + +Parsl provides two types of futures: `AppFuture` and `DataFuture`. +While related, they enable subtly different parallel patterns. + +AppFutures +---------- + +AppFutures are the basic building block upon which Parsl programs are built. Every invocation of a Parsl app returns an AppFuture that may be used to monitor and manage the task's execution. +AppFutures are inherited from Python's `concurrent library `_. +They provide three key capabilities: + +1. An AppFuture's ``result()`` function can be used to wait for an app to complete, and then access any result(s). +This function is blocking: it returns only when the app completes or fails. +The following code fragment implements an example similar to the ``expensive_function()`` example above. +Here, the ``sleep_double`` app simply doubles the input value. The program invokes +the ``sleep_double`` app twice, and returns futures in place of results. The example +shows how the future's ``result()`` function can be used to wait for the results from the +two ``sleep_double`` app invocations to be computed. + +.. code-block:: python + + @python_app + def sleep_double(x): + import time + time.sleep(2) # Sleep for 2 seconds + return x*2 + + # Start two concurrent sleep_double apps. doubled_x1 and doubled_x2 are AppFutures + doubled_x1 = sleep_double(10) + doubled_x2 = sleep_double(5) + + # The result() function will block until each of the corresponding app calls have completed + print(doubled_x1.result() + doubled_x2.result()) + +2. An AppFuture's ``done()`` function can be used to check the status of an app, *without blocking*. +The following example shows that calling the future's ``done()`` function will not stop execution of the main Python program. + +.. code-block:: python + + @python_app + def double(x): + return x*2 + + # doubled_x is an AppFuture + doubled_x = double(10) + + # Check status of doubled_x, this will print True if the result is available, else False + print(doubled_x.done()) + +3. An AppFuture provides a safe way to handle exceptions and errors while asynchronously executing +apps. The example shows how exceptions can be captured in the same way as a standard Python program +when calling the future's ``result()`` function. + +.. code-block:: python + + @python_app + def bad_divide(x): + return 6/x + + # Call bad divide with 0, to cause a divide by zero exception + doubled_x = bad_divide(0) + + # Catch and handle the exception. + try: + doubled_x.result() + except ZeroDivisionError as ze: + print('Oops! You tried to divide by 0') + except Exception as e: + print('Oops! Something really bad happened') + + +In addition to being able to capture exceptions raised by a specific app, Parsl also raises ``DependencyErrors`` when apps are unable to execute due to failures in prior dependent apps. +That is, an app that is dependent upon the successful completion of another app will fail with a dependency error if any of the apps on which it depends fail. + + +DataFutures +----------- + +While an AppFuture represents the execution of an asynchronous app, +a DataFuture represents a file to be produced by that app. +Parsl's dataflow model requires such a construct so that it can determine +when dependent apps, apps that that are to consume a file produced by another app, +can start execution. + +When calling an app that produces files as outputs, Parsl requires that a list of output files be specified (as a list of `File` objects passed in via the ``outputs`` keyword argument). Parsl will return a DataFuture for each output file as part AppFuture when the app is executed. +These DataFutures are accessible in the AppFuture's ``outputs`` attribute. + +Each DataFuture will complete when the App has finished executing, +and the corresponding file has been created (and if specified, staged out). + +When a DataFuture is passed as an argument to a subsequent app invocation, +that subsequent app will not begin execution until the DataFuture is +completed. The input argument will then be replaced with an appropriate +File object. + +The following code snippet shows how DataFutures are used. In this +example, the call to the echo Bash app specifies that the results +should be written to an output file ("hello1.txt"). The main +program inspects the status of the output file (via the future's +``outputs`` attribute) and then blocks waiting for the file to +be created (``hello.outputs[0].result()``). + +.. code-block:: python + + # This app echoes the input string to the first file specified in the + # outputs list + @bash_app + def echo(message, outputs=()): + return 'echo {} &> {}'.format(message, outputs[0]) + + # Call echo specifying the output file + hello = echo('Hello World!', outputs=[File('hello1.txt')]) + + # The AppFuture's outputs attribute is a list of DataFutures + print(hello.outputs) + + # Print the contents of the output DataFuture when complete + with open(hello.outputs[0].result().filepath, 'r') as f: + print(f.read()) diff --git a/docs/userguide/workflows/index.rst b/docs/userguide/workflows/index.rst new file mode 100644 index 0000000000..0ccb0841f2 --- /dev/null +++ b/docs/userguide/workflows/index.rst @@ -0,0 +1,18 @@ +Running Workflows +================= + +Parsl workflows are a Python "main" program that defines Apps, +how the Apps are invoked, +and how results are passed between different Apps. + +The core concept of workflows is that Parsl Apps produce **Futures** +with all features from those in Python's :mod:`concurrent.futures` module and more. + +.. toctree:: + :maxdepth: 2 + + futures + workflow + exceptions + lifted_ops + checkpoints diff --git a/docs/userguide/workflows/lifted_ops.rst b/docs/userguide/workflows/lifted_ops.rst new file mode 100644 index 0000000000..6e258b9b62 --- /dev/null +++ b/docs/userguide/workflows/lifted_ops.rst @@ -0,0 +1,56 @@ +.. _label-liftedops: + +Lifted operators +================ + +Parsl allows some operators (``[]`` and ``.``) to be used on an AppFuture in +a way that makes sense with those operators on the eventually returned +result. + +Lifted [] operator +------------------ + +When an app returns a complex structure such as a ``dict`` or a ``list``, +it is sometimes useful to pass an element of that structure to a subsequent +task, without waiting for that subsequent task to complete. + +To help with this, Parsl allows the ``[]`` operator to be used on an +`AppFuture`. This operator will return another `AppFuture` that will +complete after the initial future, with the result of ``[]`` on the value +of the initial future. + +The end result is that this assertion will hold: + +.. code-block:: python + + fut = my_app() + assert fut['x'].result() == fut.result()[x] + +but more concurrency will be available, as execution of the main workflow +code will not stop to wait for ``result()`` to complete on the initial +future. + +`AppFuture` does not implement other methods commonly associated with +dicts and lists, such as ``len``, because those methods should return a +specific type of result immediately, and that is not possible when the +results are not available until the future. + +If a key does not exist in the returned result, then the exception will +appear in the Future returned by ``[]``, rather than at the point that +the ``[]`` operator is applied. This is because the valid values that can +be used are not known until the underlying result is available. + +Lifted . operator +----------------- + +The ``.`` operator works similarly to ``[]`` described above: + +.. code-block:: python + + fut = my_app + assert fut.x.result() == fut.result().x + +Attributes beginning with ``_`` are not lifted as this usually indicates an +attribute that is used for internal purposes, and to try to avoid mixing +protocols (such as iteration in for loops) defined on AppFutures vs protocols +defined on the underlying result object. diff --git a/docs/userguide/workflows/workflow.rst b/docs/userguide/workflows/workflow.rst new file mode 100644 index 0000000000..f8b34fa6c5 --- /dev/null +++ b/docs/userguide/workflows/workflow.rst @@ -0,0 +1,243 @@ +.. _label-workflow: + +Example parallel patterns +========================= + +Parsl can be used to implement a wide range of parallel programming patterns, from bag of tasks +through to nested workflows. Parsl implicitly assembles a dataflow +dependency graph based on the data shared between apps. +The flexibility of this model allows for the implementation of a wide range +of parallel programming and workflow patterns. + +Parsl is also designed to address broad execution requirements, from programs +that run many short tasks to those that run a few long tasks. + +Below we illustrate a range of parallel programming and workflow patterns. It is important +to note that this set of examples is by no means comprehensive. + + +Bag of Tasks +------------ +Parsl can be used to execute a large bag of tasks. In this case, Parsl +assembles the set of tasks (represented as Parsl apps) and manages their concurrent +execution on available resources. + +.. code-block:: python + + from parsl import python_app + + parsl.load() + + # Map function that returns double the input integer + @python_app + def app_random(): + import random + return random.random() + + results = [] + for i in range(0, 10): + x = app_random() + results.append(x) + + for r in results: + print(r.result()) + + +Sequential workflows +-------------------- + +Sequential workflows can be created by passing an AppFuture from one task to another. For example, in the following program the ``generate`` app (a Python app) generates a random number that is consumed by the ``save`` app (a Bash app), which writes it to a file. Because ``save`` cannot execute until it receives the ``message`` produced by ``generate``, the two apps execute in sequence. + +.. code-block:: python + + from parsl import python_app + + parsl.load() + + # Generate a random number + @python_app + def generate(limit): + from random import randint + """Generate a random integer and return it""" + return randint(1, limit) + + # Write a message to a file + @bash_app + def save(message, outputs=()): + return 'echo {} &> {}'.format(message, outputs[0]) + + message = generate(10) + + saved = save(message, outputs=['output.txt']) + + with open(saved.outputs[0].result(), 'r') as f: + print(f.read()) + + +Parallel workflows +------------------ + +Parallel execution occurs automatically in Parsl, respecting dependencies among app executions. In the following example, three instances of the ``wait_sleep_double`` app are created. The first two execute concurrently, as they have no dependencies; the third must wait until the first two complete and thus the ``doubled_x`` and ``doubled_y`` futures have values. Note that this sequencing occurs even though ``wait_sleep_double`` does not in fact use its second and third arguments. + +.. code-block:: python + + from parsl import python_app + + parsl.load() + + @python_app + def wait_sleep_double(x, foo_1, foo_2): + import time + time.sleep(2) # Sleep for 2 seconds + return x*2 + + # Launch two apps, which will execute in parallel, since they do not have to + # wait on any futures + doubled_x = wait_sleep_double(10, None, None) + doubled_y = wait_sleep_double(10, None, None) + + # The third app depends on the first two: + # doubled_x doubled_y (2 s) + # \ / + # doublex_z (2 s) + doubled_z = wait_sleep_double(10, doubled_x, doubled_y) + + # doubled_z will be done in ~4s + print(doubled_z.result()) + + +Parallel workflows with loops +----------------------------- + +A common approach to executing Parsl apps in parallel is via loops. The following example uses a loop to create many random numbers in parallel. + +.. code-block:: python + + from parsl import python_app + + parsl.load() + + @python_app + def generate(limit): + """Generate a random integer and return it""" + from random import randint + return randint(1, limit) + + rand_nums = [] + for i in range(1,5): + rand_nums.append(generate(i)) + + # Wait for all apps to finish and collect the results + outputs = [r.result() for r in rand_nums] + +The :class:`~parsl.concurrent.ParslPoolExecutor` simplifies this pattern using the same interface as +`Python's native Executors `_. + +.. code-block:: python + + from parsl.concurrent import ParslPoolExecutor + from parsl.configs.htex_local import config + + # NOTE: Functions used by the ParslPoolExecutor do _not_ use decorators + def generate(limit): + """Generate a random integer and return it""" + from random import randint + return randint(1, limit) + + + with ParslPoolExecutor(config) as pool: + outputs = pool.map(generate, range(1, 5)) + + +In the preceding example, the execution of different tasks is coordinated by passing Python objects from producers to consumers. +In other cases, it can be convenient to pass data in files, as in the following reformulation. Here, a set of files, each with a random number, is created by the ``generate`` app. These files are then concatenated into a single file, which is subsequently used to compute the sum of all numbers. + +.. code-block:: python + + from parsl import python_app, bash_app + + parsl.load() + + @bash_app + def generate(outputs=()): + return 'echo $(( RANDOM % (10 - 5 + 1 ) + 5 )) &> {}'.format(outputs[0]) + + @bash_app + def concat(inputs=(), outputs=(), stdout='stdout.txt', stderr='stderr.txt'): + return 'cat {0} >> {1}'.format(' '.join(inputs), outputs[0]) + + @python_app + def total(inputs=()): + total = 0 + with open(inputs[0].filepath, 'r') as f: + for l in f: + total += int(l) + return total + + # Create 5 files with random numbers + output_files = [] + for i in range (5): + output_files.append(generate(outputs=['random-%s.txt' % i])) + + # Concatenate the files into a single file + cc = concat(inputs=[i.outputs[0] for i in output_files], outputs=['all.txt']) + + # Calculate the average of the random numbers + totals = total(inputs=[cc.outputs[0]]) + + print(totals.result()) + + +MapReduce +--------- +MapReduce is a common pattern used in data analytics. It is composed of a map phase +that filters values and a reduce phase that aggregates values. +The following example demonstrates how Parsl can be used to specify a MapReduce computation +in which the map phase doubles a set of input integers and the reduce phase computes +the sum of those results. + +.. code-block:: python + + from parsl import python_app + + parsl.load() + + # Map function that returns double the input integer + @python_app + def app_double(x): + return x*2 + + # Reduce function that returns the sum of a list + @python_app + def app_sum(inputs=()): + return sum(inputs) + + # Create a list of integers + items = range(0,4) + + # Map phase: apply the double *app* function to each item in list + mapped_results = [] + for i in items: + x = app_double(i) + mapped_results.append(x) + + # Reduce phase: apply the sum *app* function to the set of results + total = app_sum(inputs=mapped_results) + + print(total.result()) + +The program first defines two Parsl apps, ``app_double`` and ``app_sum``. +It then makes calls to the ``app_double`` app with a set of input +values. It then passes the results from ``app_double`` to the ``app_sum`` app +to aggregate values into a single result. +These tasks execute concurrently, synchronized by the ``mapped_results`` variable. +The following figure shows the resulting task graph. + +.. image:: ../../images/MapReduce.png + +Caching expensive initialisation between tasks +---------------------------------------------- + +Many tasks in workflows require a expensive "initialization" steps that, once performed, can be used across successive invocations for that task. For example, you may want to reuse a machine learning model for multiple interface tasks and avoid loading it onto GPUs more than once. + +`This ExaWorks tutorial `_ gives examples of how to do this. From ab5e24756c998803bf2912aabbf38d147537c0b0 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 6 Jan 2025 15:21:28 +0000 Subject: [PATCH 25/39] Upgrade networkx to support Numpy 2 to support Python 3.13 (#3733) This is a piece of PR #3646 to support Python 3.13 networkx 3.2 is the first version to support numpy 2. Later versions do not support Python 3.9. # Changed Behaviour This is a package upgrade that behaves the same in my testing ## Type of change - Code maintenance/cleanup --- setup.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index bed98934e7..f5aadb1e2e 100755 --- a/setup.py +++ b/setup.py @@ -11,8 +11,13 @@ 'sqlalchemy>=1.4,<2' ], 'visualization' : [ - 'pydot', - 'networkx>=2.5,<2.6', + # this pydot bound is copied from networkx's pyproject.toml, + # version 3.2 (aa2de1adecea09f7b86ff6093b212ca86f22b3ef), + # because networkx[extra] installs quite a lot of extra stuff + # that needs more OS dependencies in addition to pydot. + 'pydot>=1.4.2', + + 'networkx>=3.2,<3.3', 'Flask>=1.0.2', 'flask_sqlalchemy', 'pandas<2.2', From ed80dad172d659ec032a9a69f8394f16c9477c34 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 6 Jan 2025 16:49:48 +0000 Subject: [PATCH 26/39] Fix rounding error in htex block scale in (#3721) # Description PR #2196 calculates a number of blocks to scale in, in the htex strategy, rather than scaling in one block per strategy iteration. However, it rounds the wrong way: it scales in a rounded up, rather than rounded down, number of blocks. Issue #3696 shows that then resulting in oscillating behaviour: With 14 tasks and 48 workers per block, on alternating strategy runs, the code will either scale up to the rounded up number of needed blocks (14/48 => 1), or scale down to the rounded down number of needed blocks (14/48 => 0). This PR changes the rounding introduced in #2196 to be consistent: rounding up the number of blocks to scale up, and rounding down the number of blocks to scale down. # Changed Behaviour HTEX scale down should oscillate less # Fixes Fixes #3696 ## Type of change - Bug fix --- parsl/jobs/strategy.py | 4 +- .../test_regression_3696_oscillation.py | 103 ++++++++++++++++++ 2 files changed, 105 insertions(+), 2 deletions(-) create mode 100644 parsl/tests/test_scaling/test_regression_3696_oscillation.py diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index e0898cccb1..8ca50a47e4 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -298,8 +298,8 @@ def _general_strategy(self, executors: List[BlockProviderExecutor], *, strategy_ # Scale in for htex if isinstance(executor, HighThroughputExecutor): if active_blocks > min_blocks: - excess_slots = math.ceil(active_slots - (active_tasks * parallelism)) - excess_blocks = math.ceil(float(excess_slots) / (tasks_per_node * nodes_per_block)) + excess_slots = math.floor(active_slots - (active_tasks * parallelism)) + excess_blocks = math.floor(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 with idle time {self.max_idletime}s") executor.scale_in_facade(excess_blocks, max_idletime=self.max_idletime) diff --git a/parsl/tests/test_scaling/test_regression_3696_oscillation.py b/parsl/tests/test_scaling/test_regression_3696_oscillation.py new file mode 100644 index 0000000000..a6c76ab9e9 --- /dev/null +++ b/parsl/tests/test_scaling/test_regression_3696_oscillation.py @@ -0,0 +1,103 @@ +import math +from unittest.mock import MagicMock + +import pytest + +from parsl.executors.high_throughput.executor import HighThroughputExecutor +from parsl.jobs.states import JobState, JobStatus +from parsl.jobs.strategy import Strategy + + +# the parameterize tuple consists of: +# Input: +# * number of tasks to mock the load as +# * number of workers per node +# Expected output: +# * the number of blocks we should expect to be launched +# in this situation +# +# This test will configure an executor, then run strategize +# a few times, asserting that it converges to the correct +# number of blocks without oscillating. +@pytest.mark.local +@pytest.mark.parametrize("ns", [(14, 48, 1), # values from issue #3696 + + (1, 1, 1), # one task needs one block + + (100, 1, 20), # many one-task blocks, hitting hard-coded max blocks + + (47, 48, 1), # some edge cases around #3696 values + (48, 48, 1), # " + (49, 48, 2), # " + (149, 50, 3)]) # " +def test_htex_strategy_does_not_oscillate(ns): + """Check for oscillations in htex scaling. + In issue 3696, with a large number of workers per block + and a smaller number of active tasks, the htex scaling + strategy oscillates between 0 and 1 active block, rather + than converging to 1 active block. + """ + + n_tasks, n_workers, n_blocks = ns + + s = Strategy(strategy='htex_auto_scale', max_idletime=0) + + provider = MagicMock() + executor = MagicMock(spec=HighThroughputExecutor) + + statuses = {} + + executor.provider = provider + executor.outstanding = n_tasks + executor.status_facade = statuses + executor.workers_per_node = n_workers + + provider.parallelism = 1 + provider.init_blocks = 0 + provider.min_blocks = 0 + provider.max_blocks = 20 + provider.nodes_per_block = 1 + + def scale_out(n): + for _ in range(n): + statuses[len(statuses)] = JobStatus(state=JobState.PENDING) + + executor.scale_out_facade.side_effect = scale_out + + def scale_in(n, max_idletime=None): + # find n PENDING jobs and set them to CANCELLED + for k in statuses: + if n == 0: + return + if statuses[k].state == JobState.PENDING: + statuses[k].state = JobState.CANCELLED + n -= 1 + + executor.scale_in_facade.side_effect = scale_in + + s.add_executors([executor]) + + # In issue #3696, this first strategise does initial and load based + # scale outs, because n_tasks > n_workers*0 + s.strategize([executor]) + + executor.scale_out_facade.assert_called() + assert len(statuses) == n_blocks, "Should have launched n_blocks" + assert len([k for k in statuses if statuses[k].state == JobState.PENDING]) == n_blocks + # there might be several calls to scale_out_facade inside strategy, + # but the end effect should be that exactly one block is scaled out. + + executor.scale_in_facade.assert_not_called() + + # In issue #3696, this second strategize does a scale in, because n_tasks < n_workers*1 + s.strategize([executor]) + + # assert that there should still be n_blocks pending blocks + assert len([k for k in statuses if statuses[k].state == JobState.PENDING]) == n_blocks + # this assert fails due to issue #3696 + + # Now check scale in happens with 0 load + executor.outstanding = 0 + s.strategize([executor]) + executor.scale_in_facade.assert_called() + assert len([k for k in statuses if statuses[k].state == JobState.PENDING]) == 0 From a3255908c1c4b43000ff84ee7da6d95273051aaf Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 6 Jan 2025 18:45:20 +0000 Subject: [PATCH 27/39] Upgrade SQLAlchemy to version 2 (#3734) This is part of supporting Python 3.13 (PR #3646), as a prereq to upgrading pandas. # Changed Behaviour Development environments (which have the requirements in `test-requirements.txt` installed) will break and need `pip uninstall sqlalchemy2-stubs` - sqlalchemy packaging does not express a negative/removal requirement against that transitional package. non-parsl user code which happens to use the parsl-installed sqlalchemy might break - for example, see API changes in the test suite in this PR ## Type of change - Code maintenance/cleanup --- parsl/monitoring/db_manager.py | 2 +- parsl/tests/test_monitoring/test_app_names.py | 4 ++-- parsl/tests/test_monitoring/test_stdouterr.py | 4 ++-- setup.py | 4 +++- test-requirements.txt | 3 +-- 5 files changed, 9 insertions(+), 8 deletions(-) diff --git a/parsl/monitoring/db_manager.py b/parsl/monitoring/db_manager.py index 8c1d76dbc6..af52944021 100644 --- a/parsl/monitoring/db_manager.py +++ b/parsl/monitoring/db_manager.py @@ -78,7 +78,7 @@ def __init__(self, def _get_mapper(self, table_obj: Table) -> Mapper: all_mappers: Set[Mapper] = set() - for mapper_registry in mapperlib._all_registries(): # type: ignore[attr-defined] + for mapper_registry in mapperlib._all_registries(): all_mappers.update(mapper_registry.mappers) mapper_gen = ( mapper for mapper in all_mappers diff --git a/parsl/tests/test_monitoring/test_app_names.py b/parsl/tests/test_monitoring/test_app_names.py index 1c3b6fef11..ad21d47147 100644 --- a/parsl/tests/test_monitoring/test_app_names.py +++ b/parsl/tests/test_monitoring/test_app_names.py @@ -67,7 +67,7 @@ def test_app_name(get_app, expected_name, expected_result, tmpd_cwd): with engine.begin() as connection: def count_rows(table: str): - result = connection.execute(f"SELECT COUNT(*) FROM {table}") + result = connection.execute(sqlalchemy.text(f"SELECT COUNT(*) FROM {table}")) (c, ) = result.first() return c @@ -81,6 +81,6 @@ def count_rows(table: str): assert count_rows("try") == 1 # ... and has the expected name. - result = connection.execute("SELECT task_func_name FROM task") + result = connection.execute(sqlalchemy.text("SELECT task_func_name FROM task")) (c, ) = result.first() assert c == expected_name diff --git a/parsl/tests/test_monitoring/test_stdouterr.py b/parsl/tests/test_monitoring/test_stdouterr.py index 8e1935045f..982525b7e9 100644 --- a/parsl/tests/test_monitoring/test_stdouterr.py +++ b/parsl/tests/test_monitoring/test_stdouterr.py @@ -106,7 +106,7 @@ def test_stdstream_to_monitoring(stdx, expected_stdx, stream, tmpd_cwd, caplog): with engine.begin() as connection: def count_rows(table: str): - result = connection.execute(f"SELECT COUNT(*) FROM {table}") + result = connection.execute(sqlalchemy.text(f"SELECT COUNT(*) FROM {table}")) (c, ) = result.first() return c @@ -120,7 +120,7 @@ def count_rows(table: str): assert count_rows("try") == 1 # ... and has the expected name. - result = connection.execute(f"SELECT task_{stream} FROM task") + result = connection.execute(sqlalchemy.text(f"SELECT task_{stream} FROM task")) (c, ) = result.first() if isinstance(expected_stdx, str): diff --git a/setup.py b/setup.py index f5aadb1e2e..1511649845 100755 --- a/setup.py +++ b/setup.py @@ -8,7 +8,9 @@ extras_require = { 'monitoring' : [ - 'sqlalchemy>=1.4,<2' + # sqlalchemy does not use semantic versioning. + # see https://github.com/sqlalchemy/sqlalchemy/discussions/11391#discussioncomment-9472033 + 'sqlalchemy>=2,<2.1' ], 'visualization' : [ # this pydot bound is copied from networkx's pyproject.toml, diff --git a/test-requirements.txt b/test-requirements.txt index 82ec5172c2..2b4d81a6dd 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -14,8 +14,7 @@ mpi4py # sqlalchemy is needed for typechecking, so it's here # as well as at runtime for optional monitoring execution # (where it's specified in setup.py) -sqlalchemy>=1.4,<2 -sqlalchemy2-stubs +sqlalchemy>=2,<2.1 Sphinx==4.5.0 twine From ce7839314cf29ba7a96d60cecaaa1431080c921c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 7 Jan 2025 22:10:05 +0000 Subject: [PATCH 28/39] Remove WorkQueue and TaskVine block scale-in code (#3737) Block scaling is the responsibility of the job status poller, and end-of-run block scale-in happens in `job_status_poller.close()` The code removed by this PR is buggy and breaks in some situations: it should not be using the `blocks_to_job_id` table to enumerate live blocks. ## Type of change - Bug fix - Code maintenance/cleanup --- parsl/executors/taskvine/executor.py | 6 ------ parsl/executors/workqueue/executor.py | 6 ------ 2 files changed, 12 deletions(-) diff --git a/parsl/executors/taskvine/executor.py b/parsl/executors/taskvine/executor.py index a15a444d2c..5896d333b5 100644 --- a/parsl/executors/taskvine/executor.py +++ b/parsl/executors/taskvine/executor.py @@ -582,12 +582,6 @@ def shutdown(self, *args, **kwargs): logger.debug("TaskVine shutdown started") self._should_stop.set() - # Remove the workers that are still going - kill_ids = [self.blocks_to_job_id[block] for block in self.blocks_to_job_id.keys()] - if self.provider: - logger.debug("Cancelling blocks") - self.provider.cancel(kill_ids) - # Join all processes before exiting logger.debug("Joining on submit process") self._submit_process.join() diff --git a/parsl/executors/workqueue/executor.py b/parsl/executors/workqueue/executor.py index 155c990ab5..b1a534623e 100644 --- a/parsl/executors/workqueue/executor.py +++ b/parsl/executors/workqueue/executor.py @@ -697,12 +697,6 @@ def shutdown(self, *args, **kwargs): logger.debug("Work Queue shutdown started") self.should_stop.value = True - # Remove the workers that are still going - kill_ids = [self.blocks_to_job_id[block] for block in self.blocks_to_job_id.keys()] - if self.provider: - logger.debug("Cancelling blocks") - self.provider.cancel(kill_ids) - logger.debug("Joining on submit process") self.submit_process.join() self.submit_process.close() From 627ffd9b29287d3903600c223f9ef50518a8c197 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 7 Jan 2025 22:10:43 +0000 Subject: [PATCH 29/39] Update make_rundir docstring (#3738) # Changed Behaviour none ## Type of change - Update to human readable text: Documentation/error messages/comments --- parsl/dataflow/rundirs.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/parsl/dataflow/rundirs.py b/parsl/dataflow/rundirs.py index f5ecda6984..141773089e 100644 --- a/parsl/dataflow/rundirs.py +++ b/parsl/dataflow/rundirs.py @@ -6,17 +6,16 @@ def make_rundir(path: str) -> str: - """When a path has not been specified, make the run directory. + """Create a numbered run directory under the specified path. - Creates a rundir with the following hierarchy: - ./runinfo <- Home of all run directories + ./runinfo <- specified path |----000 |----001 <- Directories for each run | .... |----NNN - Kwargs: - - path (str): String path to a specific run dir + Args: + - path (str): String path to root of all rundirs """ try: if not os.path.exists(path): From 34a2890c65f239bb145dca6af76bbdcc0443bc3e Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 7 Jan 2025 22:11:36 +0000 Subject: [PATCH 30/39] Upgrade pandas to >=2.2 (#3735) # Description This is part of supporting Python 3.13 (PR #3646). The previous pandas constraints cannot install on Python 3.13. # Changed Behaviour none ## Type of change - Code maintenance/cleanup --- setup.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 1511649845..94551e2ba3 100755 --- a/setup.py +++ b/setup.py @@ -22,7 +22,11 @@ 'networkx>=3.2,<3.3', 'Flask>=1.0.2', 'flask_sqlalchemy', - 'pandas<2.2', + + # pandas uses "loose semantic versioning" + # https://pandas.pydata.org/docs/development/policies.html#version-policy + 'pandas<3,>=2.2', + 'plotly', 'python-daemon' ], From 5aac648928e20de408f6399942b6871e64364010 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 8 Jan 2025 22:39:41 +0000 Subject: [PATCH 31/39] Add Python 3.13 to CI (#3646) ## Type of change - Code maintenance/cleanup --- .github/workflows/ci.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index dd9fdb0d5e..83eb2716cf 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -11,7 +11,7 @@ jobs: main-test-suite: strategy: matrix: - python-version: ["3.9", "3.10", "3.11", "3.12"] + python-version: ["3.9", "3.10", "3.11", "3.12", "3.13"] runs-on: ubuntu-20.04 timeout-minutes: 60 From 0b07b8ccb115bef60fe19e73dfc1b10b4f0e6370 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 13 Jan 2025 13:38:23 +0000 Subject: [PATCH 32/39] Fix a handful of Task Vine docstring issues (#3736) ## Type of change - Update to human readable text: Documentation/error messages/comments --- docs/reference.rst | 12 ++++++++++++ parsl/executors/taskvine/factory_config.py | 2 +- parsl/executors/taskvine/manager_config.py | 4 ++-- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/docs/reference.rst b/docs/reference.rst index d4c70dd7bd..3bba8acba9 100644 --- a/docs/reference.rst +++ b/docs/reference.rst @@ -32,6 +32,8 @@ Configuration parsl.addresses.address_by_interface parsl.addresses.address_by_query parsl.addresses.address_by_route + parsl.addresses.get_all_addresses + parsl.addresses.get_any_address parsl.utils.get_all_checkpoints parsl.utils.get_last_checkpoint @@ -193,3 +195,13 @@ Internal parsl.jobs.job_status_poller.JobStatusPoller parsl.jobs.strategy.Strategy parsl.utils.Timer + +Task Vine configuration +======================= + +.. autosummary:: + :toctree: stubs + :nosignatures: + + parsl.executors.taskvine.TaskVineManagerConfig + parsl.executors.taskvine.TaskVineFactoryConfig diff --git a/parsl/executors/taskvine/factory_config.py b/parsl/executors/taskvine/factory_config.py index 145e2c05d5..f198e87438 100644 --- a/parsl/executors/taskvine/factory_config.py +++ b/parsl/executors/taskvine/factory_config.py @@ -32,7 +32,7 @@ class TaskVineFactoryConfig: worker_options: Optional[str] Additional options to pass to workers. Run - `vine_worker --help` for more details. + ``vine_worker --help`` for more details. Default is None. worker_executable: str diff --git a/parsl/executors/taskvine/manager_config.py b/parsl/executors/taskvine/manager_config.py index 0ef112fda4..26b1e460b1 100644 --- a/parsl/executors/taskvine/manager_config.py +++ b/parsl/executors/taskvine/manager_config.py @@ -24,7 +24,7 @@ class TaskVineManagerConfig: address: Optional[str] Address of the local machine. - If None, socket.gethostname() will be used to determine the address. + If None, :py:func:`parsl.addresses.get_any_address` will be used to determine the address. project_name: Optional[str] If given, TaskVine will periodically report its status and performance @@ -56,7 +56,7 @@ class TaskVineManagerConfig: environment name is given, TaskVine will package the conda environment in a tarball and send it along with tasks to be executed in a replicated conda environment. - If a tarball of packages (*.tar.gz) is given, TaskVine + If a tarball of packages (``*.tar.gz``) is given, TaskVine skips the packaging step and sends the tarball along with tasks to be executed in a replicated conda environment. From a1c7e3073567712fcede814b0e5dbcc8449de2b7 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 13 Jan 2025 13:40:33 +0000 Subject: [PATCH 33/39] Remove a deliberately non-deterministic test (#3739) This test would pass if execution either succeeded or failed with a `DependencyError`. It did not assert any relationship between those two possibilities and whether earlier tasks actually succeeded or failed. `test_fail_sequence` in the same file tests more strongly whether a failure is correctly propagated. ## Type of change - Code maintenance/cleanup --- parsl/tests/test_python_apps/test_fail.py | 25 ----------------------- 1 file changed, 25 deletions(-) diff --git a/parsl/tests/test_python_apps/test_fail.py b/parsl/tests/test_python_apps/test_fail.py index 159153be93..b29a4f0f53 100644 --- a/parsl/tests/test_python_apps/test_fail.py +++ b/parsl/tests/test_python_apps/test_fail.py @@ -41,28 +41,3 @@ def test_fail_sequence(fail_probs): with pytest.raises(DependencyError): t_final.result() - - -def test_deps(width=3): - """Random failures in branches of Map -> Map -> reduce""" - # App1 App2 ... AppN - futs = [random_fail(fail_prob=0.4) for _ in range(width)] - - # App1 App2 ... AppN - # | | | - # V V V - # App1 App2 ... AppN - - futs = [random_fail(fail_prob=0.8, inputs=[f]) for f in futs] - - # App1 App2 ... AppN - # | | | - # V V V - # App1 App2 ... AppN - # \ | / - # \ | / - # App_Final - try: - random_fail(fail_prob=0, inputs=futs).result() - except DependencyError: - pass From 4686c072570ac7f98189c04518bbf488e05cd734 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 13 Jan 2025 13:42:06 +0000 Subject: [PATCH 34/39] Change some exceptions to put human readable text in __str__ not __repr__ (#3741) # Description See #2025 for more context. # Changed Behaviour Rendering of exceptions made with `__repr__` will change to a more machine readable format. Generally Python itself reports exceptions using `__str__`, but DFK task failure history (which goes into the monitoring database) is rendered with `__repr__`. This is probably better behaviour for that history field, which is quasi-machine-readable. ## Type of change - Update to human readable text: Documentation/error messages/comments - Code maintenance/cleanup --- parsl/app/errors.py | 9 +++------ parsl/dataflow/errors.py | 5 +---- parsl/executors/high_throughput/errors.py | 5 +---- 3 files changed, 5 insertions(+), 14 deletions(-) diff --git a/parsl/app/errors.py b/parsl/app/errors.py index 24b941d285..8239c8082e 100644 --- a/parsl/app/errors.py +++ b/parsl/app/errors.py @@ -70,8 +70,8 @@ def __init__(self, reason: str, outputs: List[File]) -> None: self.reason = reason self.outputs = outputs - def __repr__(self) -> str: - return "Missing Outputs: {0}, Reason:{1}".format(self.outputs, self.reason) + def __str__(self) -> str: + return "Missing Outputs: {0}, Reason: {1}".format(self.outputs, self.reason) class BadStdStreamFile(ParslError): @@ -85,11 +85,8 @@ def __init__(self, reason: str) -> None: super().__init__(reason) self._reason = reason - def __repr__(self) -> str: - return "Bad Stream File: {}".format(self._reason) - def __str__(self) -> str: - return self.__repr__() + return "Bad Stream File: {}".format(self._reason) class RemoteExceptionWrapper: diff --git a/parsl/dataflow/errors.py b/parsl/dataflow/errors.py index 926039c4e7..88ecbcfdd1 100644 --- a/parsl/dataflow/errors.py +++ b/parsl/dataflow/errors.py @@ -25,11 +25,8 @@ class BadCheckpoint(DataFlowException): def __init__(self, reason: str) -> None: self.reason = reason - def __repr__(self) -> str: - return self.reason - def __str__(self) -> str: - return self.__repr__() + return self.reason class DependencyError(DataFlowException): diff --git a/parsl/executors/high_throughput/errors.py b/parsl/executors/high_throughput/errors.py index 9916ec506f..2b94b34869 100644 --- a/parsl/executors/high_throughput/errors.py +++ b/parsl/executors/high_throughput/errors.py @@ -38,11 +38,8 @@ def __init__(self, worker_id, hostname): self.worker_id = worker_id self.hostname = hostname - def __repr__(self): - return "Task failure due to loss of worker {} on host {}".format(self.worker_id, self.hostname) - def __str__(self): - return self.__repr__() + return "Task failure due to loss of worker {} on host {}".format(self.worker_id, self.hostname) class CommandClientTimeoutError(Exception): From c3091a01c1b3e89f96ced6a7c555e22c93ced210 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 13 Jan 2025 13:44:51 +0000 Subject: [PATCH 35/39] Tidy DataFuture argument types (#3742) The task ID is never optional: a DataFuture always comes from a Parsl task. (in the same way that an AppFuture always has a task record) Long ago, Files could be specified as strings. This behaviour was removed slightly less long ago. Right now, typeguard should be detecting if a string is passed, because of the type signature, and so the explicit type check for strings should be unnecessary. This PR removes that explicit type check. The future being tracked by the data future is not necessarily an AppFuture: it can be any kind of future that completes to indicate the represented file is now available. That is already reflected in the type annotations and code, but this PR fixes the docstring. # Changed Behaviour none ## Type of change - Code maintenance/cleanup --- parsl/app/futures.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/parsl/app/futures.py b/parsl/app/futures.py index f3bc067c84..d85c4f2576 100644 --- a/parsl/app/futures.py +++ b/parsl/app/futures.py @@ -2,7 +2,6 @@ """ import logging from concurrent.futures import Future -from typing import Optional import typeguard @@ -39,24 +38,23 @@ def parent_callback(self, parent_fu): self.set_result(self.file_obj) @typeguard.typechecked - def __init__(self, fut: Future, file_obj: File, tid: Optional[int] = None) -> None: + def __init__(self, fut: Future, file_obj: File, tid: int) -> None: """Construct the DataFuture object. If the file_obj is a string convert to a File. Args: - - fut (AppFuture) : AppFuture that this DataFuture will track - - file_obj (string/File obj) : Something representing file(s) + - fut (Future) : Future that this DataFuture will track. + Completion of ``fut`` indicates that the data is + ready. + - file_obj (File) : File that this DataFuture represents the availability of Kwargs: - tid (task_id) : Task id that this DataFuture tracks """ super().__init__() self._tid = tid - if isinstance(file_obj, File): - self.file_obj = file_obj - else: - raise ValueError("DataFuture must be initialized with a File, not {}".format(type(file_obj))) + self.file_obj = file_obj self.parent = fut self.parent.add_done_callback(self.parent_callback) From 784256d7f6175ce97cdbd35b80172db47db8514c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 14 Jan 2025 13:06:41 +0000 Subject: [PATCH 36/39] Refactor 4 different renderings of task description (#3743) Prior to this PR, there were 4 variant ways to render a future (which is usually but not always a Parsl task) as a text string: 1. describing failure of inner future in a join_app with 1 future to join - rendered task ID if the inner future has a task record attribute, without checking it is a parsl.dataflow.taskrecord.TaskRecord or that that attribute contains a task id entry. - otherwise render as None 2. describing failure of inner future in a join_app with a list of futures to join - is meant to do the same as case 1, but is buggy and always renders as None (the wrong object is checked for having a task_record attribute) 3. describing failure of a dependency future - rendered task ID if there was a task_record, and the future is from the same DFK as is rendering the code (so that a task number from a different DFK is not rendered, in the corner case of a task from one DFK being used as a dependency for a task in another DFK) - otherwise, renders the repr of the future 4. describing which dependencies will be waited on when submitting a task to the DFK - rendered task ID if the future is an instance of AppFuture or DataFuture - otherwise renders the repr of the future This PR makes a single render method. It is a member of the DFK, because rendering is done in the context of the DFK: two parsl task AppFutures will be rendered differently if they are from this DFK or a different DFK. This PR implements render_future_description which tries to combine all of the above: - if the future is an AppFuture, and is from the same DFK, render the task ID - otherwise render the repr of the future # Changed Behaviour human readable descriptions of dependencies/join inner futures will be changed # Fixes Fixes # (issue) ## Type of change Choose which options apply, and delete the ones which do not apply. - Bug fix - New feature - Update to human readable text: Documentation/error messages/comments - Code maintenance/cleanup --- parsl/dataflow/dflow.py | 37 +++++++++---------- .../test_dep_standard_futures.py | 3 ++ parsl/tests/test_python_apps/test_fail.py | 25 +++++++++---- parsl/tests/test_python_apps/test_join.py | 6 +++ 4 files changed, 43 insertions(+), 28 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index e9bf934cf1..5e91268057 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -484,24 +484,18 @@ def handle_join_update(self, task_record: TaskRecord, inner_app_future: Optional # now we know each joinable Future is done # so now look for any exceptions - exceptions_tids: List[Tuple[BaseException, Optional[str]]] + exceptions_tids: List[Tuple[BaseException, str]] exceptions_tids = [] if isinstance(joinable, Future): je = joinable.exception() if je is not None: - if hasattr(joinable, 'task_record'): - tid = joinable.task_record['id'] - else: - tid = None + tid = self.render_future_description(joinable) exceptions_tids = [(je, tid)] elif isinstance(joinable, list): for future in joinable: je = future.exception() if je is not None: - if hasattr(joinable, 'task_record'): - tid = joinable.task_record['id'] - else: - tid = None + tid = self.render_future_description(future) exceptions_tids.append((je, tid)) else: raise TypeError(f"Unknown joinable type {type(joinable)}") @@ -918,13 +912,7 @@ def _unwrap_futures(self, args: Sequence[Any], kwargs: Dict[str, Any]) \ dep_failures = [] def append_failure(e: Exception, dep: Future) -> None: - # If this Future is associated with a task inside this DFK, - # 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']) - else: - tid = repr(dep) + tid = self.render_future_description(dep) dep_failures.extend([(e, tid)]) # Replace item in args @@ -1076,10 +1064,7 @@ def submit(self, depend_descs = [] for d in depends: - if isinstance(d, AppFuture) or isinstance(d, DataFuture): - depend_descs.append("task {}".format(d.tid)) - else: - depend_descs.append(repr(d)) + depend_descs.append(self.render_future_description(d)) if depend_descs != []: waiting_message = "waiting on {}".format(", ".join(depend_descs)) @@ -1438,6 +1423,18 @@ def default_std_autopath(self, taskrecord, kw): '' if label is None else '_{}'.format(label), kw)) + def render_future_description(self, dep: Future) -> str: + """Renders a description of the future in the context of the + current DFK. + """ + if isinstance(dep, AppFuture) and dep.task_record['dfk'] == self: + tid = "task " + repr(dep.task_record['id']) + elif isinstance(dep, DataFuture): + tid = "DataFuture from task " + repr(dep.tid) + else: + tid = repr(dep) + return tid + class DataFlowKernelLoader: """Manage which DataFlowKernel is active. diff --git a/parsl/tests/test_python_apps/test_dep_standard_futures.py b/parsl/tests/test_python_apps/test_dep_standard_futures.py index 4856888d35..29bb6b2709 100644 --- a/parsl/tests/test_python_apps/test_dep_standard_futures.py +++ b/parsl/tests/test_python_apps/test_dep_standard_futures.py @@ -43,3 +43,6 @@ def test_future_fail_dependency(): # Future, plain_fut, somewhere in its str assert repr(plain_fut) in str(ex) + assert len(ex.dependent_exceptions_tids) == 1 + assert isinstance(ex.dependent_exceptions_tids[0][0], ValueError) + assert ex.dependent_exceptions_tids[0][1].startswith(" App2 ... -> AppN - """ + with pytest.raises(DependencyError): + t_final.result() - t1_fail_prob, t2_fail_prob = fail_probs - t1 = random_fail(fail_prob=t1_fail_prob) - t2 = random_fail(fail_prob=t2_fail_prob, inputs=[t1]) + assert len(t_final.exception().dependent_exceptions_tids) == 1 + assert isinstance(t_final.exception().dependent_exceptions_tids[0][0], DependencyError) + assert t_final.exception().dependent_exceptions_tids[0][1].startswith("task ") + + +def test_fail_sequence_middle(): + t1 = random_fail(fail_prob=0) + t2 = random_fail(fail_prob=1, inputs=[t1]) t_final = random_fail(fail_prob=0, inputs=[t2]) with pytest.raises(DependencyError): t_final.result() + + assert len(t_final.exception().dependent_exceptions_tids) == 1 + assert isinstance(t_final.exception().dependent_exceptions_tids[0][0], ManufacturedTestFailure) diff --git a/parsl/tests/test_python_apps/test_join.py b/parsl/tests/test_python_apps/test_join.py index dcb855a6f7..926d90bf63 100644 --- a/parsl/tests/test_python_apps/test_join.py +++ b/parsl/tests/test_python_apps/test_join.py @@ -97,7 +97,10 @@ def test_error(): f = outer_error() e = f.exception() assert isinstance(e, JoinError) + + assert len(e.dependent_exceptions_tids) == 1 assert isinstance(e.dependent_exceptions_tids[0][0], InnerError) + assert e.dependent_exceptions_tids[0][1].startswith("task ") def test_two_errors(): @@ -109,10 +112,12 @@ def test_two_errors(): de0 = e.dependent_exceptions_tids[0][0] assert isinstance(de0, InnerError) assert de0.args[0] == "Error A" + assert e.dependent_exceptions_tids[0][1].startswith("task ") de1 = e.dependent_exceptions_tids[1][0] assert isinstance(de1, InnerError) assert de1.args[0] == "Error B" + assert e.dependent_exceptions_tids[1][1].startswith("task ") def test_one_error_one_result(): @@ -125,6 +130,7 @@ def test_one_error_one_result(): de0 = e.dependent_exceptions_tids[0][0] assert isinstance(de0, InnerError) assert de0.args[0] == "Error A" + assert e.dependent_exceptions_tids[0][1].startswith("task ") @join_app From 9750976516f127883a4c6ed8658f39af7fe6ac3c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 14 Jan 2025 13:34:24 +0000 Subject: [PATCH 37/39] Remove over-verbose debug message (#3745) This was introduced in PR #2965 to help debug ZMQ hangs. PR #3709 changed this loop to run every poll period (by default, every 10ms) rather than every result message. That turns this log line into a huge source of log noise/volume. This PR removes that log line - ZMQ debugging must proceed without it. # Changed Behaviour parsl.log size from: pytest parsl/tests/ --config parsl/tests/configs/htex_local.py before this PR: 7 megabytes after this PR: 2 megabytes ## Type of change - Bug fix --- parsl/executors/high_throughput/zmq_pipes.py | 1 - 1 file changed, 1 deletion(-) diff --git a/parsl/executors/high_throughput/zmq_pipes.py b/parsl/executors/high_throughput/zmq_pipes.py index a7278cf067..9b368c713c 100644 --- a/parsl/executors/high_throughput/zmq_pipes.py +++ b/parsl/executors/high_throughput/zmq_pipes.py @@ -213,7 +213,6 @@ def get(self, timeout_ms=None): """Get a message from the queue, returning None if timeout expires without a message. timeout is measured in milliseconds. """ - logger.debug("Waiting for ResultsIncoming message") socks = dict(self.poller.poll(timeout=timeout_ms)) if self.results_receiver in socks and socks[self.results_receiver] == zmq.POLLIN: m = self.results_receiver.recv_multipart() From f8c9cd1a19d88fcad6eca1826bcd0c1681e6c74f Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 14 Jan 2025 14:15:23 +0000 Subject: [PATCH 38/39] Tidy mess from initial TaskRecord implementation (#3746) TaskRecord was merged in PR #2392 from a separate development branch, and contained some cruft which this PR removes: * a comment on checkpoint, that was a note on that development branch, not relevant to the main codebase * retries_left was removed in #1773 and has never been used in TaskRecord * fn_has was removed in #1945 and has never been used in TaskRecord ## Type of change - Code maintenance/cleanup Co-authored-by: Kevin Hunter Kesling --- parsl/dataflow/taskrecord.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/parsl/dataflow/taskrecord.py b/parsl/dataflow/taskrecord.py index f48aecceb9..0621ab8f41 100644 --- a/parsl/dataflow/taskrecord.py +++ b/parsl/dataflow/taskrecord.py @@ -43,12 +43,11 @@ class TaskRecord(TypedDict, total=False): executed on. """ - retries_left: int fail_count: int fail_cost: float fail_history: List[str] - checkpoint: bool # this change is also in #1516 + checkpoint: bool """Should this task be checkpointed? """ @@ -68,7 +67,6 @@ class TaskRecord(TypedDict, total=False): # these three could be more strongly typed perhaps but I'm not thinking about that now func: Callable - fn_hash: str args: Sequence[Any] # in some places we uses a Tuple[Any, ...] and in some places a List[Any]. # This is an attempt to correctly type both of those. From 83a20300351c8cda66a8878bf30fdc594ddb2092 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 14 Jan 2025 14:44:40 +0000 Subject: [PATCH 39/39] Remove description of exception handling from exception text (#3747) An exception should describe the exception, not other associated behaviour. ## Type of change - Update to human readable text: Documentation/error messages/comments Co-authored-by: Kevin Hunter Kesling --- parsl/executors/high_throughput/errors.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/executors/high_throughput/errors.py b/parsl/executors/high_throughput/errors.py index 2b94b34869..cb2ba5d26f 100644 --- a/parsl/executors/high_throughput/errors.py +++ b/parsl/executors/high_throughput/errors.py @@ -27,7 +27,7 @@ def __init__(self, interchange_version: str, manager_version: str): def __str__(self) -> str: return ( f"Manager version info {self.manager_version} does not match interchange" - f" version info {self.interchange_version}, causing a critical failure" + f" version info {self.interchange_version}" )