From 9783b7949f8f04ed8ea102b402e860aba377b275 Mon Sep 17 00:00:00 2001 From: Yadu Nand Babuji Date: Wed, 2 Aug 2023 13:05:55 -0500 Subject: [PATCH 01/43] Change log level for get_all_addresses and get_any_address (#2850) Logging an exception for every missed address resolution has these issues: * Even when there are no faults, users stumble on these and we receive spurious fault reports over slack. It makes for a bad first impression * These non-critical errors act as a diversion from real faults for users attempting to debug on their own. --- parsl/addresses.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/parsl/addresses.py b/parsl/addresses.py index 783aad4e97..e5a8b9ad61 100644 --- a/parsl/addresses.py +++ b/parsl/addresses.py @@ -110,7 +110,7 @@ def get_all_addresses() -> Set[str]: try: s_addresses.add(address_by_interface(interface)) except Exception: - logger.exception("Ignoring failure to fetch address from interface {}".format(interface)) + logger.info("Ignoring failure to fetch address from interface {}".format(interface)) resolution_functions: List[Callable[[], str]] resolution_functions = [address_by_hostname, address_by_route, address_by_query] @@ -118,7 +118,7 @@ def get_all_addresses() -> Set[str]: try: s_addresses.add(f()) except Exception: - logger.exception("Ignoring an address finder exception") + logger.info("Ignoring an address finder exception") return s_addresses @@ -137,7 +137,7 @@ def get_any_address() -> str: addr = address_by_interface(interface) return addr except Exception: - logger.exception("Ignoring failure to fetch address from interface {}".format(interface)) + logger.info("Ignoring failure to fetch address from interface {}".format(interface)) resolution_functions: List[Callable[[], str]] resolution_functions = [address_by_hostname, address_by_route, address_by_query] @@ -146,7 +146,7 @@ def get_any_address() -> str: addr = f() return addr except Exception: - logger.exception("Ignoring an address finder exception") + logger.info("Ignoring an address finder exception") if addr == '': raise Exception('Cannot find address of the local machine.') From dd754b1ae791dd7ba57ba807e28cf5903ce5db20 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 3 Aug 2023 18:39:23 +0200 Subject: [PATCH 02/43] Remove monitoring vs serialization cache as causes space explosion (#2852) See issue #2848 for a bunch of description and benchmarking. --- parsl/monitoring/remote.py | 128 ++++++++++++++++--------------------- 1 file changed, 56 insertions(+), 72 deletions(-) diff --git a/parsl/monitoring/remote.py b/parsl/monitoring/remote.py index 0aaeb10f68..479a7ccfa4 100644 --- a/parsl/monitoring/remote.py +++ b/parsl/monitoring/remote.py @@ -14,9 +14,6 @@ logger = logging.getLogger(__name__) -monitoring_wrapper_cache: Dict -monitoring_wrapper_cache = {} - def monitor_wrapper(f: Any, # per app args: Sequence, # per invocation @@ -33,76 +30,63 @@ def monitor_wrapper(f: Any, # per app """Wrap the Parsl app with a function that will call the monitor function and point it at the correct pid when the task begins. """ - # this makes assumptions that when subsequently executed with the same - # cache key, then the relevant parameters will not have changed from the - # first invocation with that cache key (otherwise, the resulting cached - # closure will be incorrectly cached) - cache_key = (run_id, f, radio_mode) - - if cache_key in monitoring_wrapper_cache: - wrapped = monitoring_wrapper_cache[cache_key] - - else: + @wraps(f) + def wrapped(*args: List[Any], **kwargs: Dict[str, Any]) -> Any: + task_id = kwargs.pop('_parsl_monitoring_task_id') + try_id = kwargs.pop('_parsl_monitoring_try_id') + terminate_event = Event() + # Send first message to monitoring router + send_first_message(try_id, + task_id, + monitoring_hub_url, + run_id, + radio_mode, + run_dir) + + if monitor_resources: + # create the monitor process and start + pp = ForkProcess(target=monitor, + args=(os.getpid(), + try_id, + task_id, + monitoring_hub_url, + run_id, + radio_mode, + logging_level, + sleep_dur, + run_dir, + terminate_event), + daemon=True, + name="Monitor-Wrapper-{}".format(task_id)) + pp.start() + p = pp + # TODO: awkwardness because ForkProcess is not directly a constructor + # and type-checking is expecting p to be optional and cannot + # narrow down the type of p in this block. + + else: + p = None - @wraps(f) - def wrapped(*args: List[Any], **kwargs: Dict[str, Any]) -> Any: - task_id = kwargs.pop('_parsl_monitoring_task_id') - try_id = kwargs.pop('_parsl_monitoring_try_id') - terminate_event = Event() - # Send first message to monitoring router - send_first_message(try_id, - task_id, - monitoring_hub_url, - run_id, - radio_mode, - run_dir) - - if monitor_resources: - # create the monitor process and start - pp = ForkProcess(target=monitor, - args=(os.getpid(), - try_id, - task_id, - monitoring_hub_url, - run_id, - radio_mode, - logging_level, - sleep_dur, - run_dir, - terminate_event), - daemon=True, - name="Monitor-Wrapper-{}".format(task_id)) - pp.start() - p = pp - # TODO: awkwardness because ForkProcess is not directly a constructor - # and type-checking is expecting p to be optional and cannot - # narrow down the type of p in this block. - - else: - p = None - - try: - return f(*args, **kwargs) - finally: - # There's a chance of zombification if the workers are killed by some signals (?) - if p: - terminate_event.set() - p.join(30) # 30 second delay for this -- this timeout will be hit in the case of an unusually long end-of-loop - if p.exitcode is None: - logger.warn("Event-based termination of monitoring helper took too long. Using process-based termination.") - p.terminate() - # DANGER: this can corrupt shared queues according to docs. - # So, better that the above termination event worked. - # This is why this log message is a warning - p.join() - - send_last_message(try_id, - task_id, - monitoring_hub_url, - run_id, - radio_mode, run_dir) - - monitoring_wrapper_cache[cache_key] = wrapped + try: + return f(*args, **kwargs) + finally: + # There's a chance of zombification if the workers are killed by some signals (?) + if p: + terminate_event.set() + p.join(30) # 30 second delay for this -- this timeout will be hit in the case of an unusually long end-of-loop + if p.exitcode is None: + logger.warn("Event-based termination of monitoring helper took too long. Using process-based termination.") + p.terminate() + # DANGER: this can corrupt shared queues according to docs. + # So, better that the above termination event worked. + # This is why this log message is a warning + p.join() + + send_last_message(try_id, + task_id, + monitoring_hub_url, + run_id, + radio_mode, run_dir) new_kwargs = kwargs.copy() new_kwargs['_parsl_monitoring_task_id'] = x_task_id From 84803bdc247a916b99e4995fca78ba6b2fd4fcc2 Mon Sep 17 00:00:00 2001 From: Thanh Son Phung Date: Sat, 5 Aug 2023 03:52:30 -0400 Subject: [PATCH 03/43] TaskVineExecutor: add serverless execution mode for tasks (#2849) This PR is a follow-up to PR https://github.com/Parsl/parsl/pull/2809 with the following changes: - Add support for serverless execution mode for tasks. Users can now declare serverless tasks like this: ``` @python_app def f(x, parsl_resource_specification={'exec_mode': 'serverless'}): ... ``` Under the hood, a library task is deployed to each worker and forks itself to execute an actual task/function call. - Add support for full conda environment replication on worker nodes for both serverless and regular tasks. Given a conda environment, TaskVine will distribute and cache it efficiently, so a shared filesystem is not required. The benefit mainly lies in the assumption that the conda environment can be huge in size and thousands of tasks asking the shared filesystem for the same environment directory at the same time is not very performant. TaskVine on the other hand keeps track of where files are at and has an implicit throttle mechanism to data distribution so things don't go haywire. - Add support for no conda environment replication, provided that environment is properly setup on worker nodes. Turning on `shared_fs` flag and add some setup code to `init_command` would turn off almost all TaskVine data-aware magic and the workflow run will mostly use the shared filesystem instead (e.g., `init_command="conda run -p path/to/conda"`). This serves as the baseline behavior for TaskVineExecutor. - Refactor `executor.py` into {`executor.py`, `manager.py`, and `factory.py`} as they are different components. - Change API: `use_factory` to `worker_launch_method`. - Some variable renaming and additional logging. --- .../executors/taskvine/exec_parsl_function.py | 59 +- parsl/executors/taskvine/executor.py | 667 ++++-------------- parsl/executors/taskvine/factory.py | 64 ++ parsl/executors/taskvine/manager.py | 468 ++++++++++++ parsl/executors/taskvine/manager_config.py | 1 + parsl/executors/taskvine/utils.py | 10 + parsl/tests/configs/taskvine_ex.py | 2 +- parsl/tests/scaling_tests/vineex_local.py | 2 +- 8 files changed, 712 insertions(+), 561 deletions(-) create mode 100644 parsl/executors/taskvine/factory.py create mode 100644 parsl/executors/taskvine/manager.py diff --git a/parsl/executors/taskvine/exec_parsl_function.py b/parsl/executors/taskvine/exec_parsl_function.py index cb8dc36354..0a408e1c03 100644 --- a/parsl/executors/taskvine/exec_parsl_function.py +++ b/parsl/executors/taskvine/exec_parsl_function.py @@ -78,17 +78,10 @@ def remap_all_files(mapping, fn_args, fn_kwargs): remap_location(mapping, maybe_file) -def unpack_function(function_info, user_namespace): - """ Unpack a function according to its encoding scheme.""" - return unpack_byte_code_function(function_info, user_namespace) - - -def unpack_byte_code_function(function_info, user_namespace): - """ Returns a function object, a default name, positional arguments, and keyword arguments - for a function.""" - from parsl.serialize import unpack_apply_message - func, args, kwargs = unpack_apply_message(function_info["byte code"], user_namespace, copy=False) - return (func, 'parsl_function_name', args, kwargs) +def unpack_object(serialized_obj, user_namespace): + from parsl.serialize import deserialize + obj = deserialize(serialized_obj) + return obj def encode_function(user_namespace, fn, fn_name, fn_args, fn_kwargs): @@ -119,7 +112,7 @@ def encode_byte_code_function(user_namespace, fn, fn_name, args_name, kwargs_nam return code -def load_function(map_file, function_file): +def load_function(map_file, function_file, argument_file): # Decodes the function and its file arguments to be executed into # function_code, and updates a user namespace with the function name and # the variable named result_name. When the function is executed, its result @@ -130,9 +123,14 @@ def load_function(map_file, function_file): user_ns = locals() user_ns.update({'__builtins__': __builtins__}) - function_info = load_pickled_file(function_file) + packed_function = load_pickled_file(function_file) + packed_argument = load_pickled_file(argument_file) - (fn, fn_name, fn_args, fn_kwargs) = unpack_function(function_info, user_ns) + fn = unpack_object(packed_function, user_ns) + args_dict = unpack_object(packed_argument, user_ns) + fn_args = args_dict['args'] + fn_kwargs = args_dict['kwargs'] + fn_name = 'parsl_tmp_func_name' mapping = load_pickled_file(map_file) remap_all_files(mapping, fn_args, fn_kwargs) @@ -145,29 +143,16 @@ def load_function(map_file, function_file): def execute_function(namespace, function_code, result_name): # On executing the function inside the namespace, its result will be in a # variable named result_name. - exec(function_code, namespace, namespace) result = namespace.get(result_name) return result -if __name__ == "__main__": +def run(map_file, function_file, argument_file, result_file): try: - # parse the three required command line arguments: - # map_file: contains a pickled dictionary to map original names to - # names at the execution site. - # function_file: contains the pickled parsl function to execute. - # result_file: any output (including exceptions) will be written to - # this file. try: - (map_file, function_file, result_file) = sys.argv[1:] - except ValueError: - print("Usage:\n\t{} function result mapping\n".format(sys.argv[0])) - raise - - try: - (namespace, function_code, result_name) = load_function(map_file, function_file) + (namespace, function_code, result_name) = load_function(map_file, function_file, argument_file) except Exception: print("There was an error setting up the function for execution.") raise @@ -188,3 +173,19 @@ def execute_function(namespace, function_code, result_name): print("Could not write to result file.") traceback.print_exc() sys.exit(1) + + +if __name__ == "__main__": + # parse the four required command line arguments: + # map_file: contains a pickled dictionary to map original names to + # names at the execution site. + # function_file: contains the pickled parsl function to execute. + # argument_file: contains the pickled arguments to the function call. + # result_file: any output (including exceptions) will be written to + # this file. + try: + (map_file, function_file, argument_file, result_file) = sys.argv[1:] + except ValueError: + print("Usage:\n\t{} function argument result mapping\n".format(sys.argv[0])) + raise + run(map_file, function_file, argument_file, result_file) diff --git a/parsl/executors/taskvine/executor.py b/parsl/executors/taskvine/executor.py index 35c86defd8..3d4669707a 100644 --- a/parsl/executors/taskvine/executor.py +++ b/parsl/executors/taskvine/executor.py @@ -11,7 +11,6 @@ import hashlib import subprocess import os -import time import pickle import queue import inspect @@ -24,9 +23,8 @@ # Import Parsl constructs import parsl.utils as putils -from parsl.utils import setproctitle from parsl.data_provider.staging import Staging -from parsl.serialize import pack_apply_message +from parsl.serialize import serialize from parsl.data_provider.files import File from parsl.errors import OptionalModuleMissing from parsl.providers.base import ExecutionProvider @@ -34,35 +32,20 @@ from parsl.process_loggers import wrap_with_logs from parsl.addresses import get_any_address from parsl.executors.errors import ExecutorError -from parsl.executors.errors import UnsupportedFeatureError from parsl.executors.status_handling import BlockProviderExecutor from parsl.executors.taskvine import exec_parsl_function from parsl.executors.taskvine.manager_config import TaskVineManagerConfig from parsl.executors.taskvine.factory_config import TaskVineFactoryConfig from parsl.executors.taskvine.errors import TaskVineTaskFailure from parsl.executors.taskvine.errors import TaskVineManagerFailure -from parsl.executors.taskvine.errors import TaskVineFactoryFailure from parsl.executors.taskvine.utils import ParslTaskToVine -from parsl.executors.taskvine.utils import VineTaskToParsl from parsl.executors.taskvine.utils import ParslFileToVine +from parsl.executors.taskvine.manager import _taskvine_submit_wait +from parsl.executors.taskvine.factory import _taskvine_factory # Import other libraries import typeguard -# Import TaskVine python modules -try: - from ndcctools.taskvine import cvine - from ndcctools.taskvine import Manager - from ndcctools.taskvine import Factory - from ndcctools.taskvine import Task - from ndcctools.taskvine.cvine import VINE_ALLOCATION_MODE_MAX_THROUGHPUT - from ndcctools.taskvine.cvine import VINE_ALLOCATION_MODE_EXHAUSTIVE_BUCKETING - from ndcctools.taskvine.cvine import VINE_ALLOCATION_MODE_MAX -except ImportError: - _taskvine_enabled = False -else: - _taskvine_enabled = True - logger = logging.getLogger(__name__) @@ -88,10 +71,11 @@ class TaskVineExecutor(BlockProviderExecutor, putils.RepresentationMixin): with respect to other executors. Default is "TaskVineExecutor". - use_factory: bool - Choose to whether use either the Parsl provider or - TaskVine factory to scale workers. - Default is False. + worker_launch_method: str + Choose to use Parsl provider, TaskVine factory, or + manual user-provided workers to scale workers. + Options are among {'provider', 'factory', 'manual'}. + Default is 'factory'. manager_config: TaskVineManagerConfig Configuration for the TaskVine manager. Default @@ -114,80 +98,90 @@ class TaskVineExecutor(BlockProviderExecutor, putils.RepresentationMixin): @typeguard.typechecked def __init__(self, label: str = "TaskVineExecutor", - use_factory: bool = False, + worker_launch_method: str = 'factory', manager_config: TaskVineManagerConfig = TaskVineManagerConfig(), factory_config: TaskVineFactoryConfig = TaskVineFactoryConfig(), provider: Optional[ExecutionProvider] = LocalProvider(init_blocks=1), storage_access: Optional[List[Staging]] = None): - # If TaskVine factory is used, disable the Parsl provider - if use_factory: + # Set worker launch option for this executor + if worker_launch_method == 'factory' or worker_launch_method == 'manual': provider = None + elif worker_launch_method != 'provider': + raise ExecutorError(self, "Worker launch option '{worker_launch_method}' \ + is not supported.") # Initialize the parent class with the execution provider and block error handling enabled. + # If provider is None, then no worker is launched via the provider method. BlockProviderExecutor.__init__(self, provider=provider, block_error_handler=True) # Raise an exception if there's a problem importing TaskVine - if not _taskvine_enabled: + try: + import ndcctools.taskvine + logger.debug(f'TaskVine default port: {ndcctools.taskvine.cvine.VINE_DEFAULT_PORT}') + except ImportError: raise OptionalModuleMissing(['taskvine'], "TaskVineExecutor requires the taskvine module.") # Executor configurations self.label = label - self.use_factory = use_factory + self.worker_launch_method = worker_launch_method self.manager_config = manager_config self.factory_config = factory_config self.storage_access = storage_access # Queue to send ready tasks from TaskVine executor process to TaskVine manager process - self.ready_task_queue: multiprocessing.Queue = multiprocessing.Queue() + self._ready_task_queue: multiprocessing.Queue = multiprocessing.Queue() # Queue to send finished tasks from TaskVine manager process to TaskVine executor process - self.finished_task_queue: multiprocessing.Queue = multiprocessing.Queue() + self._finished_task_queue: multiprocessing.Queue = multiprocessing.Queue() # Value to signal whether the manager and factory processes should stop running - self.should_stop = multiprocessing.Value(c_bool, False) + self._should_stop = multiprocessing.Value(c_bool, False) # TaskVine manager process - self.submit_process = None + self._submit_process = None # TaskVine factory process - self.factory_process = None + self._factory_process = None # Executor thread to collect results from TaskVine manager and set # tasks' futures to done status. - self.collector_thread = None + self._collector_thread = None # track task id of submitted parsl tasks # task ids are incremental and start from 0 - self.executor_task_counter = 0 + self._executor_task_counter = 0 # track number of tasks that are waiting/running - self.outstanding_tasks = 0 + self._outstanding_tasks = 0 - # Lock for threads to access self.outstanding_tasks attribute - self.outstanding_tasks_lock = threading.Lock() + # Lock for threads to access self._outstanding_tasks attribute + self._outstanding_tasks_lock = threading.Lock() # Threading lock to manage self.tasks dictionary object, which maps a task id # to its future object. - self.tasks_lock = threading.Lock() + self._tasks_lock = threading.Lock() # Worker command to be given to an execution provider (e.g., local or Condor) - self.worker_command = "" + self._worker_command = "" - # Path to directory that holds all tasks' data and results, only used when - # manager's task mode is 'regular'. - self.function_data_dir = "" + # Path to directory that holds all tasks' data and results. + self._function_data_dir = "" # helper scripts to prepare package tarballs for Parsl apps - self.package_analyze_script = shutil.which("poncho_package_analyze") - self.package_create_script = shutil.which("poncho_package_create") + self._package_analyze_script = shutil.which("poncho_package_analyze") + self._package_create_script = shutil.which("poncho_package_create") + if self._package_analyze_script is None or self._package_create_script is None: + self._poncho_available = False + else: + self._poncho_available = True def _get_launch_command(self, block_id): # Implements BlockProviderExecutor's abstract method. # This executor uses different terminology for worker/launch # commands than in htex. - return f"PARSL_WORKER_BLOCK_ID={block_id} {self.worker_command}" + return f"PARSL_WORKER_BLOCK_ID={block_id} {self._worker_command}" def __synchronize_manager_factory_comm_settings(self): # Synchronize the communication settings between the manager and the factory @@ -209,6 +203,7 @@ def __synchronize_manager_factory_comm_settings(self): self.factory_config._project_address = self.manager_config.address self.factory_config._project_name = self.manager_config.project_name self.factory_config._project_password_file = self.manager_config.project_password_file + logger.debug('Communication settings between TaskVine manager and factory synchronized.') def __create_data_and_logging_dirs(self): # Create neccessary data and logging directories @@ -217,16 +212,21 @@ def __create_data_and_logging_dirs(self): run_dir = self.run_dir # Create directories for data and results - self.function_data_dir = os.path.join(run_dir, self.label, "function_data") log_dir = os.path.join(run_dir, self.label) - logger.debug("function data directory: {}\nlog directory: {}".format(self.function_data_dir, log_dir)) + self._function_data_dir = os.path.join(run_dir, self.label, "function_data") os.makedirs(log_dir) - os.makedirs(self.function_data_dir) + os.makedirs(self._function_data_dir) - # put TaskVine logs inside run directory of Parsl by default + # put TaskVine logs outside of a Parsl run as TaskVine caches between runs while + # Parsl does not. + vine_log_dir = os.path.join(os.path.dirname(run_dir), self.label) if self.manager_config.vine_log_dir is None: - self.manager_config.vine_log_dir = log_dir - self.factory_config.scratch_dir = log_dir + self.manager_config.vine_log_dir = vine_log_dir + + # factory logs go with manager logs regardless + self.factory_config.scratch_dir = self.manager_config.vine_log_dir + logger.debug(f"Function data directory: {self._function_data_dir}, log directory: {log_dir}") + logger.debug(f"TaskVine manager log directory: {self.manager_config.vine_log_dir}, factory log directory: {self.factory_config.scratch_dir}") def start(self): """Create submit process and collector thread to create, send, and @@ -242,53 +242,53 @@ def start(self): logger.debug("Starting TaskVineExecutor") # Create a process to run the TaskVine manager. - submit_process_kwargs = {"ready_task_queue": self.ready_task_queue, - "finished_task_queue": self.finished_task_queue, - "should_stop": self.should_stop, + submit_process_kwargs = {"ready_task_queue": self._ready_task_queue, + "finished_task_queue": self._finished_task_queue, + "should_stop": self._should_stop, "manager_config": self.manager_config} - self.submit_process = multiprocessing.Process(target=_taskvine_submit_wait, - name="TaskVine-Submit-Process", - kwargs=submit_process_kwargs) + self._submit_process = multiprocessing.Process(target=_taskvine_submit_wait, + name="TaskVine-Submit-Process", + kwargs=submit_process_kwargs) # Create a process to run the TaskVine factory if enabled. - if self.use_factory: - factory_process_kwargs = {"should_stop": self.should_stop, + if self.worker_launch_method == 'factory': + factory_process_kwargs = {"should_stop": self._should_stop, "factory_config": self.factory_config} - self.factory_process = multiprocessing.Process(target=_taskvine_factory, - name="TaskVine-Factory-Process", - kwargs=factory_process_kwargs) + self._factory_process = multiprocessing.Process(target=_taskvine_factory, + name="TaskVine-Factory-Process", + kwargs=factory_process_kwargs) # Run thread to collect results and set tasks' futures. - self.collector_thread = threading.Thread(target=self._collect_taskvine_results, - name="TaskVine-Collector-Thread") + self._collector_thread = threading.Thread(target=self._collect_taskvine_results, + name="TaskVine-Collector-Thread") # Interpreter can exit without waiting for this thread. - self.collector_thread.daemon = True + self._collector_thread.daemon = True # Begin work - self.submit_process.start() + self._submit_process.start() # Run worker scaler either with Parsl provider or TaskVine factory - if self.use_factory: - self.factory_process.start() + if self.worker_launch_method == 'factory': + self._factory_process.start() else: self.initialize_scaling() - self.collector_thread.start() + self._collector_thread.start() logger.debug("All components in TaskVineExecutor started") def _path_in_task(self, executor_task_id, *path_components): """ - Only used when task mode is `regular`. Returns a filename fixed and specific to a task. It is used for the following filename's: (not given): The subdirectory per task that contains function, result, etc. 'function': Pickled file that contains the function to be executed. + 'argument': Pickled file that contains the arguments of the function call. 'result': Pickled file that (will) contain the result of the function. 'map': Pickled file with a dict between local parsl names, and remote taskvine names. """ task_dir = "{:04d}".format(executor_task_id) - return os.path.join(self.function_data_dir, task_dir, *path_components) + return os.path.join(self._function_data_dir, task_dir, *path_components) def submit(self, func, resource_specification, *args, **kwargs): """Processes the Parsl app by its arguments and submits the function @@ -304,19 +304,18 @@ def submit(self, func, resource_specification, *args, **kwargs): resource_specification: dict Dictionary containing relevant info about task. Include information about resources of task, execution mode - of task (out of {regular, python, serverless}), and which app - type this function was submitted as (out of {python, bash}). + of task (out of {regular, serverless}). args : list Arguments to the Parsl app kwargs : dict Keyword arguments to the Parsl app """ - # Default execution mode of apps is regular (using TaskVineExecutor serialization and execution mode) - exec_mode = resource_specification.get('exec_mode', 'regular') - logger.debug(f'Got resource specification: {resource_specification}') + # Default execution mode of apps is regular + exec_mode = resource_specification.get('exec_mode', 'regular') + # Detect resources and features of a submitted Parsl app cores = None memory = None @@ -343,10 +342,10 @@ def submit(self, func, resource_specification, *args, **kwargs): running_time_min = resource_specification[k] # Assign executor task id to app - executor_task_id = self.executor_task_counter - self.executor_task_counter += 1 + executor_task_id = self._executor_task_counter + self._executor_task_counter += 1 - # Create a per task directory for the function, map, and result files + # Create a per task directory for the function, argument, map, and result files os.mkdir(self._path_in_task(executor_task_id)) input_files = [] @@ -372,28 +371,32 @@ def submit(self, func, resource_specification, *args, **kwargs): # Create a Future object and have it be mapped from the task ID in the tasks dictionary fu = Future() fu.parsl_executor_task_id = executor_task_id - with self.tasks_lock: + with self._tasks_lock: self.tasks[str(executor_task_id)] = fu - logger.debug("Creating task {} for function {} of type {} with args {}".format(executor_task_id, func, type(func), args)) - + # Setup files to be used on a worker to execute the function function_file = None + argument_file = None result_file = None map_file = None - # Use executor's serialization method if app mode is 'regular' - if exec_mode == 'regular': - # Get path to files that will contain the pickled function, result, and map of input and output files - function_file = self._path_in_task(executor_task_id, "function") - result_file = self._path_in_task(executor_task_id, "result") - map_file = self._path_in_task(executor_task_id, "map") - logger.debug("Creating executor task {} with function at: {} and result to be found at: {}".format(executor_task_id, function_file, result_file)) + # Get path to files that will contain the pickled function, + # arguments, result, and map of input and output files + function_file = self._path_in_task(executor_task_id, "function") + argument_file = self._path_in_task(executor_task_id, "argument") + result_file = self._path_in_task(executor_task_id, "result") + map_file = self._path_in_task(executor_task_id, "map") - # Pickle the result into object to pass into message buffer - self._serialize_function(function_file, func, args, kwargs) + logger.debug("Creating executor task {} with function at: {}, argument at: {}, \ + and result to be found at: {}".format(executor_task_id, function_file, argument_file, result_file)) - # Construct the map file of local filenames at worker - self._construct_map_file(map_file, input_files, output_files) + # Serialize function object and arguments, separately + self._serialize_object(function_file, func) + args_dict = {'args': args, 'kwargs': kwargs} + self._serialize_object(argument_file, args_dict) + + # Construct the map file of local filenames at worker + self._construct_map_file(map_file, input_files, output_files) # Register a tarball containing all package dependencies for this app if instructed if self.manager_config.app_pack: @@ -401,9 +404,6 @@ def submit(self, func, resource_specification, *args, **kwargs): else: env_pkg = None - if not self.submit_process.is_alive(): - raise ExecutorError(self, "taskvine Submit Process is not alive") - # Create message to put into the message queue logger.debug("Placing task {} on message queue".format(executor_task_id)) @@ -411,9 +411,6 @@ def submit(self, func, resource_specification, *args, **kwargs): if category is None: category = func.__name__ if self.manager_config.autocategory else 'parsl-default' - # support for python and serverless exec mode delayed - if exec_mode == 'python' or exec_mode == 'serverless': - raise UnsupportedFeatureError(f'Execution mode {exec_mode} is not currently supported.', 'TaskVineExecutor', None) task_info = ParslTaskToVine(executor_id=executor_task_id, exec_mode=exec_mode, category=category, @@ -421,6 +418,7 @@ def submit(self, func, resource_specification, *args, **kwargs): output_files=output_files, map_file=map_file, function_file=function_file, + argument_file=argument_file, result_file=result_file, cores=cores, memory=memory, @@ -431,12 +429,17 @@ def submit(self, func, resource_specification, *args, **kwargs): env_pkg=env_pkg) # Send ready task to manager process - self.ready_task_queue.put_nowait(task_info) + if not self._submit_process.is_alive(): + raise ExecutorError(self, "taskvine Submit Process is not alive") + + self._ready_task_queue.put_nowait(task_info) # Increment outstanding task counter - with self.outstanding_tasks_lock: - self.outstanding_tasks += 1 + with self._outstanding_tasks_lock: + self._outstanding_tasks += 1 + # Return the future for this function, will be set by collector thread when result + # comes back from the TaskVine manager. return fu def _construct_worker_command(self): @@ -458,19 +461,16 @@ def _patch_providers(self): # (Currently only for the CondorProvider) if isinstance(self.provider, CondorProvider): path_to_worker = shutil.which('vine_worker') - self.worker_command = './' + self.worker_command + self._worker_command = './' + self._worker_command self.provider.transfer_input_files.append(path_to_worker) if self.project_password_file: self.provider.transfer_input_files.append(self.project_password_file) - def _serialize_function(self, fn_path, parsl_fn, parsl_fn_args, parsl_fn_kwargs): - """Takes the function application parsl_fn(*parsl_fn_args, **parsl_fn_kwargs) - and serializes it to the file fn_path.""" - function_info = {"byte code": pack_apply_message(parsl_fn, parsl_fn_args, parsl_fn_kwargs, - buffer_threshold=1024 * 1024)} - - with open(fn_path, "wb") as f_out: - pickle.dump(function_info, f_out) + def _serialize_object(self, path, obj): + """Takes any object and serializes it to the file path.""" + serialized_obj = serialize(obj, buffer_threshold=1024 * 1024) + with open(path, 'wb') as f_out: + pickle.dump(serialized_obj, f_out) def _construct_map_file(self, map_file, input_files, output_files): """ Map local filepath of parsl files to the filenames at the execution worker. @@ -511,6 +511,10 @@ def _std_output_to_vine(self, fdname, stdfspec): def _prepare_package(self, fn, extra_pkgs): """ Look at source code of apps to figure out their package depedencies and output a tarball containing those to send along with tasks for execution.""" + + if not self._poncho_available: + raise ExecutorError(self, 'poncho package is not available to individually pack apps.') + fn_id = id(fn) fn_name = fn.__name__ if fn_id in self.cached_envs: @@ -521,7 +525,7 @@ def _prepare_package(self, fn, extra_pkgs): os.makedirs(pkg_dir, exist_ok=True) with tempfile.NamedTemporaryFile(suffix='.yaml') as spec: logger.info("Analyzing dependencies of %s", fn_name) - analyze_cmdline = [self.package_analyze_script, exec_parsl_function.__file__, '-', spec.name] + analyze_cmdline = [self._package_analyze_script, exec_parsl_function.__file__, '-', spec.name] for p in extra_pkgs: analyze_cmdline += ["--extra-pkg", p] subprocess.run(analyze_cmdline, input=source_code, check=True) @@ -537,7 +541,7 @@ def _prepare_package(self, fn, extra_pkgs): os.close(fd) logger.info("Creating dependency package for %s", fn_name) logger.debug("Writing deps for %s to %s", fn_name, tarball) - subprocess.run([self.package_create_script, spec.name, tarball], stdout=subprocess.DEVNULL, check=True) + subprocess.run([self._package_create_script, spec.name, tarball], stdout=subprocess.DEVNULL, check=True) logger.debug("Done with conda-pack; moving %s to %s", tarball, pkg) os.rename(tarball, pkg) self.cached_envs[fn_id] = pkg @@ -550,7 +554,7 @@ def initialize_scaling(self): """ # Start scaling in/out logger.debug("Starting TaskVineExecutor with provider: %s", self.provider) - self.worker_command = self._construct_worker_command() + self._worker_command = self._construct_worker_command() self._patch_providers() if hasattr(self.provider, 'init_blocks'): @@ -563,8 +567,8 @@ def initialize_scaling(self): @property def outstanding(self) -> int: """Count the number of outstanding tasks.""" - logger.debug(f"Counted {self.outstanding_tasks} outstanding tasks") - return self.outstanding_tasks + logger.debug(f"Counted {self._outstanding_tasks} outstanding tasks") + return self._outstanding_tasks @property def workers_per_node(self) -> Union[int, float]: @@ -588,7 +592,7 @@ def shutdown(self, *args, **kwargs): collector thread, which shuts down the TaskVine system submission. """ logger.debug("TaskVine shutdown started") - self.should_stop.value = True + self._should_stop.value = True # Remove the workers that are still going kill_ids = [self.blocks[block] for block in self.blocks.keys()] @@ -598,12 +602,12 @@ def shutdown(self, *args, **kwargs): # Join all processes before exiting logger.debug("Joining on submit process") - self.submit_process.join() + self._submit_process.join() logger.debug("Joining on collector thread") - self.collector_thread.join() - if self.use_factory: + self._collector_thread.join() + if self.worker_launch_method == 'factory': logger.debug("Joining on factory process") - self.factory_process.join() + self._factory_process.join() logger.debug("TaskVine shutdown completed") return True @@ -614,22 +618,22 @@ def _collect_taskvine_results(self): """ logger.debug("Starting Collector Thread") try: - while not self.should_stop.value: - if not self.submit_process.is_alive(): + while not self._should_stop.value: + if not self._submit_process.is_alive(): raise ExecutorError(self, "taskvine Submit Process is not alive") - # Get the result message from the finished_task_queue + # Get the result message from the _finished_task_queue try: - task_report = self.finished_task_queue.get(timeout=1) + task_report = self._finished_task_queue.get(timeout=1) except queue.Empty: continue # Obtain the future from the tasks dictionary - with self.tasks_lock: + with self._tasks_lock: future = self.tasks.pop(task_report.executor_id) - logger.debug("Updating Future for Parsl Task {}".format(task_report.executor_id)) - logger.debug(f'task {task_report.executor_id} has result_received set to {task_report.result_received} and result to {task_report.result}') + logger.debug(f'Updating Future for Parsl Task: {task_report.executor_id}. \ + Task {task_report.executor_id} has result_received set to {task_report.result_received}') if task_report.result_received: future.set_result(task_report.result) else: @@ -638,412 +642,15 @@ def _collect_taskvine_results(self): future.set_exception(TaskVineTaskFailure(task_report.reason, task_report.result)) # decrement outstanding task counter - with self.outstanding_tasks_lock: - self.outstanding_tasks -= 1 + with self._outstanding_tasks_lock: + self._outstanding_tasks -= 1 finally: logger.debug(f"Marking all {self.outstanding} outstanding tasks as failed") logger.debug("Acquiring tasks_lock") - with self.tasks_lock: + with self._tasks_lock: logger.debug("Acquired tasks_lock") # set exception for tasks waiting for results that taskvine did not execute for fu in self.tasks.values(): if not fu.done(): fu.set_exception(TaskVineManagerFailure("taskvine executor failed to execute the task.")) logger.debug("Exiting Collector Thread") - - -@wrap_with_logs -def _taskvine_submit_wait(ready_task_queue=None, - finished_task_queue=None, - should_stop=None, - manager_config=None - ): - """Process to handle Parsl app submissions to the TaskVine objects. - Takes in Parsl functions submitted using submit(), and creates a - TaskVine task with the appropriate specifications, which is then - submitted to TaskVine. After tasks are completed, processes the - exit status and exit code of the task, and sends results to the - TaskVine collector thread. - To avoid python's global interpreter lock with taskvine's wait, this - function should be launched as a process, not as a lightweight thread. This - means that any communication should be done using the multiprocessing - module capabilities, rather than shared memory. - """ - logger.debug("Starting TaskVine Submit/Wait Process") - setproctitle("parsl: TaskVine submit/wait") - - # Enable debugging flags and create logging file - if manager_config.vine_log_dir is not None: - logger.debug("Setting debugging flags and creating logging file at {}".format(manager_config.vine_log_dir)) - - # Create TaskVine queue object - logger.debug("Creating TaskVine Object") - try: - logger.debug("Listening on port {}".format(manager_config.port)) - m = Manager(port=manager_config.port, - name=manager_config.project_name, - run_info_path=manager_config.vine_log_dir) - except Exception as e: - logger.error("Unable to create TaskVine object: {}".format(e)) - raise e - - # Specify TaskVine manager attributes - if manager_config.project_password_file: - m.set_password_file(manager_config.project_password_file) - - # Autolabeling resources require monitoring to be enabled - if manager_config.autolabel: - m.enable_monitoring() - if manager_config.autolabel_window is not None: - m.tune('category-steady-n-tasks', manager_config.autolabel_window) - - # Specify number of workers to wait for before sending the first task - if manager_config.wait_for_workers: - m.tune("wait-for-workers", manager_config.wait_for_workers) - - # Enable peer transfer feature between workers if specified - if manager_config.enable_peer_transfers: - m.enable_peer_transfers() - - # Get parent pid, useful to shutdown this process when its parent, the taskvine - # executor process, exits. - orig_ppid = os.getppid() - - result_file_of_task_id = {} # Mapping executor task id -> result file for active regular tasks. - - poncho_env_to_file = {} # Mapping poncho_env file to File object in TaskVine - - # Mapping of parsl local file name to TaskVine File object - # dict[str] -> vine File object - parsl_file_name_to_vine_file = {} - - # Mapping of tasks from vine id to parsl id - # Dict[str] -> str - vine_id_to_executor_task_id = {} - - # Find poncho run script to activate an environment tarball - poncho_run_script = shutil.which("poncho_package_run") - - # Declare helper scripts as cache-able and peer-transferable - package_run_script_file = m.declare_file(poncho_run_script, cache=True, peer_transfer=True) - exec_parsl_function_file = m.declare_file(exec_parsl_function.__file__, cache=True, peer_transfer=True) - - logger.debug("Entering main loop of TaskVine manager") - - while not should_stop.value: - # Monitor the task queue - ppid = os.getppid() - if ppid != orig_ppid: - logger.debug("new Process") - break - - # Submit tasks - while ready_task_queue.qsize() > 0 and not should_stop.value: - # Obtain task from ready_task_queue - try: - task = ready_task_queue.get(timeout=1) - logger.debug("Removing executor task from queue") - except queue.Empty: - logger.debug("Queue is empty") - continue - if task.exec_mode == 'regular': - # Create command string - launch_cmd = "python3 exec_parsl_function.py {mapping} {function} {result}" - if manager_config.init_command != '': - launch_cmd = "{init_cmd};" + launch_cmd - command_str = launch_cmd.format(init_cmd=manager_config.init_command, - mapping=os.path.basename(task.map_file), - function=os.path.basename(task.function_file), - result=os.path.basename(task.result_file)) - logger.debug("Sending executor task {} (mode: regular) with command: {}".format(task.executor_id, command_str)) - try: - t = Task(command_str) - except Exception as e: - logger.error("Unable to create executor task (mode:regular): {}".format(e)) - finished_task_queue.put_nowait(VineTaskToParsl(executor_id=task.executor_id, - result_received=False, - result=None, - reason="task could not be created by taskvine", - status=-1)) - continue - else: - raise Exception(f'Unrecognized task mode {task.exec_mode}. Exiting...') - - # Add environment file to the task if possible - # Prioritize local poncho environment over global poncho environment - # (local: use app_pack, global: use env_pack) - poncho_env_file = None - - # check if env_pack is specified - if manager_config.env_pack is not None: - # check if the environment file is not already created - if manager_config.env_pack not in poncho_env_to_file: - # if the environment is already packaged as a tarball, then add the file - # otherwise it is an environment name or path, so create a poncho tarball then add it - if not manager_config.env_pack.endswith('.tar.gz'): - env_tarball = str(uuid.uuid4()) + '.tar.gz' - subprocess.run([poncho_run_script, manager_config.env_pack, env_tarball], stdout=subprocess.DEVNULL, check=True) - poncho_env_file = m.declare_poncho(manager_config.env_pack, cache=True, peer_transfer=True) - poncho_env_to_file[manager_config.env_pack] = poncho_env_file - else: - poncho_env_file = poncho_env_to_file[manager_config.env_pack] - - # check if app_pack is used, override if possible - if task.env_pkg is not None: - if task.env_pkg not in poncho_env_to_file: - poncho_env_file = m.declare_poncho(task.env_pkg, cache=True, peer_transfer=True) - poncho_env_to_file[task.env_pkg] = poncho_env_file - else: - poncho_env_file = poncho_env_to_file[task.env_pkg] - - # Add environment to the task - if poncho_env_file is not None: - t.add_environment(poncho_env_file) - t.add_input(package_run_script_file, "poncho_package_run") - - t.set_category(task.category) - if manager_config.autolabel: - if manager_config.autolabel_algorithm == 'max-xput': - m.set_category_mode(task.category, VINE_ALLOCATION_MODE_MAX_THROUGHPUT) - elif manager_config.autolabel_algorithm == 'bucketing': - m.set_category_mode(task.category, VINE_ALLOCATION_MODE_EXHAUSTIVE_BUCKETING) - elif manager_config.autolabel_algorithm == 'max': - m.set_category_mode(task.category, VINE_ALLOCATION_MODE_MAX) - else: - logger.warning(f'Unrecognized autolabeling algorithm named {manager_config.autolabel_algorithm} for taskvine manager.') - raise Exception(f'Unrecognized autolabeling algorithm named {manager_config.autolabel_algorithm} for taskvine manager.') - - if task.cores is not None: - t.set_cores(task.cores) - if task.memory is not None: - t.set_memory(task.memory) - if task.disk is not None: - t.set_disk(task.disk) - if task.gpus is not None: - t.set_gpus(task.gpus) - if task.priority is not None: - t.set_priority(task.priority) - if task.running_time_min is not None: - t.set_time_min(task.running_time_min) - - if manager_config.max_retries is not None: - logger.debug(f"Specifying max_retries {manager_config.max_retries}") - t.set_retries(manager_config.max_retries) - else: - logger.debug("Not specifying max_retries") - - # Specify environment variables for the task - if manager_config.env_vars is not None: - for var in manager_config.env_vars: - t.set_env_var(str(var), str(manager_config.env_vars[var])) - - if task.exec_mode == 'regular': - # Add helper files that execute parsl functions on remote nodes - # only needed for tasks with 'regular' mode - t.add_input(exec_parsl_function_file, "exec_parsl_function.py") - - # Declare and add task-specific function, data, and result files to task - task_function_file = m.declare_file(task.function_file, cache=False, peer_transfer=False) - t.add_input(task_function_file, "function") - - task_map_file = m.declare_file(task.map_file, cache=False, peer_transfer=False) - t.add_input(task_map_file, "map") - - task_result_file = m.declare_file(task.result_file, cache=False, peer_transfer=False) - t.add_output(task_result_file, "result") - - result_file_of_task_id[str(task.executor_id)] = task.result_file - - logger.debug("Executor task id: {}".format(task.executor_id)) - - # Specify input/output files that need to be staged. - # Absolute paths are assumed to be in shared filesystem, and thus - # not staged by taskvine. - # Files that share the same local path are assumed to be the same - # and thus use the same Vine File object if detected. - if not manager_config.shared_fs: - for spec in task.input_files: - if spec.stage: - if spec.parsl_name in parsl_file_name_to_vine_file: - task_in_file = parsl_file_name_to_vine_file[spec.parsl_name] - else: - task_in_file = m.declare_file(spec.parsl_name, cache=spec.cache, peer_transfer=True) - parsl_file_name_to_vine_file[spec.parsl_name] = task_in_file - t.add_input(task_in_file, spec.parsl_name) - - for spec in task.output_files: - if spec.stage: - if spec.parsl_name in parsl_file_name_to_vine_file: - task_out_file = parsl_file_name_to_vine_file[spec.parsl_name] - else: - task_out_file = m.declare_file(spec.parsl_name, cache=spec.cache, peer_transfer=True) - t.add_output(task_out_file, spec.parsl_name) - - # Submit the task to the TaskVine object - logger.debug("Submitting executor task {}, {} to TaskVine".format(task.executor_id, t)) - try: - vine_id = m.submit(t) - logger.debug("Submitted executor task {} to TaskVine".format(task.executor_id)) - vine_id_to_executor_task_id[str(vine_id)] = str(task.executor_id), task.exec_mode - except Exception as e: - logger.error("Unable to submit task to taskvine: {}".format(e)) - finished_task_queue.put_nowait(VineTaskToParsl(executor_id=task.executor_id, - result_received=False, - result=None, - reason="task could not be submited to taskvine", - status=-1)) - continue - - logger.debug("Executor task {} submitted as TaskVine task with id {}".format(task.executor_id, vine_id)) - - # If the queue is not empty wait on the TaskVine queue for a task - task_found = True - if not m.empty(): - while task_found and not should_stop.value: - # Obtain the task from the queue - t = m.wait(1) - if t is None: - task_found = False - continue - logger.debug('Found a task') - executor_task_id = vine_id_to_executor_task_id[str(t.id)][0] - exec_mode_of_task = vine_id_to_executor_task_id[str(t.id)][1] - vine_id_to_executor_task_id.pop(str(t.id)) - # When a task is found - if exec_mode_of_task == 'regular': - result_file = result_file_of_task_id.pop(executor_task_id) - - logger.debug(f"completed executor task info: {executor_task_id}, {t.category}, {t.command}, {t.std_output}") - - # A tasks completes 'succesfully' if it has result file, - # and it can be loaded. This may mean that the 'success' is - # an exception. - logger.debug("Looking for result in {}".format(result_file)) - try: - with open(result_file, "rb") as f_in: - result = pickle.load(f_in) - logger.debug("Found result in {}".format(result_file)) - finished_task_queue.put_nowait(VineTaskToParsl(executor_id=executor_task_id, - result_received=True, - result=result, - reason=None, - status=t.exit_code)) - # If a result file could not be generated, explain the - # failure according to taskvine error codes. We generate - # an exception and wrap it with RemoteExceptionWrapper, to - # match the positive case. - except Exception as e: - reason = _explain_taskvine_result(t) - logger.debug("Did not find result in {}".format(result_file)) - logger.debug("Wrapper Script status: {}\nTaskVine Status: {}" - .format(t.exit_code, t.result)) - logger.debug("Task with executor id {} / vine id {} failed because:\n{}" - .format(executor_task_id, t.id, reason)) - finished_task_queue.put_nowait(VineTaskToParsl(executor_id=executor_task_id, - result_received=False, - result=e, - reason=reason, - status=t.exit_code)) - else: - raise Exception(f'Unknown exec mode for executor task {executor_task_id}: {exec_mode_of_task}.') - - logger.debug("Exiting TaskVine Monitoring Process") - return 0 - - -def _explain_taskvine_result(vine_task): - """Returns a string with the reason why a task failed according to taskvine.""" - - vine_result = vine_task.result - - reason = "taskvine result: " - if vine_result == cvine.VINE_RESULT_SUCCESS: - reason += "succesful execution with exit code {}".format(vine_task.return_status) - elif vine_result == cvine.VINE_RESULT_OUTPUT_MISSING: - reason += "The result file was not transfered from the worker.\n" - reason += "This usually means that there is a problem with the python setup,\n" - reason += "or the wrapper that executes the function." - reason += "\nTrace:\n" + str(vine_task.output) - elif vine_result == cvine.VINE_RESULT_INPUT_MISSING: - reason += "missing input file" - elif vine_result == cvine.VINE_RESULT_STDOUT_MISSING: - reason += "stdout has been truncated" - elif vine_result == cvine.VINE_RESULT_SIGNAL: - reason += "task terminated with a signal" - elif vine_result == cvine.VINE_RESULT_RESOURCE_EXHAUSTION: - reason += "task used more resources than requested" - elif vine_result == cvine.VINE_RESULT_MAX_END_TIME: - reason += "task ran past the specified end time" - elif vine_result == cvine.VINE_RESULT_UNKNOWN: - reason += "result could not be classified" - elif vine_result == cvine.VINE_RESULT_FORSAKEN: - reason += "task failed, but not a task error" - elif vine_result == cvine.VINE_RESULT_MAX_RETRIES: - reason += "unable to complete after specified number of retries" - elif vine_result == cvine.VINE_RESULT_MAX_WALL_TIME: - reason += "task ran for more than the specified time" - elif vine_result == cvine.VINE_RESULT_RMONITOR_ERROR: - reason += "task failed because the monitor did not produce an output" - elif vine_result == cvine.VINE_RESULT_OUTPUT_TRANSFER_ERROR: - reason += "task failed because output transfer fails" - elif vine_result == cvine.VINE_RESULT_FIXED_LOCATION_MISSING: - reason += "task failed because no worker could satisfy the fixed \n" - reason += "location input file requirements" - else: - reason += "unable to process TaskVine system failure" - return reason - - -@wrap_with_logs -def _taskvine_factory(should_stop, factory_config): - logger.debug("Starting TaskVine factory process") - - try: - # create the factory according to the project name if given - if factory_config._project_name: - factory = Factory(batch_type=factory_config.batch_type, - manager_name=factory_config._project_name, - ) - else: - factory = Factory(batch_type=factory_config.batch_type, - manager_host_port=f"{factory_config._project_address}:{factory_config._project_port}", - ) - except Exception as e: - raise TaskVineFactoryFailure(f'Cannot create factory with exception {e}') - - # Set attributes of this factory - if factory_config._project_password_file: - factory.password = factory_config._project_password_file - factory.factory_timeout = factory_config.factory_timeout - factory.scratch_dir = factory_config.scratch_dir - factory.min_workers = factory_config.min_workers - factory.max_workers = factory_config.max_workers - factory.workers_per_cycle = factory_config.workers_per_cycle - - if factory_config.worker_options: - factory.extra_options = factory_config.worker_options - factory.timeout = factory_config.worker_timeout - if factory_config.cores: - factory.cores = factory_config.cores - if factory_config.gpus: - factory.gpus = factory_config.gpus - if factory_config.memory: - factory.memory = factory_config.memory - if factory_config.disk: - factory.disk = factory_config.disk - if factory_config.python_env: - factory.python_env = factory_config.python_env - - if factory_config.condor_requirements: - factory.condor_requirements = factory_config.condor_requirements - if factory_config.batch_options: - factory.batch_options = factory_config.batch_options - - # setup factory context and sleep for a second in every loop to - # avoid wasting CPU - with factory: - while not should_stop.value: - time.sleep(1) - - logger.debug("Exiting TaskVine factory process") - return 0 diff --git a/parsl/executors/taskvine/factory.py b/parsl/executors/taskvine/factory.py new file mode 100644 index 0000000000..1967b89974 --- /dev/null +++ b/parsl/executors/taskvine/factory.py @@ -0,0 +1,64 @@ +import logging +import time + +from parsl.process_loggers import wrap_with_logs +from parsl.executors.taskvine.errors import TaskVineFactoryFailure + +from ndcctools.taskvine import Factory + +logger = logging.getLogger(__name__) + + +@wrap_with_logs +def _taskvine_factory(should_stop, factory_config): + logger.debug("Starting TaskVine factory process") + + try: + # create the factory according to the project name if given + if factory_config._project_name: + factory = Factory(batch_type=factory_config.batch_type, + manager_name=factory_config._project_name, + ) + else: + factory = Factory(batch_type=factory_config.batch_type, + manager_host_port=f"{factory_config._project_address}:{factory_config._project_port}", + ) + except Exception as e: + raise TaskVineFactoryFailure(f'Cannot create factory with exception {e}') + + # Set attributes of this factory + if factory_config._project_password_file: + factory.password = factory_config._project_password_file + factory.factory_timeout = factory_config.factory_timeout + factory.scratch_dir = factory_config.scratch_dir + factory.min_workers = factory_config.min_workers + factory.max_workers = factory_config.max_workers + factory.workers_per_cycle = factory_config.workers_per_cycle + + if factory_config.worker_options: + factory.extra_options = factory_config.worker_options + factory.timeout = factory_config.worker_timeout + if factory_config.cores: + factory.cores = factory_config.cores + if factory_config.gpus: + factory.gpus = factory_config.gpus + if factory_config.memory: + factory.memory = factory_config.memory + if factory_config.disk: + factory.disk = factory_config.disk + if factory_config.python_env: + factory.python_env = factory_config.python_env + + if factory_config.condor_requirements: + factory.condor_requirements = factory_config.condor_requirements + if factory_config.batch_options: + factory.batch_options = factory_config.batch_options + + # setup factory context and sleep for a second in every loop to + # avoid wasting CPU + with factory: + while not should_stop.value: + time.sleep(1) + + logger.debug("Exiting TaskVine factory process") + return 0 diff --git a/parsl/executors/taskvine/manager.py b/parsl/executors/taskvine/manager.py new file mode 100644 index 0000000000..3394ffce9e --- /dev/null +++ b/parsl/executors/taskvine/manager.py @@ -0,0 +1,468 @@ +import logging +import hashlib +import subprocess +import os +import pickle +import queue +import shutil +import uuid + +from parsl.utils import setproctitle +from parsl.process_loggers import wrap_with_logs +from parsl.executors.taskvine import exec_parsl_function +from parsl.executors.taskvine.utils import VineTaskToParsl +from parsl.executors.taskvine.utils import run_parsl_function + +try: + from ndcctools.taskvine import cvine + from ndcctools.taskvine import Manager + from ndcctools.taskvine import Task + from ndcctools.taskvine import FunctionCall + from ndcctools.taskvine.cvine import VINE_ALLOCATION_MODE_MAX_THROUGHPUT + from ndcctools.taskvine.cvine import VINE_ALLOCATION_MODE_EXHAUSTIVE_BUCKETING + from ndcctools.taskvine.cvine import VINE_ALLOCATION_MODE_MAX +except ImportError: + _taskvine_enabled = False +else: + _taskvine_enabled = True + +logger = logging.getLogger(__name__) + + +def _set_manager_attributes(m, config): + """ Set various manager global attributes.""" + if config.project_password_file: + m.set_password_file(config.project_password_file) + + # Autolabeling resources require monitoring to be enabled + if config.autolabel: + m.enable_monitoring() + if config.autolabel_window is not None: + m.tune('category-steady-n-tasks', config.autolabel_window) + + # Specify number of workers to wait for before sending the first task + if config.wait_for_workers: + m.tune("wait-for-workers", config.wait_for_workers) + + # Enable peer transfer feature between workers if specified + if config.enable_peer_transfers: + m.enable_peer_transfers() + + +def _prepare_environment_serverless(manager_config, env_cache_dir, poncho_create_script): + # Return path to a packaged poncho environment + poncho_env_path = '' + if not manager_config.shared_fs: + if manager_config.env_pack is None: + raise Exception('TaskVine manager needs env_pack to be specified when running tasks in serverless mode and with no shared_fs') + + poncho_env_path = manager_config.env_pack + + # If a conda environment name or path is given, then use the hash of the headers of + # all contained packages as the name of the to-be-packaged poncho tarball, + # and package it if it's not cached. + if not poncho_env_path.endswith('tar.gz'): + if os.path.isabs(poncho_env_path): + conda_env_signature = hashlib.md5(subprocess.check_output(['conda', 'list', '-p', poncho_env_path, '--json'])).hexdigest() + logger.debug(f'Signature of conda environment at {poncho_env_path}: {conda_env_signature}') + else: + conda_env_signature = hashlib.md5(subprocess.check_output(['conda', 'list', '-n', poncho_env_path, '--json'])).hexdigest() + logger.debug(f'Signature of conda environment named {poncho_env_path}: {conda_env_signature}') + + # If env is cached then use it, + # else create a new env tarball + poncho_env_path = os.path.join(env_cache_dir, '.'.join([conda_env_signature, 'tar.gz'])) + if not os.path.isfile(poncho_env_path): + logger.debug(f'No cached poncho environment. Creating poncho environment for library task at {poncho_env_path}') + try: + subprocess.run([poncho_create_script, manager_config.env_pack, poncho_env_path], stdout=subprocess.DEVNULL, check=True) + except Exception: + logger.error('Cannot create a poncho environment. Removing it.') + if os.path.isfile(poncho_env_path): + os.remove(poncho_env_path) + raise + else: + logger.debug(f'Found cached poncho environment at {poncho_env_path}. Reusing it.') + else: + logger.debug(f'Use the given poncho environment at {manager_config.env_pack} to setup library task.') + return poncho_env_path + + +def _prepare_environment_regular(m, manager_config, t, task, poncho_env_to_file, poncho_create_script): + # Add environment file to the task if possible + # Prioritize local poncho environment over global poncho environment + # (local: use app_pack, global: use env_pack) + poncho_env_file = None + + # check if env_pack is specified + if manager_config.env_pack is not None: + + # check if the environment file is not already created + if manager_config.env_pack not in poncho_env_to_file: + + # if the environment is already packaged as a tarball, then add the file + # otherwise it is an environment name or path, so create a poncho tarball then add it + if not manager_config.env_pack.endswith('.tar.gz'): + env_tarball = str(uuid.uuid4()) + '.tar.gz' + logger.debug(f'Creating a poncho environment at {env_tarball} from conda environment {manager_config.env_pack}') + subprocess.run([poncho_create_script, manager_config.env_pack, env_tarball], stdout=subprocess.DEVNULL, check=True) + poncho_env_file = m.declare_poncho(env_tarball, cache=True, peer_transfer=True) + poncho_env_to_file[manager_config.env_pack] = poncho_env_file + else: + poncho_env_file = poncho_env_to_file[manager_config.env_pack] + logger.debug(f'Found cached poncho environment for {manager_config.env_pack}. Reusing it.') + + # check if app_pack is used, override if possible + if task.env_pkg is not None: + if task.env_pkg not in poncho_env_to_file: + poncho_env_file = m.declare_poncho(task.env_pkg, cache=True, peer_transfer=True) + poncho_env_to_file[task.env_pkg] = poncho_env_file + else: + poncho_env_file = poncho_env_to_file[task.env_pkg] + + # Add environment to the task + if poncho_env_file is not None: + t.add_environment(poncho_env_file) + + +@wrap_with_logs +def _taskvine_submit_wait(ready_task_queue=None, + finished_task_queue=None, + should_stop=None, + manager_config=None + ): + """Process to handle Parsl app submissions to the TaskVine objects. + Takes in Parsl functions submitted using submit(), and creates a + TaskVine task with the appropriate specifications, which is then + submitted to TaskVine. After tasks are completed, processes the + exit status and exit code of the task, and sends results to the + TaskVine collector thread. + To avoid python's global interpreter lock with taskvine's wait, this + function should be launched as a process, not as a lightweight thread. This + means that any communication should be done using the multiprocessing + module capabilities, rather than shared memory. + """ + logger.debug("Starting TaskVine Submit/Wait Process") + setproctitle("parsl: TaskVine submit/wait") + + # Enable debugging flags and create logging file + if manager_config.vine_log_dir is not None: + logger.debug("Setting debugging flags and creating logging file at {}".format(manager_config.vine_log_dir)) + + # Create TaskVine queue object + logger.debug("Creating TaskVine Object") + try: + logger.debug("Listening on port {}".format(manager_config.port)) + m = Manager(port=manager_config.port, + name=manager_config.project_name, + run_info_path=manager_config.vine_log_dir) + except Exception as e: + logger.error("Unable to create TaskVine object: {}".format(e)) + raise e + + # Specify TaskVine manager attributes + _set_manager_attributes(m, manager_config) + + # Get parent pid, useful to shutdown this process when its parent, the taskvine + # executor process, exits. + orig_ppid = os.getppid() + + result_file_of_task_id = {} # Mapping executor task id -> result file. + + poncho_env_to_file = {} # Mapping poncho_env id to File object in TaskVine + + # Mapping of parsl local file name to TaskVine File object + # dict[str] -> vine File object + parsl_file_name_to_vine_file = {} + + # Mapping of tasks from vine id to parsl id + # Dict[str] -> str + vine_id_to_executor_task_id = {} + + # Find poncho scripts to create and activate an environment tarball + poncho_create_script = shutil.which("poncho_package_create") + + # Declare helper script as cache-able and peer-transferable + exec_parsl_function_file = m.declare_file(exec_parsl_function.__file__, cache=True, peer_transfer=True) + + # Flag to make sure library for serverless tasks is declared and installed only once. + lib_installed = False + + # Create cache dir for environment files + env_cache_dir = os.path.join(manager_config.vine_log_dir, 'vine-cache', 'vine-poncho-env-cache') + os.makedirs(env_cache_dir, exist_ok=True) + + logger.debug("Entering main loop of TaskVine manager") + + while not should_stop.value: + # Monitor the task queue + ppid = os.getppid() + if ppid != orig_ppid: + logger.debug("new Process") + break + + # Submit tasks + while ready_task_queue.qsize() > 0 and not should_stop.value: + # Obtain task from ready_task_queue + try: + task = ready_task_queue.get(timeout=1) + logger.debug("Removing executor task from queue") + except queue.Empty: + logger.debug("Queue is empty") + continue + if task.exec_mode == 'regular': + # Create command string + launch_cmd = "python3 exec_parsl_function.py {mapping} {function} {argument} {result}" + if manager_config.init_command != '': + launch_cmd = "{init_cmd} " + launch_cmd + command_str = launch_cmd.format(init_cmd=manager_config.init_command, + mapping=os.path.basename(task.map_file), + function=os.path.basename(task.function_file), + argument=os.path.basename(task.argument_file), + result=os.path.basename(task.result_file)) + logger.debug("Sending executor task {} (mode: regular) with command: {}".format(task.executor_id, command_str)) + try: + t = Task(command_str) + except Exception as e: + logger.error("Unable to create executor task (mode:regular): {}".format(e)) + finished_task_queue.put_nowait(VineTaskToParsl(executor_id=task.executor_id, + result_received=False, + result=None, + reason="task could not be created by taskvine", + status=-1)) + continue + elif task.exec_mode == 'serverless': + if not lib_installed: + # Declare and install common library for serverless tasks. + # Library requires an environment setup properly, which is + # different from setup of regular tasks. + # If shared_fs is True, then no environment preparation is done. + # Only the core serverless code is created. + poncho_env_path = _prepare_environment_serverless(manager_config, env_cache_dir, poncho_create_script) + + # Don't automatically add environment so manager can declare and cache the vine file associated with the environment file + add_env = False + serverless_lib = m.create_library_from_functions('common-parsl-taskvine-lib', + run_parsl_function, + poncho_env=poncho_env_path, + init_command=manager_config.init_command, + add_env=add_env) + if poncho_env_path: + serverless_lib_env_file = m.declare_poncho(poncho_env_path, cache=True, peer_transfer=True) + serverless_lib.add_environment(serverless_lib_env_file) + poncho_env_to_file[manager_config.env_pack] = serverless_lib_env_file + logger.debug(f'Created library task using poncho environment at {poncho_env_path}.') + else: + logger.debug('Created minimal library task with no environment.') + + m.install_library(serverless_lib) + lib_installed = True + try: + # run_parsl_function only needs remote names of map_file, function_file, argument_file, + # and result_file, which are simply named map, function, argument, result. + # These names are given when these files are declared below. + t = FunctionCall('common-parsl-taskvine-lib', run_parsl_function.__name__, 'map', 'function', 'argument', 'result') + except Exception as e: + logger.error("Unable to create executor task (mode:serverless): {}".format(e)) + finished_task_queue.put_nowait(VineTaskToParsl(executor_id=task.executor_id, + result_received=False, + result=None, + reason="task could not be created by taskvine", + status=-1)) + else: + raise Exception(f'Unrecognized task mode {task.exec_mode}. Exiting...') + + # prepare environment for regular tasks if not using shared_fs + if task.exec_mode == 'regular' and not manager_config.shared_fs: + _prepare_environment_regular(m, manager_config, t, task, poncho_env_to_file, poncho_create_script) + + t.set_category(task.category) + + # Set autolabel mode + if manager_config.autolabel: + if manager_config.autolabel_algorithm == 'max-xput': + m.set_category_mode(task.category, VINE_ALLOCATION_MODE_MAX_THROUGHPUT) + elif manager_config.autolabel_algorithm == 'bucketing': + m.set_category_mode(task.category, VINE_ALLOCATION_MODE_EXHAUSTIVE_BUCKETING) + elif manager_config.autolabel_algorithm == 'max': + m.set_category_mode(task.category, VINE_ALLOCATION_MODE_MAX) + else: + logger.warning(f'Unrecognized autolabeling algorithm named {manager_config.autolabel_algorithm} for taskvine manager.') + raise Exception(f'Unrecognized autolabeling algorithm named {manager_config.autolabel_algorithm} for taskvine manager.') + + if task.cores is not None: + t.set_cores(task.cores) + if task.memory is not None: + t.set_memory(task.memory) + if task.disk is not None: + t.set_disk(task.disk) + if task.gpus is not None: + t.set_gpus(task.gpus) + if task.priority is not None: + t.set_priority(task.priority) + if task.running_time_min is not None: + t.set_time_min(task.running_time_min) + + if manager_config.max_retries is not None: + logger.debug(f"Specifying max_retries {manager_config.max_retries}") + t.set_retries(manager_config.max_retries) + + # Specify environment variables for the task + if manager_config.env_vars is not None: + for var in manager_config.env_vars: + t.set_env_var(str(var), str(manager_config.env_vars[var])) + + if task.exec_mode == 'regular': + # Add helper files that execute parsl functions on remote nodes + # only needed to add as file for tasks with 'regular' mode + t.add_input(exec_parsl_function_file, "exec_parsl_function.py") + + # Declare and add task-specific function, data, and result files to task + task_function_file = m.declare_file(task.function_file, cache=False, peer_transfer=False) + t.add_input(task_function_file, "function") + + task_argument_file = m.declare_file(task.argument_file, cache=False, peer_transfer=False) + t.add_input(task_argument_file, "argument") + + task_map_file = m.declare_file(task.map_file, cache=False, peer_transfer=False) + t.add_input(task_map_file, "map") + + task_result_file = m.declare_file(task.result_file, cache=False, peer_transfer=False) + t.add_output(task_result_file, "result") + + result_file_of_task_id[str(task.executor_id)] = task.result_file + + logger.debug("Executor task id: {}".format(task.executor_id)) + + # Specify input/output files that need to be staged. + # Absolute paths are assumed to be in shared filesystem, and thus + # not staged by taskvine. + # Files that share the same local path are assumed to be the same + # and thus use the same Vine File object if detected. + if not manager_config.shared_fs: + for spec in task.input_files: + if spec.stage: + if spec.parsl_name in parsl_file_name_to_vine_file: + task_in_file = parsl_file_name_to_vine_file[spec.parsl_name] + else: + task_in_file = m.declare_file(spec.parsl_name, cache=spec.cache, peer_transfer=True) + parsl_file_name_to_vine_file[spec.parsl_name] = task_in_file + t.add_input(task_in_file, spec.parsl_name) + + for spec in task.output_files: + if spec.stage: + if spec.parsl_name in parsl_file_name_to_vine_file: + task_out_file = parsl_file_name_to_vine_file[spec.parsl_name] + else: + task_out_file = m.declare_file(spec.parsl_name, cache=spec.cache, peer_transfer=True) + t.add_output(task_out_file, spec.parsl_name) + + # Submit the task to the TaskVine object + logger.debug("Submitting executor task {}, {} to TaskVine".format(task.executor_id, t)) + try: + vine_id = m.submit(t) + logger.debug("Submitted executor task {} to TaskVine".format(task.executor_id)) + vine_id_to_executor_task_id[str(vine_id)] = str(task.executor_id), task.exec_mode + except Exception as e: + logger.error("Unable to submit task to taskvine: {}".format(e)) + finished_task_queue.put_nowait(VineTaskToParsl(executor_id=task.executor_id, + result_received=False, + result=None, + reason="task could not be submited to taskvine", + status=-1)) + continue + + logger.debug("Executor task {} submitted as TaskVine task with id {}".format(task.executor_id, vine_id)) + + # If the queue is not empty wait on the TaskVine queue for a task + task_found = True + if not m.empty(): + while task_found and not should_stop.value: + # Obtain the task from the queue + t = m.wait(1) + if t is None: + task_found = False + continue + logger.debug('Found a task') + executor_task_id = vine_id_to_executor_task_id[str(t.id)][0] + vine_id_to_executor_task_id.pop(str(t.id)) + + # When a task is found + result_file = result_file_of_task_id.pop(executor_task_id) + + logger.debug(f"completed executor task info: {executor_task_id}, {t.category}, {t.command}, {t.std_output}") + + # A tasks completes 'succesfully' if it has result file, + # and it can be loaded. This may mean that the 'success' is + # an exception. + logger.debug("Looking for result in {}".format(result_file)) + try: + with open(result_file, "rb") as f_in: + result = pickle.load(f_in) + logger.debug("Found result in {}".format(result_file)) + finished_task_queue.put_nowait(VineTaskToParsl(executor_id=executor_task_id, + result_received=True, + result=result, + reason=None, + status=t.exit_code)) + # If a result file could not be generated, explain the + # failure according to taskvine error codes. We generate + # an exception and wrap it with RemoteExceptionWrapper, to + # match the positive case. + except Exception as e: + reason = _explain_taskvine_result(t) + logger.debug("Did not find result in {}".format(result_file)) + logger.debug("Wrapper Script status: {}\nTaskVine Status: {}" + .format(t.exit_code, t.result)) + logger.debug("Task with executor id {} / vine id {} failed because:\n{}" + .format(executor_task_id, t.id, reason)) + finished_task_queue.put_nowait(VineTaskToParsl(executor_id=executor_task_id, + result_received=False, + result=e, + reason=reason, + status=t.exit_code)) + + logger.debug("Exiting TaskVine Monitoring Process") + return 0 + + +def _explain_taskvine_result(vine_task): + """Returns a string with the reason why a task failed according to taskvine.""" + + vine_result = vine_task.result + reason = "taskvine result: " + if vine_result == cvine.VINE_RESULT_SUCCESS: + reason += "succesful execution with exit code {}".format(vine_task.return_status) + elif vine_result == cvine.VINE_RESULT_OUTPUT_MISSING: + reason += "The result file was not transfered from the worker.\n" + reason += "This usually means that there is a problem with the python setup,\n" + reason += "or the wrapper that executes the function." + reason += "\nTrace:\n" + str(vine_task.output) + elif vine_result == cvine.VINE_RESULT_INPUT_MISSING: + reason += "missing input file" + elif vine_result == cvine.VINE_RESULT_STDOUT_MISSING: + reason += "stdout has been truncated" + elif vine_result == cvine.VINE_RESULT_SIGNAL: + reason += "task terminated with a signal" + elif vine_result == cvine.VINE_RESULT_RESOURCE_EXHAUSTION: + reason += "task used more resources than requested" + elif vine_result == cvine.VINE_RESULT_MAX_END_TIME: + reason += "task ran past the specified end time" + elif vine_result == cvine.VINE_RESULT_UNKNOWN: + reason += "result could not be classified" + elif vine_result == cvine.VINE_RESULT_FORSAKEN: + reason += "task failed, but not a task error" + elif vine_result == cvine.VINE_RESULT_MAX_RETRIES: + reason += "unable to complete after specified number of retries" + elif vine_result == cvine.VINE_RESULT_MAX_WALL_TIME: + reason += "task ran for more than the specified time" + elif vine_result == cvine.VINE_RESULT_RMONITOR_ERROR: + reason += "task failed because the monitor did not produce an output" + elif vine_result == cvine.VINE_RESULT_OUTPUT_TRANSFER_ERROR: + reason += "task failed because output transfer fails" + elif vine_result == cvine.VINE_RESULT_FIXED_LOCATION_MISSING: + reason += "task failed because no worker could satisfy the fixed \n" + reason += "location input file requirements" + else: + reason += "unable to process TaskVine system failure" + return reason diff --git a/parsl/executors/taskvine/manager_config.py b/parsl/executors/taskvine/manager_config.py index e59d31bd88..ee7c65a873 100644 --- a/parsl/executors/taskvine/manager_config.py +++ b/parsl/executors/taskvine/manager_config.py @@ -47,6 +47,7 @@ class TaskVineManagerConfig: Used to encapsulate package dependencies of tasks to execute them remotely without needing a shared filesystem. Recommended way to manage tasks' dependency requirements. + All tasks will be executed in the encapsulated environment. If an absolute path to a conda environment or a conda environment name is given, TaskVine will package the conda environment in a tarball and send it along with tasks to be diff --git a/parsl/executors/taskvine/utils.py b/parsl/executors/taskvine/utils.py index 6c521a5e7a..9f0d9f7c05 100644 --- a/parsl/executors/taskvine/utils.py +++ b/parsl/executors/taskvine/utils.py @@ -12,6 +12,7 @@ def __init__(self, output_files: list, # list of output files to this function map_file: Optional[str], # pickled file containing mapping of local to remote names of files function_file: Optional[str], # pickled file containing the function information + argument_file: Optional[str], # pickled file containing the arguments to the function call result_file: Optional[str], # path to the pickled result object of the function execution cores: Optional[float], # number of cores to allocate memory: Optional[int], # amount of memory in MBs to allocate @@ -26,6 +27,7 @@ def __init__(self, self.category = category self.map_file = map_file self.function_file = function_file + self.argument_file = argument_file self.result_file = result_file self.input_files = input_files self.output_files = output_files @@ -73,3 +75,11 @@ def __init__(self, self.parsl_name = parsl_name self.stage = stage self.cache = cache + + +def run_parsl_function(map_file, function_file, argument_file, result_file): + """ + Wrapper function to deploy with FunctionCall as serverless tasks. + """ + from parsl.executors.taskvine.exec_parsl_function import run + run(map_file, function_file, argument_file, result_file) diff --git a/parsl/tests/configs/taskvine_ex.py b/parsl/tests/configs/taskvine_ex.py index 6b4d03b507..a573ab5bae 100644 --- a/parsl/tests/configs/taskvine_ex.py +++ b/parsl/tests/configs/taskvine_ex.py @@ -9,5 +9,5 @@ def fresh_config(): return Config(executors=[TaskVineExecutor(manager_config=TaskVineManagerConfig(port=9000), - use_factory=True, + worker_launch_method='factory', storage_access=[FTPInTaskStaging(), HTTPInTaskStaging(), NoOpFileStaging()])]) diff --git a/parsl/tests/scaling_tests/vineex_local.py b/parsl/tests/scaling_tests/vineex_local.py index d5f418163a..e3fc9bba2f 100644 --- a/parsl/tests/scaling_tests/vineex_local.py +++ b/parsl/tests/scaling_tests/vineex_local.py @@ -5,7 +5,7 @@ config = Config( executors=[TaskVineExecutor(label='VineExec', - use_factory=True, + worker_launch_method='factory', manager_config=TaskVineManagerConfig(port=50055), )] ) From 1682e217c7fcef00c123f5bafb6206fe384d68d3 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 7 Aug 2023 17:53:56 +0200 Subject: [PATCH 04/43] Split and remove JobErrorHandler (#2853) * refactor JobErrorHandler class out of existence * step 2 of job error handler refactor make simple_error_handler top level moves simple_error_handler into its own module so it can be imported without causing import loops that happen if it lives alongside JobErrorHandler * jeh tidy 3 * jeh tidy 4 * JEH tidy 5 * jeh 6 * jeh tidy 7 * jeh 8 --- parsl/executors/base.py | 4 +-- parsl/executors/status_handling.py | 7 ++-- parsl/jobs/job_error_handler.py | 54 ------------------------------ parsl/jobs/job_status_poller.py | 10 ++++-- parsl/jobs/simple_error_handler.py | 42 +++++++++++++++++++++++ 5 files changed, 53 insertions(+), 64 deletions(-) delete mode 100644 parsl/jobs/job_error_handler.py create mode 100644 parsl/jobs/simple_error_handler.py diff --git a/parsl/executors/base.py b/parsl/executors/base.py index ff7aa6a7ee..95a9c7117a 100644 --- a/parsl/executors/base.py +++ b/parsl/executors/base.py @@ -167,14 +167,12 @@ def error_management_enabled(self) -> bool: pass @abstractmethod - def handle_errors(self, error_handler: "parsl.jobs.job_error_handler.JobErrorHandler", - status: Dict[str, JobStatus]) -> None: + def handle_errors(self, status: Dict[str, JobStatus]) -> None: """This method is called by the error management infrastructure after a status poll. The executor implementing this method is then responsible for detecting abnormal conditions based on the status of submitted jobs. If the executor does not implement any special error handling, this method should return False, in which case a generic error handling scheme will be used. - :param error_handler: a reference to the generic error handler calling this method :param status: status of all jobs launched by this executor """ pass diff --git a/parsl/executors/status_handling.py b/parsl/executors/status_handling.py index cfdd6703b4..b95a8bd610 100644 --- a/parsl/executors/status_handling.py +++ b/parsl/executors/status_handling.py @@ -12,6 +12,7 @@ from parsl.providers.base import ExecutionProvider from parsl.utils import AtomicIDCounter +import parsl.jobs.simple_error_handler as error_handler logger = logging.getLogger(__name__) @@ -136,8 +137,7 @@ def executor_exception(self): def error_management_enabled(self): return self.block_error_handler - def handle_errors(self, error_handler: "parsl.jobs.job_error_handler.JobErrorHandler", - status: Dict[str, JobStatus]) -> None: + def handle_errors(self, status: Dict[str, JobStatus]) -> None: if not self.block_error_handler: return init_blocks = 3 @@ -236,8 +236,7 @@ def set_bad_state_and_fail_all(self, exception: Exception): def status(self): return {} - def handle_errors(self, error_handler: "parsl.jobs.job_error_handler.JobErrorHandler", - status: Dict[str, JobStatus]) -> None: + def handle_errors(self, status: Dict[str, JobStatus]) -> None: pass @property diff --git a/parsl/jobs/job_error_handler.py b/parsl/jobs/job_error_handler.py deleted file mode 100644 index 8740acb2e8..0000000000 --- a/parsl/jobs/job_error_handler.py +++ /dev/null @@ -1,54 +0,0 @@ -from __future__ import annotations - -from typing import List, Dict - -import parsl.jobs.job_status_poller as jsp - -from parsl.executors.base import ParslExecutor -from parsl.jobs.states import JobStatus, JobState - - -class JobErrorHandler: - def run(self, status: List[jsp.PollItem]): - for es in status: - self._check_irrecoverable_executor(es) - - def _check_irrecoverable_executor(self, es: jsp.PollItem): - if not es.executor.error_management_enabled: - return - es.executor.handle_errors(self, es.status) - - def simple_error_handler(self, executor: ParslExecutor, status: Dict[str, JobStatus], threshold: int): - (total_jobs, failed_jobs) = self.count_jobs(status) - if total_jobs >= threshold and failed_jobs == total_jobs: - executor.set_bad_state_and_fail_all(self.get_error(status)) - - def count_jobs(self, status: Dict[str, JobStatus]): - total = 0 - failed = 0 - for js in status.values(): - total += 1 - if js.state == JobState.FAILED: - failed += 1 - return total, failed - - def get_error(self, status: Dict[str, JobStatus]) -> Exception: - """Concatenate all errors.""" - err = "" - count = 1 - for js in status.values(): - if js.message is not None: - err = err + "{}. {}\n".format(count, js.message) - count += 1 - stdout = js.stdout_summary - if stdout: - err = err + "\tSTDOUT: {}\n".format(stdout) - stderr = js.stderr_summary - if stderr: - err = err + "\tSTDERR: {}\n".format(stderr) - - if len(err) == 0: - err = "[No error message received]" - # wrapping things in an exception here doesn't really help in providing more information - # than the string itself - return Exception(err) diff --git a/parsl/jobs/job_status_poller.py b/parsl/jobs/job_status_poller.py index 2f7e761976..d5d65a9419 100644 --- a/parsl/jobs/job_status_poller.py +++ b/parsl/jobs/job_status_poller.py @@ -6,7 +6,6 @@ from typing import List # noqa F401 (used in type annotation) from parsl.executors.base import ParslExecutor -from parsl.jobs.job_error_handler import JobErrorHandler from parsl.jobs.states import JobStatus, JobState from parsl.jobs.strategy import Strategy from parsl.monitoring.message_type import MessageType @@ -107,14 +106,19 @@ def __init__(self, dfk: "parsl.dataflow.dflow.DataFlowKernel"): self.dfk = dfk self._strategy = Strategy(strategy=dfk.config.strategy, max_idletime=dfk.config.max_idletime) - self._error_handler = JobErrorHandler() super().__init__(self.poll, interval=5, name="JobStatusPoller") def poll(self): self._update_state() - self._error_handler.run(self._poll_items) + self._run_error_handlers(self._poll_items) self._strategy.strategize(self._poll_items) + def _run_error_handlers(self, status: List[PollItem]): + for es in status: + if not es.executor.error_management_enabled: + return + es.executor.handle_errors(es.status) + def _update_state(self) -> None: now = time.time() for item in self._poll_items: diff --git a/parsl/jobs/simple_error_handler.py b/parsl/jobs/simple_error_handler.py new file mode 100644 index 0000000000..d1baca15d9 --- /dev/null +++ b/parsl/jobs/simple_error_handler.py @@ -0,0 +1,42 @@ +from typing import Dict + +from parsl.executors.base import ParslExecutor +from parsl.jobs.states import JobStatus, JobState + + +def simple_error_handler(executor: ParslExecutor, status: Dict[str, JobStatus], threshold: int): + (total_jobs, failed_jobs) = _count_jobs(status) + if total_jobs >= threshold and failed_jobs == total_jobs: + executor.set_bad_state_and_fail_all(_get_error(status)) + + +def _count_jobs(status: Dict[str, JobStatus]): + total = 0 + failed = 0 + for js in status.values(): + total += 1 + if js.state == JobState.FAILED: + failed += 1 + return total, failed + + +def _get_error(status: Dict[str, JobStatus]) -> Exception: + """Concatenate all errors.""" + err = "" + count = 1 + for js in status.values(): + if js.message is not None: + err = err + "{}. {}\n".format(count, js.message) + count += 1 + stdout = js.stdout_summary + if stdout: + err = err + "\tSTDOUT: {}\n".format(stdout) + stderr = js.stderr_summary + if stderr: + err = err + "\tSTDERR: {}\n".format(stderr) + + if len(err) == 0: + err = "[No error message received]" + # wrapping things in an exception here doesn't really help in providing more information + # than the string itself + return Exception(err) From af4c5ac6a25c34eb154f3850213d67f07e89e400 Mon Sep 17 00:00:00 2001 From: Yadu Nand Babuji Date: Mon, 7 Aug 2023 12:31:36 -0500 Subject: [PATCH 05/43] Removing address_from_hostname instances from example configs (#2854) Removes instances where address_from_hostname is used Updating configs to use address_by_interface where internal network interfaces are known --- docs/userguide/configuring.rst | 8 ++++---- parsl/configs/bridges.py | 2 ++ parsl/configs/illinoiscluster.py | 2 -- parsl/configs/midway.py | 2 ++ parsl/configs/polaris.py | 4 ++-- parsl/configs/stampede2.py | 2 ++ parsl/configs/theta.py | 3 ++- 7 files changed, 14 insertions(+), 9 deletions(-) diff --git a/docs/userguide/configuring.rst b/docs/userguide/configuring.rst index 9ad9d935c3..91685e275c 100644 --- a/docs/userguide/configuring.rst +++ b/docs/userguide/configuring.rst @@ -17,8 +17,8 @@ supercomputer at TACC. This config uses the `parsl.executors.HighThroughputExecutor` to submit tasks from a login node (`parsl.channels.LocalChannel`). It requests an allocation of 128 nodes, deploying 1 worker for each of the 56 cores per node, from the normal partition. -The config uses the `address_by_hostname()` helper function to determine -the login node's IP address. +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 @@ -27,13 +27,13 @@ the login node's IP address. from parsl.providers import SlurmProvider from parsl.executors import HighThroughputExecutor from parsl.launchers import SrunLauncher - from parsl.addresses import address_by_hostname + from parsl.addresses import address_by_interface config = Config( executors=[ HighThroughputExecutor( label="frontera_htex", - address=address_by_hostname(), + address=address_by_interface('ib0'), max_workers=56, provider=SlurmProvider( channel=LocalChannel(), diff --git a/parsl/configs/bridges.py b/parsl/configs/bridges.py index 9791138502..6da44c2c0a 100644 --- a/parsl/configs/bridges.py +++ b/parsl/configs/bridges.py @@ -2,6 +2,7 @@ from parsl.providers import SlurmProvider from parsl.launchers import SrunLauncher from parsl.executors import HighThroughputExecutor +from parsl.addresses import address_by_interface """ This config assumes that it is used to launch parsl tasks from the login nodes of Bridges at PSC. Each job submitted to the scheduler will request 2 nodes for 10 minutes. @@ -11,6 +12,7 @@ executors=[ HighThroughputExecutor( label='Bridges_HTEX_multinode', + address=address_by_interface('ens3f0'), max_workers=1, provider=SlurmProvider( 'YOUR_PARTITION_NAME', # Specify Partition / QOS, for eg. RM-small diff --git a/parsl/configs/illinoiscluster.py b/parsl/configs/illinoiscluster.py index 55e058f7df..3e417d62a7 100644 --- a/parsl/configs/illinoiscluster.py +++ b/parsl/configs/illinoiscluster.py @@ -2,7 +2,6 @@ from parsl.providers import SlurmProvider from parsl.executors import HighThroughputExecutor from parsl.launchers import SrunLauncher -from parsl.addresses import address_by_hostname """ This config assumes that it is used to launch parsl tasks from the login nodes of the Campus Cluster at UIUC. Each job submitted to the scheduler will request 2 nodes for 10 minutes. @@ -12,7 +11,6 @@ HighThroughputExecutor( label="CC_htex", worker_debug=False, - address=address_by_hostname(), cores_per_worker=16.0, # each worker uses a full node provider=SlurmProvider( partition='secondary-fdr', # partition diff --git a/parsl/configs/midway.py b/parsl/configs/midway.py index 015b58698a..bac101df83 100644 --- a/parsl/configs/midway.py +++ b/parsl/configs/midway.py @@ -2,11 +2,13 @@ from parsl.providers import SlurmProvider from parsl.launchers import SrunLauncher from parsl.executors import HighThroughputExecutor +from parsl.addresses import address_by_interface config = Config( executors=[ HighThroughputExecutor( label='Midway_HTEX_multinode', + address=address_by_interface('bond0'), worker_debug=False, max_workers=2, provider=SlurmProvider( diff --git a/parsl/configs/polaris.py b/parsl/configs/polaris.py index a5f2ef3431..29a89ddcd7 100644 --- a/parsl/configs/polaris.py +++ b/parsl/configs/polaris.py @@ -1,4 +1,4 @@ -from parsl.addresses import address_by_hostname +from parsl.addresses import address_by_interface from parsl.executors import HighThroughputExecutor from parsl.launchers import MpiExecLauncher from parsl.providers import PBSProProvider @@ -15,7 +15,7 @@ executors=[ HighThroughputExecutor( available_accelerators=4, # Ensures one worker per accelerator - address=address_by_hostname(), + address=address_by_interface('bond0'), cpu_affinity="alternating", # Prevents thread contention prefetch_capacity=0, # Increase if you have many more tasks than workers start_method="spawn", # Needed to avoid interactions between MPI and os.fork diff --git a/parsl/configs/stampede2.py b/parsl/configs/stampede2.py index 1e231eb994..2f6ec961ce 100644 --- a/parsl/configs/stampede2.py +++ b/parsl/configs/stampede2.py @@ -3,12 +3,14 @@ from parsl.launchers import SrunLauncher from parsl.executors import HighThroughputExecutor from parsl.data_provider.globus import GlobusStaging +from parsl.addresses import address_by_interface config = Config( executors=[ HighThroughputExecutor( label='Stampede2_HTEX', + address=address_by_interface('em3'), max_workers=2, provider=SlurmProvider( nodes_per_block=2, diff --git a/parsl/configs/theta.py b/parsl/configs/theta.py index 94d0584c22..9363165512 100644 --- a/parsl/configs/theta.py +++ b/parsl/configs/theta.py @@ -2,12 +2,13 @@ from parsl.providers import CobaltProvider from parsl.launchers import AprunLauncher from parsl.executors import HighThroughputExecutor - +from parsl.addresses import address_by_interface config = Config( executors=[ HighThroughputExecutor( label='theta_local_htex_multinode', + address=address_by_interface('vlan2360'), max_workers=4, cpu_affinity='block', # Ensures that workers use cores on the same tile provider=CobaltProvider( From 4b50ab03e330f78ee5b17ab723b2fcb0bdea640f Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 7 Aug 2023 22:25:36 +0200 Subject: [PATCH 06/43] Remove NoStatusHandlingExecutor (#2855) Executors which want to use status handling should subclass the BlockProviderExecutor, formerly (PR #2071) known as the StatusHanldingExecutor. Code which performs status handling should only operate on BlockProviderExecutor instances - the status handling code shouldn't do anything to other ParslExecutor instances. --- parsl/dataflow/dflow.py | 17 ++++--- parsl/executors/base.py | 81 ------------------------------ parsl/executors/flux/executor.py | 6 +-- parsl/executors/status_handling.py | 81 ++++++++++++++++++------------ parsl/executors/threads.py | 6 +-- parsl/jobs/job_status_poller.py | 8 +-- parsl/jobs/simple_error_handler.py | 6 ++- 7 files changed, 72 insertions(+), 133 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index 01725f8061..936a32acf4 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -1128,7 +1128,8 @@ def add_executors(self, executors): msg = executor.create_monitoring_info(new_status) logger.debug("Sending monitoring message {} to hub from DFK".format(msg)) self.monitoring.send(MessageType.BLOCK_INFO, msg) - self.job_status_poller.add_executors(executors) + block_executors = [e for e in executors if isinstance(e, BlockProviderExecutor)] + self.job_status_poller.add_executors(block_executors) def atexit_cleanup(self) -> None: if not self.cleanup_called: @@ -1197,8 +1198,8 @@ def cleanup(self) -> None: logger.info("Scaling in and shutting down executors") for executor in self.executors.values(): - if not executor.bad_state_is_set: - if isinstance(executor, BlockProviderExecutor): + if isinstance(executor, BlockProviderExecutor): + if not executor.bad_state_is_set: logger.info(f"Scaling in executor {executor.label}") if executor.provider: job_ids = executor.provider.resources.keys() @@ -1210,11 +1211,11 @@ def cleanup(self) -> None: msg = executor.create_monitoring_info(new_status) logger.debug("Sending message {} to hub from DFK".format(msg)) self.monitoring.send(MessageType.BLOCK_INFO, msg) - logger.info(f"Shutting down executor {executor.label}") - executor.shutdown() - logger.info(f"Shut down executor {executor.label}") - else: # and bad_state_is_set - logger.warning(f"Not shutting down executor {executor.label} because it is in bad state") + else: # and bad_state_is_set + logger.warning(f"Not shutting down executor {executor.label} because it is in bad state") + logger.info(f"Shutting down executor {executor.label}") + executor.shutdown() + logger.info(f"Shut down executor {executor.label}") logger.info("Terminated executors") self.time_completed = datetime.datetime.now() diff --git a/parsl/executors/base.py b/parsl/executors/base.py index 95a9c7117a..579fd904af 100644 --- a/parsl/executors/base.py +++ b/parsl/executors/base.py @@ -120,87 +120,6 @@ def monitor_resources(self) -> bool: """ return True - @abstractmethod - def status(self) -> Dict[str, JobStatus]: - """Return the status of all jobs/blocks currently known to this executor. - - :return: a dictionary mapping block ids (in string) to job status - """ - pass - - @property - @abstractmethod - def status_polling_interval(self) -> int: - """Returns the interval, in seconds, at which the status method should be called. The - assumption here is that, once initialized, an executor's polling interval is fixed. - In practice, at least given the current situation, the executor uses a single task provider - and this method is a delegate to the corresponding method in the provider. - - :return: the number of seconds to wait between calls to status() or zero if no polling - should be done - """ - pass - - @property - @abstractmethod - def error_management_enabled(self) -> bool: - """Indicates whether worker error management is supported by this executor. Worker error - management is done externally to the executor. However, the executor must implement - certain status handling methods that allow this to function. These methods are: - - :method:handle_errors - :method:set_bad_state_and_fail_all - - The basic idea of worker error management is that an external entity maintains a view of - the state of the workers by calling :method:status() which is then processed to detect - abnormal conditions. This can be done externally, as well as internally, through - :method:handle_errors. If an entity external to the executor detects an abnormal condition, - it can notify the executor using :method:set_bad_state_and_fail_all(exception). - - Some of the scaffolding needed for implementing error management inside executors, - including implementations for the status handling methods above, is available in - :class:parsl.executors.status_handling.BlockProviderExecutor, which interested executors - should inherit from. Noop versions of methods that are related to status handling and - running parsl tasks through workers are implemented by - :class:parsl.executors.status_handling.NoStatusHandlingExecutor. - """ - pass - - @abstractmethod - def handle_errors(self, status: Dict[str, JobStatus]) -> None: - """This method is called by the error management infrastructure after a status poll. The - executor implementing this method is then responsible for detecting abnormal conditions - based on the status of submitted jobs. If the executor does not implement any special - error handling, this method should return False, in which case a generic error handling - scheme will be used. - :param status: status of all jobs launched by this executor - """ - pass - - @abstractmethod - def set_bad_state_and_fail_all(self, exception: Exception) -> None: - """Allows external error handlers to mark this executor as irrecoverably bad and cause - all tasks submitted to it now and in the future to fail. The executor is responsible - for checking :method:bad_state_is_set() in the :method:submit() method and raising the - appropriate exception, which is available through :method:executor_exception(). - """ - pass - - @property - @abstractmethod - def bad_state_is_set(self) -> bool: - """Returns true if this executor is in an irrecoverable error state. If this method - returns true, :property:executor_exception should contain an exception indicating the - cause. - """ - pass - - @property - @abstractmethod - def executor_exception(self) -> Exception: - """Returns an exception that indicates why this executor is in an irrecoverable state.""" - pass - @property def run_dir(self) -> str: """Path to the run directory. diff --git a/parsl/executors/flux/executor.py b/parsl/executors/flux/executor.py index 65c7f701a9..6bb8d4fde0 100644 --- a/parsl/executors/flux/executor.py +++ b/parsl/executors/flux/executor.py @@ -18,7 +18,7 @@ import zmq from parsl.utils import RepresentationMixin -from parsl.executors.status_handling import NoStatusHandlingExecutor +from parsl.executors.base import ParslExecutor from parsl.executors.flux.execute_parsl_task import __file__ as _WORKER_PATH from parsl.executors.flux.flux_instance_manager import __file__ as _MANAGER_PATH from parsl.executors.errors import ScalingFailed @@ -124,7 +124,7 @@ def _complete_future( ) -class FluxExecutor(NoStatusHandlingExecutor, RepresentationMixin): +class FluxExecutor(ParslExecutor, RepresentationMixin): """Executor that uses Flux to schedule and run jobs. Every callable submitted to the executor is wrapped into a Flux job. @@ -189,7 +189,7 @@ def __init__( super().__init__() if provider is None: provider = LocalProvider() - self._provider = provider + self.provider = provider self.label = label if working_dir is None: working_dir = self.label + "_" + str(uuid.uuid4()) diff --git a/parsl/executors/status_handling.py b/parsl/executors/status_handling.py index b95a8bd610..495041297b 100644 --- a/parsl/executors/status_handling.py +++ b/parsl/executors/status_handling.py @@ -82,6 +82,14 @@ def _make_status_dict(self, block_ids: List[str], status_list: List[JobStatus]) @property def status_polling_interval(self): + """Returns the interval, in seconds, at which the status method should be called. The + assumption here is that, once initialized, an executor's polling interval is fixed. + In practice, at least given the current situation, the executor uses a single task provider + and this method is a delegate to the corresponding method in the provider. + + :return: the number of seconds to wait between calls to status() or zero if no polling + should be done + """ if self._provider is None: return 0 else: @@ -104,8 +112,10 @@ def outstanding(self) -> int: "outstanding()") def status(self) -> Dict[str, JobStatus]: - """Return status of all blocks.""" + """Return the status of all jobs/blocks currently known to this executor. + :return: a dictionary mapping block ids (in string) to job status + """ if self._provider: block_ids, job_ids = self._get_block_and_job_ids() status = self._make_status_dict(block_ids, self._provider.status(job_ids)) @@ -116,6 +126,11 @@ def status(self) -> Dict[str, JobStatus]: return status def set_bad_state_and_fail_all(self, exception: Exception): + """Allows external error handlers to mark this executor as irrecoverably bad and cause + all tasks submitted to it now and in the future to fail. The executor is responsible + for checking :method:bad_state_is_set() in the :method:submit() method and raising the + appropriate exception, which is available through :method:executor_exception(). + """ logger.exception("Setting bad state due to exception", exc_info=exception) self._executor_exception = exception # Set bad state to prevent new tasks from being submitted @@ -127,17 +142,50 @@ def set_bad_state_and_fail_all(self, exception: Exception): @property def bad_state_is_set(self): + """Returns true if this executor is in an irrecoverable error state. If this method + returns true, :property:executor_exception should contain an exception indicating the + cause. + """ return self._executor_bad_state.is_set() @property def executor_exception(self): + """Returns an exception that indicates why this executor is in an irrecoverable state.""" return self._executor_exception @property def error_management_enabled(self): + """Indicates whether worker error management is supported by this executor. Worker error + management is done externally to the executor. However, the executor must implement + certain status handling methods that allow this to function. These methods are: + + :method:handle_errors + :method:set_bad_state_and_fail_all + + The basic idea of worker error management is that an external entity maintains a view of + the state of the workers by calling :method:status() which is then processed to detect + abnormal conditions. This can be done externally, as well as internally, through + :method:handle_errors. If an entity external to the executor detects an abnormal condition, + it can notify the executor using :method:set_bad_state_and_fail_all(exception). + + Some of the scaffolding needed for implementing error management inside executors, + including implementations for the status handling methods above, is available in + :class:parsl.executors.status_handling.BlockProviderExecutor, which interested executors + should inherit from. Noop versions of methods that are related to status handling and + running parsl tasks through workers are implemented by + :class:parsl.executors.status_handling.NoStatusHandlingExecutor. + """ + return self.block_error_handler def handle_errors(self, status: Dict[str, JobStatus]) -> None: + """This method is called by the error management infrastructure after a status poll. The + executor implementing this method is then responsible for detecting abnormal conditions + based on the status of submitted jobs. If the executor does not implement any special + error handling, this method should return False, in which case a generic error handling + scheme will be used. + :param status: status of all jobs launched by this executor + """ if not self.block_error_handler: return init_blocks = 3 @@ -211,34 +259,3 @@ def _get_block_and_job_ids(self) -> Tuple[List[str], List[Any]]: @abstractproperty def workers_per_node(self) -> Union[int, float]: pass - - -class NoStatusHandlingExecutor(ParslExecutor): - @property - def status_polling_interval(self): - return -1 - - @property - def bad_state_is_set(self): - return False - - @property - def error_management_enabled(self): - return False - - @property - def executor_exception(self): - return None - - def set_bad_state_and_fail_all(self, exception: Exception): - pass - - def status(self): - return {} - - def handle_errors(self, status: Dict[str, JobStatus]) -> None: - pass - - @property - def provider(self): - return self._provider diff --git a/parsl/executors/threads.py b/parsl/executors/threads.py index 912dd7a79d..234756b9ce 100644 --- a/parsl/executors/threads.py +++ b/parsl/executors/threads.py @@ -5,7 +5,7 @@ from typing import List, Optional from parsl.data_provider.staging import Staging -from parsl.executors.status_handling import NoStatusHandlingExecutor +from parsl.executors.base import ParslExecutor from parsl.utils import RepresentationMixin from parsl.executors.errors import UnsupportedFeatureError @@ -13,7 +13,7 @@ logger = logging.getLogger(__name__) -class ThreadPoolExecutor(NoStatusHandlingExecutor, RepresentationMixin): +class ThreadPoolExecutor(ParslExecutor, RepresentationMixin): """A thread-based executor. Parameters @@ -30,7 +30,7 @@ class ThreadPoolExecutor(NoStatusHandlingExecutor, RepresentationMixin): def __init__(self, label: str = 'threads', max_threads: int = 2, thread_name_prefix: str = '', storage_access: Optional[List[Staging]] = None, working_dir: Optional[str] = None): - NoStatusHandlingExecutor.__init__(self) + ParslExecutor.__init__(self) self.label = label self.max_threads = max_threads self.thread_name_prefix = thread_name_prefix diff --git a/parsl/jobs/job_status_poller.py b/parsl/jobs/job_status_poller.py index d5d65a9419..56d5b1cb0d 100644 --- a/parsl/jobs/job_status_poller.py +++ b/parsl/jobs/job_status_poller.py @@ -5,9 +5,9 @@ from typing import Dict, Sequence from typing import List # noqa F401 (used in type annotation) -from parsl.executors.base import ParslExecutor from parsl.jobs.states import JobStatus, JobState from parsl.jobs.strategy import Strategy +from parsl.executors.status_handling import BlockProviderExecutor from parsl.monitoring.message_type import MessageType @@ -18,7 +18,7 @@ class PollItem: - def __init__(self, executor: ParslExecutor, dfk: "parsl.dataflow.dflow.DataFlowKernel"): + def __init__(self, executor: BlockProviderExecutor, dfk: "parsl.dataflow.dflow.DataFlowKernel"): self._executor = executor self._dfk = dfk self._interval = executor.status_polling_interval @@ -70,7 +70,7 @@ def status(self) -> Dict[str, JobStatus]: return self._status @property - def executor(self) -> ParslExecutor: + def executor(self) -> BlockProviderExecutor: return self._executor def scale_in(self, n, force=True, max_idletime=None): @@ -124,7 +124,7 @@ def _update_state(self) -> None: for item in self._poll_items: item.poll(now) - def add_executors(self, executors: Sequence[ParslExecutor]) -> None: + def add_executors(self, executors: Sequence[BlockProviderExecutor]) -> None: for executor in executors: if executor.status_polling_interval > 0: logger.debug("Adding executor {}".format(executor.label)) diff --git a/parsl/jobs/simple_error_handler.py b/parsl/jobs/simple_error_handler.py index d1baca15d9..72481c6932 100644 --- a/parsl/jobs/simple_error_handler.py +++ b/parsl/jobs/simple_error_handler.py @@ -1,10 +1,12 @@ +from __future__ import annotations + from typing import Dict -from parsl.executors.base import ParslExecutor +import parsl.executors.status_handling as status_handling from parsl.jobs.states import JobStatus, JobState -def simple_error_handler(executor: ParslExecutor, status: Dict[str, JobStatus], threshold: int): +def simple_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int): (total_jobs, failed_jobs) = _count_jobs(status) if total_jobs >= threshold and failed_jobs == total_jobs: executor.set_bad_state_and_fail_all(_get_error(status)) From abf1b96a7dfd867ced0d59ab32f2ed17d20a6aaf Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 7 Aug 2023 23:01:06 +0200 Subject: [PATCH 07/43] Remove executor-level error management enable/disable (#2856) After this PR, calls to handle_errors are always enabled, and it is the responsibility of the handle_errors implementation to ignore errors if the user has so chosen. The BlockProviderExecutor handle_errors implementation already has a test for that (duplicating the test in the now removed error_management_enabled code) so nothing needs to change there: this code only removes the redundant code path. --- parsl/executors/status_handling.py | 25 ------------------------- parsl/jobs/job_status_poller.py | 2 -- 2 files changed, 27 deletions(-) diff --git a/parsl/executors/status_handling.py b/parsl/executors/status_handling.py index 495041297b..6a06ea45d0 100644 --- a/parsl/executors/status_handling.py +++ b/parsl/executors/status_handling.py @@ -153,31 +153,6 @@ def executor_exception(self): """Returns an exception that indicates why this executor is in an irrecoverable state.""" return self._executor_exception - @property - def error_management_enabled(self): - """Indicates whether worker error management is supported by this executor. Worker error - management is done externally to the executor. However, the executor must implement - certain status handling methods that allow this to function. These methods are: - - :method:handle_errors - :method:set_bad_state_and_fail_all - - The basic idea of worker error management is that an external entity maintains a view of - the state of the workers by calling :method:status() which is then processed to detect - abnormal conditions. This can be done externally, as well as internally, through - :method:handle_errors. If an entity external to the executor detects an abnormal condition, - it can notify the executor using :method:set_bad_state_and_fail_all(exception). - - Some of the scaffolding needed for implementing error management inside executors, - including implementations for the status handling methods above, is available in - :class:parsl.executors.status_handling.BlockProviderExecutor, which interested executors - should inherit from. Noop versions of methods that are related to status handling and - running parsl tasks through workers are implemented by - :class:parsl.executors.status_handling.NoStatusHandlingExecutor. - """ - - return self.block_error_handler - def handle_errors(self, status: Dict[str, JobStatus]) -> None: """This method is called by the error management infrastructure after a status poll. The executor implementing this method is then responsible for detecting abnormal conditions diff --git a/parsl/jobs/job_status_poller.py b/parsl/jobs/job_status_poller.py index 56d5b1cb0d..44bc151e24 100644 --- a/parsl/jobs/job_status_poller.py +++ b/parsl/jobs/job_status_poller.py @@ -115,8 +115,6 @@ def poll(self): def _run_error_handlers(self, status: List[PollItem]): for es in status: - if not es.executor.error_management_enabled: - return es.executor.handle_errors(es.status) def _update_state(self) -> None: From eb7ffe90b6476f542760151356e163f8363f1842 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 11 Aug 2023 08:17:25 +0200 Subject: [PATCH 08/43] Upgrade flake8 and subsequent fixes, in preparation for Python 3.12 (#2857) Upcoming Python 3.12 (at least in python 3.12-rc1) requires a newer version of flake8 as flake8 6.0.0 gets confused by things happening inside format strings with Python 3.12-rc1. This PR makes that upgrade. Flake8 6.1.0 introduces a new rule to use is-equality, rather than ==-equality, when comparing types, and so this PR fixes violations of that rule too. In theory this shouldn't change behaviour. --- parsl/dataflow/memoization.py | 6 +++--- parsl/tests/integration/test_channels/test_ssh_errors.py | 2 +- parsl/tests/test_bash_apps/test_keyword_overlaps.py | 2 +- test-requirements.txt | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/parsl/dataflow/memoization.py b/parsl/dataflow/memoization.py index b58a179fb5..e4d657ccaa 100644 --- a/parsl/dataflow/memoization.py +++ b/parsl/dataflow/memoization.py @@ -60,7 +60,7 @@ def id_for_memo_pickle(obj: object, output_ref: bool = False) -> bytes: @id_for_memo.register(list) def id_for_memo_list(denormalized_list: list, output_ref: bool = False) -> bytes: - if type(denormalized_list) != list: + if type(denormalized_list) is not list: raise ValueError("id_for_memo_list cannot work on subclasses of list") normalized_list = [] @@ -73,7 +73,7 @@ def id_for_memo_list(denormalized_list: list, output_ref: bool = False) -> bytes @id_for_memo.register(tuple) def id_for_memo_tuple(denormalized_tuple: tuple, output_ref: bool = False) -> bytes: - if type(denormalized_tuple) != tuple: + if type(denormalized_tuple) is not tuple: raise ValueError("id_for_memo_tuple cannot work on subclasses of tuple") normalized_list = [] @@ -91,7 +91,7 @@ def id_for_memo_dict(denormalized_dict: dict, output_ref: bool = False) -> bytes When output_ref=True, the values are normalised as output refs, but the keys are not. """ - if type(denormalized_dict) != dict: + if type(denormalized_dict) is not dict: raise ValueError("id_for_memo_dict cannot work on subclasses of dict") keys = sorted(denormalized_dict) diff --git a/parsl/tests/integration/test_channels/test_ssh_errors.py b/parsl/tests/integration/test_channels/test_ssh_errors.py index 715b3a14bb..7da7cc34ef 100644 --- a/parsl/tests/integration/test_channels/test_ssh_errors.py +++ b/parsl/tests/integration/test_channels/test_ssh_errors.py @@ -13,7 +13,7 @@ def test_error_1(): try: connect_and_list("bad.url.gov", "ubuntu") except Exception as e: - assert type(e) == SSHException, "Expected SSException, got: {0}".format(e) + assert type(e) is SSHException, "Expected SSException, got: {0}".format(e) def test_error_2(): diff --git a/parsl/tests/test_bash_apps/test_keyword_overlaps.py b/parsl/tests/test_bash_apps/test_keyword_overlaps.py index b72714bfce..41a9ba7429 100644 --- a/parsl/tests/test_bash_apps/test_keyword_overlaps.py +++ b/parsl/tests/test_bash_apps/test_keyword_overlaps.py @@ -3,7 +3,7 @@ @parsl.bash_app def my_app(cache=7): - assert type(cache) == int + assert type(cache) is int return "true" diff --git a/test-requirements.txt b/test-requirements.txt index 536434ca13..bbca07b128 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -1,4 +1,4 @@ -flake8==6.0.0 +flake8==6.1.0 ipyparallel pandas pytest>=7.4.0,<8 From 7a34c3dc2476098f9efb759dc576104deac411ff Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 11 Aug 2023 15:10:03 +0200 Subject: [PATCH 09/43] Remove mypy configuration for removed ExecutorStatus (#2859) ExecutorStatus and its containing module were removed in PR #2835. --- mypy.ini | 5 ----- 1 file changed, 5 deletions(-) diff --git a/mypy.ini b/mypy.ini index 14272918b7..739e572870 100644 --- a/mypy.ini +++ b/mypy.ini @@ -32,11 +32,6 @@ disallow_untyped_defs = True disallow_any_expr = True disallow_any_decorated = True -[mypy-parsl.dataflow.executor_status.*] -disallow_untyped_defs = True -disallow_any_expr = True -disallow_any_decorated = True - [mypy-parsl.dataflow.futures.*] disallow_untyped_defs = True disallow_any_decorated = True From 0ea4ef74e2478da707466e7332f2039769f34e48 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 11 Aug 2023 16:30:37 +0200 Subject: [PATCH 10/43] Correct incorrect type annotation inside strategy code (#2860) This was not causing an error in master because no uses of the incorrectly annotated attribute (self.executors) were type checked. This PR introduces a typed dict to represent the (single element) dictionary used for strategy state. (A futher tidyup could be to remove that typed dict entirely and replace with the single value it contains, but this PR is trying to be descriptive of existing code) The incorrect type annotation was added in #2712 --- parsl/jobs/strategy.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index 02af4dc207..032ba16d8c 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -3,12 +3,11 @@ import time import math import warnings -from typing import Dict, List, Optional +from typing import Dict, List, Optional, TypedDict import parsl.jobs.job_status_poller as jsp from parsl.executors import HighThroughputExecutor -from parsl.executors.base import ParslExecutor from parsl.executors.status_handling import BlockProviderExecutor from parsl.jobs.states import JobState from parsl.process_loggers import wrap_with_logs @@ -17,6 +16,16 @@ logger = logging.getLogger(__name__) +class ExecutorState(TypedDict): + """Strategy relevant state for an executor + """ + + idle_since: Optional[float] + """The timestamp at which an executor became idle. + If the executor is not idle, then None. + """ + + class Strategy: """Scaling strategy. @@ -115,7 +124,7 @@ class Strategy: def __init__(self, *, strategy: Optional[str], max_idletime: float): """Initialize strategy.""" - self.executors: Dict[str, ParslExecutor] + self.executors: Dict[str, ExecutorState] self.executors = {} self.max_idletime = max_idletime From 1a70041ead00f8142cf6616b1a7bb29e46f5748f Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 11 Aug 2023 17:37:23 +0200 Subject: [PATCH 11/43] Fix mutable ignore_for_cache default in python apps (#2862) The rest of the codebase already accepts `None` as a default here (for example, the corresponding parameter to bash_app) so there aren't any further code changes needed to deal with this. This comes from working through flake8-bugbear errors in draft PR #2832 --- parsl/app/python.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/app/python.py b/parsl/app/python.py index 9107e16c47..27a2c464ba 100644 --- a/parsl/app/python.py +++ b/parsl/app/python.py @@ -36,7 +36,7 @@ def inject_exception(thread): class PythonApp(AppBase): """Extends AppBase to cover the Python App.""" - def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ignore_for_cache=[], join=False): + def __init__(self, func, data_flow_kernel=None, cache=False, executors='all', ignore_for_cache=None, join=False): super().__init__( wrap_error(func), data_flow_kernel=data_flow_kernel, From 976deea966d6204c8c547343f43f882b160c498c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 11 Aug 2023 20:22:15 +0200 Subject: [PATCH 12/43] Tidyup messy join app refactoring PR #2621 (#2861) PR #2621 didn't remove the user facing join parameter it was intended to, instead just ignoring it. That PR #2621 also added a new executors parameter to the join_app decorator, which is then ignored (as it should be - because join_apps should be forced to run in the _parsl_internal executor). This PR removes that spurious parameter. This will be a breaking change for anyone explicitly specifying the _parsl_internal decorator to join_app, or for anyone explicitly specifying join=False to python_app. (Other uses of those parameters are already broken because those parameters are ignored, and they will now be broken in different ways) --- parsl/app/app.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/parsl/app/app.py b/parsl/app/app.py index dd7282cd5f..77510a2863 100644 --- a/parsl/app/app.py +++ b/parsl/app/app.py @@ -73,15 +73,14 @@ def python_app(function=None, data_flow_kernel: Optional[DataFlowKernel] = None, cache: bool = False, executors: Union[List[str], Literal['all']] = 'all', - ignore_for_cache: Optional[List[str]] = None, - join: bool = False): + ignore_for_cache: Optional[List[str]] = None): """Decorator function for making python apps. Parameters ---------- function : function Do not pass this keyword argument directly. This is needed in order to allow for omitted parenthesis, - for example, ``@join_app`` if using all defaults or ``@python_app(walltime=120)``. If the + for example, ``@python_app`` if using all defaults or ``@python_app(walltime=120)``. If the decorator is used alone, function will be the actual function being decorated, whereas if it is called with arguments, function will be None. Default is None. data_flow_kernel : DataFlowKernel @@ -112,7 +111,6 @@ def wrapper(f): def join_app(function=None, data_flow_kernel: Optional[DataFlowKernel] = None, cache: bool = False, - executors: Union[List[str], Literal['all']] = 'all', ignore_for_cache: Optional[List[str]] = None): """Decorator function for making join apps From 063033a8c6680da49c4f8af381296b3730872ea7 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Sat, 12 Aug 2023 14:51:38 +0200 Subject: [PATCH 13/43] Remove unused key in iteration of task dictionary (#2863) This comes from a flake8-bugbear warning. --- parsl/dataflow/dflow.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index 936a32acf4..dd13b6adbd 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -1146,8 +1146,11 @@ def wait_for_current_tasks(self) -> None: logger.info("Waiting for all remaining tasks to complete") - items = list(self.tasks.items()) - for task_id, task_record in items: + # .values is made into a list immediately to reduce (although not + # eliminate) a race condition where self.tasks can be modified + # elsewhere by a completing task being removed from the dictionary. + task_records = list(self.tasks.values()) + for task_record in task_records: # .exception() is a less exception throwing way of # waiting for completion than .result() fut = task_record['app_fu'] From 322f01b7cf1611c3cb4af44239e2ec87f057c9ce Mon Sep 17 00:00:00 2001 From: Yadu Nand Babuji Date: Sat, 12 Aug 2023 09:20:04 -0500 Subject: [PATCH 14/43] Add user-pluggable block error handling and a new sliding-window error handler (#2858) This PR aims to address the limited capabilities of the current simple_error_handler at stopping the Parsl runtime when there are repeated failures. The current system only fails if all jobs fail, which is only indicative of configuration errors or problem with the batch scheduler. This PR adds new behavior that updates the existing block_error_handler bool variable to take a custom error handler. This PR also adds a new windows_error_handler to better handle long running workflows. This handler shuts down if and only if the last N jobs all failed. Co-authored-by: Ben Clifford --- parsl/executors/high_throughput/executor.py | 5 +- parsl/executors/status_handling.py | 27 +-- ...ple_error_handler.py => error_handlers.py} | 17 +- .../test_scaling/test_block_error_handler.py | 168 ++++++++++++++++++ 4 files changed, 201 insertions(+), 16 deletions(-) rename parsl/jobs/{simple_error_handler.py => error_handlers.py} (65%) create mode 100644 parsl/tests/test_scaling/test_block_error_handler.py diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index cc499ef1c3..69af77a86b 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -9,12 +9,13 @@ import warnings from multiprocessing import Queue from typing import Dict, Sequence # noqa F401 (used in type annotation) -from typing import List, Optional, Tuple, Union +from typing import List, Optional, Tuple, Union, Callable import math from parsl.serialize import pack_apply_message, deserialize from parsl.serialize.errors import SerializationError, DeserializationError from parsl.app.errors import RemoteExceptionWrapper +from parsl.jobs.states import JobStatus from parsl.executors.high_throughput import zmq_pipes from parsl.executors.high_throughput import interchange from parsl.executors.errors import ( @@ -212,7 +213,7 @@ def __init__(self, poll_period: int = 10, address_probe_timeout: Optional[int] = None, worker_logdir_root: Optional[str] = None, - block_error_handler: bool = True): + block_error_handler: Union[bool, Callable[[BlockProviderExecutor, Dict[str, JobStatus]], None]] = True): logger.debug("Initializing HighThroughputExecutor") diff --git a/parsl/executors/status_handling.py b/parsl/executors/status_handling.py index 6a06ea45d0..29070ae4d5 100644 --- a/parsl/executors/status_handling.py +++ b/parsl/executors/status_handling.py @@ -1,19 +1,19 @@ +from __future__ import annotations import logging import threading from itertools import compress from abc import abstractmethod, abstractproperty from concurrent.futures import Future -from typing import List, Any, Dict, Optional, Tuple, Union +from typing import List, Any, Dict, Optional, Tuple, Union, Callable import parsl # noqa F401 from parsl.executors.base import ParslExecutor from parsl.executors.errors import BadStateException, ScalingFailed from parsl.jobs.states import JobStatus, JobState +from parsl.jobs.error_handlers import simple_error_handler, noop_error_handler from parsl.providers.base import ExecutionProvider from parsl.utils import AtomicIDCounter -import parsl.jobs.simple_error_handler as error_handler - logger = logging.getLogger(__name__) @@ -47,10 +47,18 @@ class BlockProviderExecutor(ParslExecutor): """ def __init__(self, *, provider: Optional[ExecutionProvider], - block_error_handler: bool): + block_error_handler: Union[bool, Callable[[BlockProviderExecutor, Dict[str, JobStatus]], None]]): super().__init__() self._provider = provider - self.block_error_handler = block_error_handler + self.block_error_handler: Callable[[BlockProviderExecutor, Dict[str, JobStatus]], None] + if isinstance(block_error_handler, bool): + if block_error_handler: + self.block_error_handler = simple_error_handler + else: + self.block_error_handler = noop_error_handler + else: + self.block_error_handler = block_error_handler + # errors can happen during the submit call to the provider; this is used # to keep track of such errors so that they can be handled in one place # together with errors reported by status() @@ -161,14 +169,7 @@ def handle_errors(self, status: Dict[str, JobStatus]) -> None: scheme will be used. :param status: status of all jobs launched by this executor """ - if not self.block_error_handler: - return - init_blocks = 3 - if hasattr(self.provider, 'init_blocks'): - init_blocks = self.provider.init_blocks - if init_blocks < 1: - init_blocks = 1 - error_handler.simple_error_handler(self, status, init_blocks) + self.block_error_handler(self, status) @property def tasks(self) -> Dict[object, Future]: diff --git a/parsl/jobs/simple_error_handler.py b/parsl/jobs/error_handlers.py similarity index 65% rename from parsl/jobs/simple_error_handler.py rename to parsl/jobs/error_handlers.py index 72481c6932..2bd91a8c4b 100644 --- a/parsl/jobs/simple_error_handler.py +++ b/parsl/jobs/error_handlers.py @@ -6,12 +6,27 @@ from parsl.jobs.states import JobStatus, JobState -def simple_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int): +def noop_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): + pass + + +def simple_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): (total_jobs, failed_jobs) = _count_jobs(status) + if hasattr(executor.provider, "init_blocks"): + threshold = max(1, executor.provider.init_blocks) + if total_jobs >= threshold and failed_jobs == total_jobs: executor.set_bad_state_and_fail_all(_get_error(status)) +def windowed_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): + sorted_status = [(key, status[key]) for key in sorted(status, key=lambda x: int(x))] + current_window = dict(sorted_status[-threshold:]) + total, failed = _count_jobs(current_window) + if failed == threshold: + executor.set_bad_state_and_fail_all(_get_error(status)) + + def _count_jobs(status: Dict[str, JobStatus]): total = 0 failed = 0 diff --git a/parsl/tests/test_scaling/test_block_error_handler.py b/parsl/tests/test_scaling/test_block_error_handler.py new file mode 100644 index 0000000000..9d680212e3 --- /dev/null +++ b/parsl/tests/test_scaling/test_block_error_handler.py @@ -0,0 +1,168 @@ +import pytest + +from parsl.executors import HighThroughputExecutor +from parsl.providers import LocalProvider +from unittest.mock import Mock +from parsl.jobs.states import JobStatus, JobState +from parsl.jobs.error_handlers import simple_error_handler, windowed_error_handler, noop_error_handler +from functools import partial + + +@pytest.mark.local +def test_block_error_handler_false(): + mock = Mock() + htex = HighThroughputExecutor(block_error_handler=False) + assert htex.block_error_handler is noop_error_handler + htex.set_bad_state_and_fail_all = mock + + bad_jobs = {'1': JobStatus(JobState.FAILED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.FAILED), + '4': JobStatus(JobState.FAILED)} + + htex.handle_errors(bad_jobs) + mock.assert_not_called() + + +@pytest.mark.local +def test_block_error_handler_mock(): + handler_mock = Mock() + htex = HighThroughputExecutor(block_error_handler=handler_mock) + assert htex.block_error_handler is handler_mock + + bad_jobs = {'1': JobStatus(JobState.FAILED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.FAILED), + '4': JobStatus(JobState.FAILED)} + + htex.handle_errors(bad_jobs) + handler_mock.assert_called() + handler_mock.assert_called_with(htex, bad_jobs) + + +@pytest.mark.local +def test_simple_error_handler(): + htex = HighThroughputExecutor(block_error_handler=simple_error_handler, + provider=LocalProvider(init_blocks=3)) + + assert htex.block_error_handler is simple_error_handler + + bad_state_mock = Mock() + htex.set_bad_state_and_fail_all = bad_state_mock + + bad_jobs = {'1': JobStatus(JobState.FAILED), + '2': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + # Check the bad behavior where if any job is not failed + # bad state won't be set + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.FAILED), + '4': JobStatus(JobState.FAILED)} + + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.FAILED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.FAILED), + '4': JobStatus(JobState.FAILED)} + + htex.handle_errors(bad_jobs) + bad_state_mock.assert_called() + + +@pytest.mark.local +def test_windowed_error_handler(): + htex = HighThroughputExecutor(block_error_handler=windowed_error_handler) + assert htex.block_error_handler is windowed_error_handler + + bad_state_mock = Mock() + htex.set_bad_state_and_fail_all = bad_state_mock + + bad_jobs = {'1': JobStatus(JobState.FAILED), + '2': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.FAILED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.COMPLETED), + '4': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.FAILED), + '4': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_called() + + +@pytest.mark.local +def test_windowed_error_handler_sorting(): + htex = HighThroughputExecutor(block_error_handler=windowed_error_handler) + assert htex.block_error_handler is windowed_error_handler + + bad_state_mock = Mock() + htex.set_bad_state_and_fail_all = bad_state_mock + + bad_jobs = {'8': JobStatus(JobState.FAILED), + '9': JobStatus(JobState.FAILED), + '10': JobStatus(JobState.FAILED), + '11': JobStatus(JobState.COMPLETED), + '12': JobStatus(JobState.COMPLETED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'8': JobStatus(JobState.COMPLETED), + '9': JobStatus(JobState.FAILED), + '21': JobStatus(JobState.FAILED), + '22': JobStatus(JobState.FAILED), + '10': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_called() + + +@pytest.mark.local +def test_windowed_error_handler_with_threshold(): + error_handler = partial(windowed_error_handler, threshold=2) + htex = HighThroughputExecutor(block_error_handler=error_handler) + assert htex.block_error_handler is error_handler + + bad_state_mock = Mock() + htex.set_bad_state_and_fail_all = bad_state_mock + + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.FAILED), + '3': JobStatus(JobState.COMPLETED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.COMPLETED), + '3': JobStatus(JobState.COMPLETED), + '4': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_not_called() + + bad_jobs = {'1': JobStatus(JobState.COMPLETED), + '2': JobStatus(JobState.COMPLETED), + '3': JobStatus(JobState.FAILED), + '4': JobStatus(JobState.FAILED)} + htex.handle_errors(bad_jobs) + bad_state_mock.assert_called() From cb1b365bae39c92c7378d9d9ba5cbc46261b3b20 Mon Sep 17 00:00:00 2001 From: "Andrew S. Rosen" Date: Sat, 12 Aug 2023 22:12:22 -0700 Subject: [PATCH 15/43] Update .wci.yml (#2868) --- .wci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.wci.yml b/.wci.yml index 9ad60dacfb..c11a2e82b4 100644 --- a/.wci.yml +++ b/.wci.yml @@ -34,6 +34,7 @@ execution_environment: - LSF - PBS - Cobalt + - Flux - GridEngine - HTCondor - AWS From 1ffa6c3fce4ebf7c38be2c44526111f39502fc10 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 21 Aug 2023 17:43:56 +0200 Subject: [PATCH 16/43] Make resource monitor exit when monitored process has gone away (#2866) Issue #2840 reports that the process monitor does not always exit if the workflow process exits unexpectedly, and looking at the cpython implementation, it looks like these processes will only be terminated at worker shutdown if the worker shuts down normally. If it is surprise terminated/killed, it will not shut down its dependent daemons such as any resource monitors it has launched. This PR adds a liveness check for the process that it is monitoring, which is the parent worker process; so that if that process goes away without telling the resource monitor to exit, then the resource monitor will exit anyway. --- parsl/monitoring/remote.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/monitoring/remote.py b/parsl/monitoring/remote.py index 479a7ccfa4..d42e0079b4 100644 --- a/parsl/monitoring/remote.py +++ b/parsl/monitoring/remote.py @@ -262,7 +262,7 @@ def accumulate_and_prepare() -> Dict[str, Any]: next_send = time.time() accumulate_dur = 5.0 # TODO: make configurable? - while not terminate_event.is_set(): + while not terminate_event.is_set() and pm.is_running(): logging.debug("start of monitoring loop") try: d = accumulate_and_prepare() From 2e158b04eeb3e0ddcae3231eec789a59d2b9a10c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 21 Aug 2023 18:35:07 +0200 Subject: [PATCH 17/43] Rework _get_error in simple_error_handler to give more information (#2867) Before (in pytest output, with htex launch_cmd="/bin/false"): ``` Exception: STDOUT: Found cores : 8 Launching worker: 1 ``` After: ``` parsl.jobs.errors.TooManyJobFailuresError: Error 1: EXIT CODE: 1 STDOUT: Found cores : 8 Launching worker: 1 ``` * Fix counting so it counts even when js.message is None * Add exit code * Use a new error subclass rather than Exception * Pluralise messages in for no-messages string and remove brackets '[No error message received]' vs 'No error messages received' --- parsl/jobs/error_handlers.py | 16 ++++++++++++---- parsl/jobs/errors.py | 7 +++++++ 2 files changed, 19 insertions(+), 4 deletions(-) create mode 100644 parsl/jobs/errors.py diff --git a/parsl/jobs/error_handlers.py b/parsl/jobs/error_handlers.py index 2bd91a8c4b..c35a87447d 100644 --- a/parsl/jobs/error_handlers.py +++ b/parsl/jobs/error_handlers.py @@ -4,6 +4,7 @@ import parsl.executors.status_handling as status_handling from parsl.jobs.states import JobStatus, JobState +from parsl.jobs.errors import TooManyJobFailuresError def noop_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): @@ -42,18 +43,25 @@ def _get_error(status: Dict[str, JobStatus]) -> Exception: err = "" count = 1 for js in status.values(): + err = err + f"Error {count}:\n" + count += 1 + if js.message is not None: - err = err + "{}. {}\n".format(count, js.message) - count += 1 + err = err + f"\t{js.message}\n" + + if js.exit_code is not None: + err = err + f"\tEXIT CODE: {js.exit_code}\n" + stdout = js.stdout_summary if stdout: err = err + "\tSTDOUT: {}\n".format(stdout) + stderr = js.stderr_summary if stderr: err = err + "\tSTDERR: {}\n".format(stderr) if len(err) == 0: - err = "[No error message received]" + err = "No error messages received" # wrapping things in an exception here doesn't really help in providing more information # than the string itself - return Exception(err) + return TooManyJobFailuresError(err) diff --git a/parsl/jobs/errors.py b/parsl/jobs/errors.py new file mode 100644 index 0000000000..6d42f429be --- /dev/null +++ b/parsl/jobs/errors.py @@ -0,0 +1,7 @@ +from parsl.errors import ParslError + + +class TooManyJobFailuresError(ParslError): + """Indicates that executor is shut down because of too many block failures. + """ + pass From 93a141373870dd4de1121931d02b969c72a83461 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 21 Aug 2023 19:08:25 +0200 Subject: [PATCH 18/43] Remove unused re-used DFK default configuration object (#2864) The default configuration actually comes from DataFlowKernelLoader.load() and the Config() object specified in the DataFlowKernel.__init__ default parameters is never used, as all constructions of DataFlowKernel in the codebase specify a config. This comes from flake8-bugbear --- parsl/dataflow/dflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index dd13b6adbd..b87ff0ded8 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -69,7 +69,7 @@ class DataFlowKernel: """ @typechecked - def __init__(self, config: Config = Config()) -> None: + def __init__(self, config: Config) -> None: """Initialize the DataFlowKernel. Parameters From 40b969f7a9e16b3fcab2db127753bf85cbcb5375 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 21 Aug 2023 19:52:53 +0200 Subject: [PATCH 19/43] Remove unused defaults from dfk.submit (#2865) All of these parameters are specified by invocations in the relevant app decorators and so these defaults are never used. Removing the default app_kwargs = {} is especially interesting for mutability safety. This comes from flake8-bugbear --- parsl/dataflow/dflow.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index b87ff0ded8..e9f9247cc2 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -898,10 +898,10 @@ def _unwrap_futures(self, args, kwargs): def submit(self, func: Callable, app_args: Sequence[Any], - executors: Union[str, Sequence[str]] = 'all', - cache: bool = False, - ignore_for_cache: Optional[Sequence[str]] = None, - app_kwargs: Dict[str, Any] = {}, + executors: Union[str, Sequence[str]], + cache: bool, + ignore_for_cache: Optional[Sequence[str]], + app_kwargs: Dict[str, Any], join: bool = False) -> AppFuture: """Add task to the dataflow system. From 465eb43be7d276a73af19d13c47da3c4f60eaaf8 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 5 Sep 2023 14:55:46 +0200 Subject: [PATCH 20/43] Inhibit release if current master is already a release (#2617) --- .github/workflows/python-publish-to-testpypi.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/python-publish-to-testpypi.yml b/.github/workflows/python-publish-to-testpypi.yml index e280d61c57..de2afa7e3b 100644 --- a/.github/workflows/python-publish-to-testpypi.yml +++ b/.github/workflows/python-publish-to-testpypi.yml @@ -32,6 +32,12 @@ jobs: steps: - uses: actions/checkout@v3 + + - name: Check if this commit is already released + id: already_released + run: | + if git tag --contains HEAD | grep -e '^[0-9]\{4\}\.[0-9]\{2\}\.[0-9]\{2\}$' ; then exit 1 ; fi + - name: Set up Python uses: actions/setup-python@v3 with: From df235e4ec67d3035bcdff645de866fe0e0cdac89 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 15 Sep 2023 11:00:20 +0200 Subject: [PATCH 21/43] Make RuntimeErrors in DFK into custom error classes (#2878) Fixes issue #2873 --- parsl/dataflow/dflow.py | 12 ++++++------ parsl/errors.py | 10 ++++++++++ 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index e9f9247cc2..fc1b38effc 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -32,7 +32,7 @@ from parsl.dataflow.rundirs import make_rundir from parsl.dataflow.states import States, FINAL_STATES, FINAL_FAILURE_STATES from parsl.dataflow.taskrecord import TaskRecord -from parsl.errors import ConfigurationError +from parsl.errors import ConfigurationError, InternalConsistencyError, NoDataFlowKernelError from parsl.jobs.job_status_poller import JobStatusPoller from parsl.jobs.states import JobStatus, JobState from parsl.usage_tracking.usage import UsageTracker @@ -295,7 +295,7 @@ def handle_exec_update(self, task_record: TaskRecord, future: Future) -> None: task_record['try_time_returned'] = datetime.datetime.now() if not future.done(): - raise RuntimeError("done callback called, despite future not reporting itself as done") + raise InternalConsistencyError("done callback called, despite future not reporting itself as done") try: res = self._unwrap_remote_exception_wrapper(future) @@ -535,7 +535,7 @@ def handle_app_update(self, task_record: TaskRecord, future: AppFuture) -> None: elif self.checkpoint_mode is None: pass else: - raise RuntimeError(f"Invalid checkpoint mode {self.checkpoint_mode}") + raise InternalConsistencyError(f"Invalid checkpoint mode {self.checkpoint_mode}") self.wipe_task(task_id) return @@ -933,7 +933,7 @@ def submit(self, ignore_for_cache = list(ignore_for_cache) if self.cleanup_called: - raise RuntimeError("Cannot submit to a DFK that has been cleaned up") + raise NoDataFlowKernelError("Cannot submit to a DFK that has been cleaned up") task_id = self.task_count self.task_count += 1 @@ -1420,7 +1420,7 @@ def load(cls, config: Optional[Config] = None) -> DataFlowKernel: - DataFlowKernel : The loaded DataFlowKernel object. """ if cls._dfk is not None: - raise RuntimeError('Config has already been loaded') + raise ConfigurationError('Config has already been loaded') if config is None: cls._dfk = DataFlowKernel(Config()) @@ -1441,5 +1441,5 @@ def wait_for_current_tasks(cls) -> None: def dfk(cls) -> DataFlowKernel: """Return the currently-loaded DataFlowKernel.""" if cls._dfk is None: - raise RuntimeError('Must first load config') + raise ConfigurationError('Must first load config') return cls._dfk diff --git a/parsl/errors.py b/parsl/errors.py index 571ced1e8d..5de6010dcd 100644 --- a/parsl/errors.py +++ b/parsl/errors.py @@ -25,3 +25,13 @@ def __str__(self) -> str: return "The functionality requested requires optional modules {0} which could not be imported, because: {1}".format( self.module_names, self.reason ) + + +class InternalConsistencyError(ParslError): + """Raised when a component enounters an internal inconsistency. + """ + + +class NoDataFlowKernelError(ParslError): + """Raised when no DataFlowKernel is available for an operation that needs one. + """ From f4d5408e7ee0f4d3535c156bc761576df6203383 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Fri, 15 Sep 2023 11:32:01 +0200 Subject: [PATCH 22/43] Add ignore_for_cache docstring to decorators. Fixes #2875 (#2877) --- parsl/app/app.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/parsl/app/app.py b/parsl/app/app.py index 77510a2863..07b2ef8198 100644 --- a/parsl/app/app.py +++ b/parsl/app/app.py @@ -34,6 +34,7 @@ def __init__(self, func, data_flow_kernel=None, executors='all', cache=False, ig after calling :meth:`parsl.dataflow.dflow.DataFlowKernelLoader.load`. - executors (str|list) : Labels of the executors that this app can execute over. Default is 'all'. - cache (Bool) : Enable caching of this app ? + - ignore_for_cache (list|None): Names of arguments which will be ignored by the caching mechanism. Returns: - App object. @@ -90,6 +91,8 @@ def python_app(function=None, Labels of the executors that this app can execute over. Default is 'all'. cache : bool Enable caching of the app call. Default is False. + ignore_for_cache : (list|None) + Names of arguments which will be ignored by the caching mechanism. """ from parsl.app.python import PythonApp @@ -126,6 +129,8 @@ def join_app(function=None, be omitted only after calling :meth:`parsl.dataflow.dflow.DataFlowKernelLoader.load`. Default is None. cache : bool Enable caching of the app call. Default is False. + ignore_for_cache : (list|None) + Names of arguments which will be ignored by the caching mechanism. """ from parsl.app.python import PythonApp @@ -167,6 +172,8 @@ def bash_app(function=None, Labels of the executors that this app can execute over. Default is 'all'. cache : bool Enable caching of the app call. Default is False. + ignore_for_cache : (list|None) + Names of arguments which will be ignored by the caching mechanism. """ from parsl.app.bash import BashApp From 5b26a0332d62c35fbcfd5b78a4fa890af4d2bef9 Mon Sep 17 00:00:00 2001 From: KiwiFac3 <114454917+KiwiFac3@users.noreply.github.com> Date: Mon, 18 Sep 2023 05:14:07 -0700 Subject: [PATCH 23/43] Update more of Globus data provider to Globus Transfer v3 API (#2745) Some Globus functions have different arguments that was previously indicated. I changed those function to have the correct parameters and did the necessary changes to the code to maintain it functionality. Co-authored-by: Mousa Abughosh Co-authored-by: Ben Clifford Co-authored-by: Kevin Hunter Kesling --- parsl/data_provider/globus.py | 23 +++++++++-------------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/parsl/data_provider/globus.py b/parsl/data_provider/globus.py index b950dc9088..50bddc5ecf 100644 --- a/parsl/data_provider/globus.py +++ b/parsl/data_provider/globus.py @@ -97,19 +97,14 @@ def transfer_file(cls, src_ep, dst_ep, src_path, dst_path): while not tc.task_wait(task['task_id'], timeout=60): task = tc.get_task(task['task_id']) # Get the last error Globus event - events = tc.task_event_list(task['task_id'], num_results=1, filter='is_error:1') - try: - event = next(events) - # No error reported, the transfer is still running - except StopIteration: - continue - # Print the error event to stderr and Parsl file log if it was not yet printed - if event['time'] != last_event_time: - last_event_time = event['time'] - logger.warning('Non-critical Globus Transfer error event for globus://{}{}: "{}" at {}. Retrying...'.format( - src_ep, src_path, event['description'], event['time'])) - logger.debug('Globus Transfer error details: {}'.format(event['details'])) - + task_id = task['task_id'] + for event in tc.task_event_list(task_id): + if event['time'] != last_event_time: + last_event_time = event['time'] + logger.warning( + 'Non-critical Globus Transfer error event for globus://{}{}: "{}" at {}. Retrying...'.format( + src_ep, src_path, event['description'], event['time'])) + logger.debug('Globus Transfer error details: {}'.format(event['details'])) """ The Globus transfer job (task) has been terminated (is not ACTIVE). Check if the transfer SUCCEEDED or FAILED. @@ -120,7 +115,7 @@ def transfer_file(cls, src_ep, dst_ep, src_path, dst_path): task['task_id'], src_ep, src_path, dst_ep, dst_path)) else: logger.debug('Globus Transfer task: {}'.format(task)) - events = tc.task_event_list(task['task_id'], num_results=1, filter='is_error:1') + events = tc.task_event_list(task['task_id']) event = events.data[0] raise Exception('Globus transfer {}, from {}{} to {}{} failed due to error: "{}"'.format( task['task_id'], src_ep, src_path, dst_ep, dst_path, event['details'])) From 5d5bb1151f27d94eadef094eda26d6a2d00503f8 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 14:49:54 +0200 Subject: [PATCH 24/43] Tighten MissingOutputs file type (#2884) Outputs must be Files - although early on in Parsl, it was possible to specify them as string pathnames. This now-removed string-or-File dual annotation is probably a holdover from that. --- parsl/app/errors.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/parsl/app/errors.py b/parsl/app/errors.py index 4a15558d37..b9da018b6e 100644 --- a/parsl/app/errors.py +++ b/parsl/app/errors.py @@ -64,10 +64,9 @@ class MissingOutputs(ParslError): Contains: reason(string) - outputs(List of strings/files..) + outputs(List of files) """ - - def __init__(self, reason: str, outputs: List[Union[str, File]]) -> None: + def __init__(self, reason: str, outputs: List[File]) -> None: super().__init__(reason, outputs) self.reason = reason self.outputs = outputs From 0698390748a1718e4ecc6017e4c38e45a22362ad Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 14:37:16 +0100 Subject: [PATCH 25/43] Upgrade mypy (#2879) --- parsl/monitoring/monitoring.py | 7 ++++++- test-requirements.txt | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/parsl/monitoring/monitoring.py b/parsl/monitoring/monitoring.py index 8a238d2b4e..783ee8d098 100644 --- a/parsl/monitoring/monitoring.py +++ b/parsl/monitoring/monitoring.py @@ -463,7 +463,12 @@ def start(self, if 'exit_now' in msg[1] and msg[1]['exit_now']: router_keep_going = False else: - self.logger.error(f"Discarding message from interchange with unknown type {msg[0].value}") + # There is a type: ignore here because if msg[0] + # is of the correct type, this code is unreachable, + # but there is no verification that the message + # received from ic_channel.recv_pyobj() is actually + # of that type. + self.logger.error(f"Discarding message from interchange with unknown type {msg[0].value}") # type: ignore[unreachable] except zmq.Again: pass except Exception: diff --git a/test-requirements.txt b/test-requirements.txt index bbca07b128..aa492ef5ce 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -7,7 +7,7 @@ pytest-random-order mock>=1.0.0 nbsphinx sphinx_rtd_theme -mypy==1.1.1 +mypy==1.5.1 types-python-dateutil types-requests types-six From d522a9e46f1bb627ef3013320cd8bf89589993ba Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 15:49:22 +0100 Subject: [PATCH 26/43] Require mypy # type: ignore specifies what is being ignored (#2880) --- mypy.ini | 1 + parsl/addresses.py | 2 +- parsl/monitoring/db_manager.py | 2 +- parsl/utils.py | 2 +- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/mypy.ini b/mypy.ini index 739e572870..22acd55173 100644 --- a/mypy.ini +++ b/mypy.ini @@ -6,6 +6,7 @@ plugins = sqlalchemy.ext.mypy.plugin # which is commonly done with manager IDs in the parsl # codebase. disable_error_code = str-bytes-safe +enable_error_code = ignore-without-code no_implicit_reexport = True warn_redundant_casts = True diff --git a/parsl/addresses.py b/parsl/addresses.py index e5a8b9ad61..b3d4b1fd88 100644 --- a/parsl/addresses.py +++ b/parsl/addresses.py @@ -13,7 +13,7 @@ try: import fcntl except ImportError: - fcntl = None # type: ignore + fcntl = None # type: ignore[assignment] import struct import typeguard import psutil diff --git a/parsl/monitoring/db_manager.py b/parsl/monitoring/db_manager.py index 1b33550d42..b25753c093 100644 --- a/parsl/monitoring/db_manager.py +++ b/parsl/monitoring/db_manager.py @@ -69,7 +69,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 + for mapper_registry in mapperlib._all_registries(): # type: ignore[attr-defined] all_mappers.update(mapper_registry.mappers) mapper_gen = ( mapper for mapper in all_mappers diff --git a/parsl/utils.py b/parsl/utils.py index f90c36de10..ef04490dbb 100644 --- a/parsl/utils.py +++ b/parsl/utils.py @@ -192,7 +192,7 @@ def __init__(self, first, second, third='three', fourth='fourth'): __max_width__ = 80 def __repr__(self) -> str: - init = self.__init__ # type: ignore + init = self.__init__ # type: ignore[misc] # This test looks for a single layer of wrapping performed by # functools.update_wrapper, commonly used in decorators. This will From b09b8dccd34f14502e8445cadb5d486eb9eac65b Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 16:25:58 +0100 Subject: [PATCH 27/43] Remove unreachable code when app_fu is None (#2881) The task record entry app_fu is never None: it is either missing (during construction) or a Future. So, this None check will never fire. Discovered by mypy 0.5.1 warn_unreachable --- parsl/dataflow/dflow.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index fc1b38effc..e51d6dd39a 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -1283,9 +1283,6 @@ def checkpoint(self, tasks: Optional[Sequence[TaskRecord]] = None) -> str: for task_record in checkpoint_queue: task_id = task_record['id'] - if task_record['app_fu'] is None: - continue - app_fu = task_record['app_fu'] if app_fu.done() and app_fu.exception() is None: From c60a59e9a5f33ca51b26059e3bd70aef29162132 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 16:56:07 +0100 Subject: [PATCH 28/43] Remove spurious docstring in htex initialize_scaling (#2882) The removed text looks like it was written as if for an abstract base class, not for a concrete implementation like htex. --- parsl/executors/high_throughput/executor.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index 69af77a86b..5a6561438c 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -305,10 +305,7 @@ def __init__(self, radio_mode = "htex" def initialize_scaling(self): - """ Compose the launch command and call the scale_out - - This should be implemented in the child classes to take care of - executor specific oddities. + """Compose the launch command and scale out the initial blocks. """ debug_opts = "--debug" if self.worker_debug else "" max_workers = "" if self.max_workers == float('inf') else "--max_workers={}".format(self.max_workers) From d0002ae09fa95dcf3e5fe13a0900ccf2e7b868cf Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 17:39:38 +0100 Subject: [PATCH 29/43] Use capital letter at start of log message (#2883) --- parsl/executors/high_throughput/executor.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parsl/executors/high_throughput/executor.py b/parsl/executors/high_throughput/executor.py index 5a6561438c..2ae6dab314 100644 --- a/parsl/executors/high_throughput/executor.py +++ b/parsl/executors/high_throughput/executor.py @@ -387,7 +387,7 @@ def _queue_management_worker(self): The `None` message is a die request. """ - logger.debug("queue management worker starting") + logger.debug("Queue management worker starting") while not self.bad_state_is_set: try: @@ -456,7 +456,7 @@ def _queue_management_worker(self): else: raise BadMessage("Message received with unknown type {}".format(msg['type'])) - logger.info("queue management worker finished") + logger.info("Queue management worker finished") def _start_local_interchange_process(self): """ Starts the interchange process locally From 4a684dd62887c4f658b56433a206dbdb55c0b28c Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 18 Sep 2023 20:04:43 +0100 Subject: [PATCH 30/43] Move scale_{in,out} from ParslExecutor to BlockProviderExecutor (#2885) Scale in/out behaviour is only for BlockProviderExecutors. Other fairly recent PRs have been consolidating that behaviour, and this PR is another one in that direction. The non-BlockProviderExecutors (threads and flux) had meaningless stub methods to make ABCMeta allow them to be instantiated. This PR removes those stubs. --- parsl/executors/base.py | 22 ---------------------- parsl/executors/flux/executor.py | 6 ------ parsl/executors/status_handling.py | 14 ++++++++++++++ parsl/executors/threads.py | 22 ---------------------- 4 files changed, 14 insertions(+), 50 deletions(-) diff --git a/parsl/executors/base.py b/parsl/executors/base.py index 579fd904af..f9fca513fe 100644 --- a/parsl/executors/base.py +++ b/parsl/executors/base.py @@ -73,28 +73,6 @@ def submit(self, func: Callable, resource_specification: Dict[str, Any], *args: """ pass - @abstractmethod - def scale_out(self, blocks: int) -> List[str]: - """Scale out method. - - :return: A list of block ids corresponding to the blocks that were added. - """ - pass - - @abstractmethod - def scale_in(self, blocks: int) -> List[str]: - """Scale in method. - - Cause the executor to reduce the number of blocks by count. - - We should have the scale in method simply take resource object - which will have the scaling methods, scale_in itself should be a coroutine, since - scaling tasks can be slow. - - :return: A list of block ids corresponding to the blocks that were removed. - """ - pass - @abstractmethod def shutdown(self) -> bool: """Shutdown the executor. diff --git a/parsl/executors/flux/executor.py b/parsl/executors/flux/executor.py index 6bb8d4fde0..2c4aada20f 100644 --- a/parsl/executors/flux/executor.py +++ b/parsl/executors/flux/executor.py @@ -297,12 +297,6 @@ def submit( ) return future - def scale_in(self, *args, **kwargs): - pass - - def scale_out(self): - pass - def _submit_wrapper( submission_queue: queue.Queue, stop_event: threading.Event, *args, **kwargs diff --git a/parsl/executors/status_handling.py b/parsl/executors/status_handling.py index 29070ae4d5..38f8d530b2 100644 --- a/parsl/executors/status_handling.py +++ b/parsl/executors/status_handling.py @@ -206,6 +206,20 @@ def scale_out(self, blocks: int = 1) -> List[str]: "Failed to start block {}: {}".format(block_id, ex)) return block_ids + @abstractmethod + def scale_in(self, blocks: int) -> List[str]: + """Scale in method. + + Cause the executor to reduce the number of blocks by count. + + We should have the scale in method simply take resource object + which will have the scaling methods, scale_in itself should be a coroutine, since + scaling tasks can be slow. + + :return: A list of block ids corresponding to the blocks that were removed. + """ + pass + def _launch_block(self, block_id: str) -> Any: launch_cmd = self._get_launch_command(block_id) job_name = f"parsl.{self.label}.block-{block_id}" diff --git a/parsl/executors/threads.py b/parsl/executors/threads.py index 234756b9ce..cb4c695cd6 100644 --- a/parsl/executors/threads.py +++ b/parsl/executors/threads.py @@ -59,28 +59,6 @@ def submit(self, func, resource_specification, *args, **kwargs): return self.executor.submit(func, *args, **kwargs) - def scale_out(self, workers=1): - """Scales out the number of active workers by 1. - - This method is notImplemented for threads and will raise the error if called. - - Raises: - NotImplemented exception - """ - - raise NotImplementedError - - def scale_in(self, blocks): - """Scale in the number of active blocks by specified amount. - - This method is not implemented for threads and will raise the error if called. - - Raises: - NotImplemented exception - """ - - raise NotImplementedError - def shutdown(self, block=True): """Shutdown the ThreadPool. The underlying concurrent.futures thread pool implementation will not terminate tasks that are being executed, because it From f71f050126fbd3b443e73ecd1c77c0f4013494b2 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 19 Sep 2023 08:38:44 +0100 Subject: [PATCH 31/43] Remove execute_wait stdout, stderr optionality (#2886) No in-codebase channels return None for stdout or stderr, and some cluster providers (at least pbspro and slurm) do not handle the case when stdout is None, instead assuming it will be a str. This PR makes those assumptions concrete. This PR will introduce API breakage for any externally implemented channels which return None here, although in practice because of the above, it's likely that use of those channels woudl already be broken. However, I am not aware of any externally implemented channels. Co-authored-by: Kevin Hunter Kesling --- parsl/channels/base.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/parsl/channels/base.py b/parsl/channels/base.py index d4ae835068..0069ba34ff 100644 --- a/parsl/channels/base.py +++ b/parsl/channels/base.py @@ -1,6 +1,6 @@ from abc import ABCMeta, abstractmethod, abstractproperty -from typing import Dict, Optional, Tuple +from typing import Dict, Tuple class Channel(metaclass=ABCMeta): @@ -38,7 +38,7 @@ class Channel(metaclass=ABCMeta): """ @abstractmethod - def execute_wait(self, cmd: str, walltime: int = 0, envs: Dict[str, str] = {}) -> Tuple[int, Optional[str], Optional[str]]: + def execute_wait(self, cmd: str, walltime: int = 0, envs: Dict[str, str] = {}) -> Tuple[int, str, str]: ''' Executes the cmd, with a defined walltime. Args: @@ -49,9 +49,7 @@ def execute_wait(self, cmd: str, walltime: int = 0, envs: Dict[str, str] = {}) - - envs (Dict[str, str]) : Environment variables to push to the remote side Returns: - - (exit_code, stdout, stderr) (int, optional string, optional string) - If the exit code is a failure code, the stdout and stderr return values - may be None. + - (exit_code, stdout, stderr) (int, string, string) ''' pass From 2b611f20d9db8a11197accb2dbbe9056419143e7 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 19 Sep 2023 09:38:28 +0100 Subject: [PATCH 32/43] Remove three dead assignments from local channel (#2887) These initial values will never be used and in the stdout/err case, can give the impression that stdout, stderr might be returned as None. There is no code path which uses these initial assignments as the function will raise an exception in the case that they are not all re-assigned later on. --- parsl/channels/local/local.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/parsl/channels/local/local.py b/parsl/channels/local/local.py index ee2a7a5088..e933f0687f 100644 --- a/parsl/channels/local/local.py +++ b/parsl/channels/local/local.py @@ -51,10 +51,6 @@ def execute_wait(self, cmd, walltime=None, envs={}): Raises: None. ''' - retcode = -1 - stdout = None - stderr = None - current_env = copy.deepcopy(self._envs) current_env.update(envs) From 4ba3df2e1d53591ff300cbe114e86b3f6a879038 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 19 Sep 2023 11:09:58 +0100 Subject: [PATCH 33/43] Add some more type annotations to parsl.jobs (#2888) These come from the benc-mypy branch --- parsl/jobs/error_handlers.py | 8 ++++---- parsl/jobs/job_status_poller.py | 8 ++++---- parsl/jobs/strategy.py | 11 ++++++----- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/parsl/jobs/error_handlers.py b/parsl/jobs/error_handlers.py index c35a87447d..e0b94dfc6c 100644 --- a/parsl/jobs/error_handlers.py +++ b/parsl/jobs/error_handlers.py @@ -1,17 +1,17 @@ from __future__ import annotations -from typing import Dict +from typing import Dict, Tuple import parsl.executors.status_handling as status_handling from parsl.jobs.states import JobStatus, JobState from parsl.jobs.errors import TooManyJobFailuresError -def noop_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): +def noop_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3) -> None: pass -def simple_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3): +def simple_error_handler(executor: status_handling.BlockProviderExecutor, status: Dict[str, JobStatus], threshold: int = 3) -> None: (total_jobs, failed_jobs) = _count_jobs(status) if hasattr(executor.provider, "init_blocks"): threshold = max(1, executor.provider.init_blocks) @@ -28,7 +28,7 @@ def windowed_error_handler(executor: status_handling.BlockProviderExecutor, stat executor.set_bad_state_and_fail_all(_get_error(status)) -def _count_jobs(status: Dict[str, JobStatus]): +def _count_jobs(status: Dict[str, JobStatus]) -> Tuple[int, int]: total = 0 failed = 0 for js in status.values(): diff --git a/parsl/jobs/job_status_poller.py b/parsl/jobs/job_status_poller.py index 44bc151e24..07fbc1cbe4 100644 --- a/parsl/jobs/job_status_poller.py +++ b/parsl/jobs/job_status_poller.py @@ -54,7 +54,7 @@ def poll(self, now: float) -> None: if delta_status: self.send_monitoring_info(delta_status) - def send_monitoring_info(self, status: Dict): + def send_monitoring_info(self, status: Dict) -> None: # Send monitoring info for HTEX when monitoring enabled if self.monitoring_enabled: msg = self._executor.create_monitoring_info(status) @@ -101,19 +101,19 @@ def __repr__(self) -> str: class JobStatusPoller(Timer): - def __init__(self, dfk: "parsl.dataflow.dflow.DataFlowKernel"): + def __init__(self, dfk: "parsl.dataflow.dflow.DataFlowKernel") -> None: self._poll_items = [] # type: List[PollItem] self.dfk = dfk self._strategy = Strategy(strategy=dfk.config.strategy, max_idletime=dfk.config.max_idletime) super().__init__(self.poll, interval=5, name="JobStatusPoller") - def poll(self): + def poll(self) -> None: self._update_state() self._run_error_handlers(self._poll_items) self._strategy.strategize(self._poll_items) - def _run_error_handlers(self, status: List[PollItem]): + def _run_error_handlers(self, status: List[PollItem]) -> None: for es in status: es.executor.handle_errors(es.status) diff --git a/parsl/jobs/strategy.py b/parsl/jobs/strategy.py index 032ba16d8c..beb5e50249 100644 --- a/parsl/jobs/strategy.py +++ b/parsl/jobs/strategy.py @@ -3,11 +3,12 @@ import time import math import warnings -from typing import Dict, List, Optional, TypedDict +from typing import Dict, List, Optional, Sequence, TypedDict import parsl.jobs.job_status_poller as jsp from parsl.executors import HighThroughputExecutor +from parsl.executors.base import ParslExecutor from parsl.executors.status_handling import BlockProviderExecutor from parsl.jobs.states import JobState from parsl.process_loggers import wrap_with_logs @@ -122,7 +123,7 @@ class Strategy: """ - def __init__(self, *, strategy: Optional[str], max_idletime: float): + def __init__(self, *, strategy: Optional[str], max_idletime: float) -> None: """Initialize strategy.""" self.executors: Dict[str, ExecutorState] self.executors = {} @@ -141,7 +142,7 @@ def __init__(self, *, strategy: Optional[str], max_idletime: float): logger.debug("Scaling strategy: {0}".format(strategy)) - def add_executors(self, executors): + def add_executors(self, executors: Sequence[ParslExecutor]) -> None: for executor in executors: self.executors[executor.label] = {'idle_since': None} @@ -150,10 +151,10 @@ def _strategy_noop(self, status: List[jsp.PollItem]) -> None: """ logger.debug("strategy_noop: doing nothing") - def _strategy_simple(self, status_list) -> None: + def _strategy_simple(self, status_list: List[jsp.PollItem]) -> None: self._general_strategy(status_list, strategy_type='simple') - def _strategy_htex_auto_scale(self, status_list) -> None: + def _strategy_htex_auto_scale(self, status_list: List[jsp.PollItem]) -> None: """HTEX specific auto scaling strategy This strategy works only for HTEX. This strategy will scale out by From a3203a4453bc640a9532087ac116e15381b7a599 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 19 Sep 2023 14:37:45 +0100 Subject: [PATCH 34/43] Replace [appname] with English text in bash app exceptions (#2889) Co-authored-by: Kevin Hunter Kesling --- parsl/app/bash.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parsl/app/bash.py b/parsl/app/bash.py index 5e0ca4237e..6de57b96f1 100644 --- a/parsl/app/bash.py +++ b/parsl/app/bash.py @@ -77,10 +77,10 @@ def open_std_fd(fdname): returncode = proc.returncode except subprocess.TimeoutExpired: - raise pe.AppTimeout("[{}] App exceeded walltime: {} seconds".format(func_name, timeout)) + raise pe.AppTimeout(f"App {func_name} exceeded walltime: {timeout} seconds") except Exception as e: - raise pe.AppException("[{}] App caught exception with returncode: {}".format(func_name, returncode), e) + raise pe.AppException(f"App {func_name} caught exception with returncode: {returncode}", e) if returncode != 0: raise pe.BashExitFailure(func_name, proc.returncode) @@ -95,7 +95,7 @@ def open_std_fd(fdname): missing.extend([outputfile]) if missing: - raise pe.MissingOutputs("[{}] Missing outputs".format(func_name), missing) + raise pe.MissingOutputs(f"Missing outputs from app {func_name}", missing) return returncode From d7f083072d80e0a07a0f26f5ccfa6a15f51b5961 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 21 Sep 2023 10:02:55 +0100 Subject: [PATCH 35/43] Document parsl-perf (#2891) --- docs/userguide/index.rst | 1 + docs/userguide/parsl_perf.rst | 53 +++++++++++++++++++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 docs/userguide/parsl_perf.rst diff --git a/docs/userguide/index.rst b/docs/userguide/index.rst index ae7df17450..21de9eb704 100644 --- a/docs/userguide/index.rst +++ b/docs/userguide/index.rst @@ -19,4 +19,5 @@ User guide joins usage_tracking plugins + parsl_perf performance diff --git a/docs/userguide/parsl_perf.rst b/docs/userguide/parsl_perf.rst new file mode 100644 index 0000000000..2ea1adb00f --- /dev/null +++ b/docs/userguide/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 + From c78926218a3bd4af45430f5ff04a0dc7beba4356 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Thu, 21 Sep 2023 10:39:03 +0100 Subject: [PATCH 36/43] Switch empty dfk() error from ConfigurationError to NoDataFlowKernelError (#2890) This is in support of https://github.com/Parsl/parsl/issues/2873#issuecomment-1726540856 --- parsl/dataflow/dflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/dataflow/dflow.py b/parsl/dataflow/dflow.py index e51d6dd39a..c42d536c70 100644 --- a/parsl/dataflow/dflow.py +++ b/parsl/dataflow/dflow.py @@ -1438,5 +1438,5 @@ def wait_for_current_tasks(cls) -> None: def dfk(cls) -> DataFlowKernel: """Return the currently-loaded DataFlowKernel.""" if cls._dfk is None: - raise ConfigurationError('Must first load config') + raise NoDataFlowKernelError('Must first load config') return cls._dfk From f8fb612b53f0d6293df7f288571a1cece5790524 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 3 Oct 2023 07:31:24 +0100 Subject: [PATCH 37/43] Make parsl-perf iterations have at least 1 task (#2896) Prior to this PR: In some startup situations, the estimated number of tasks for the next iteration can round down to 0, and then parsl-perf can sometimes enter an infinite loop without increasing the loop size: ==== Iteration 29463 ==== Will run 0 tasks to target 20.0 seconds runtime Submitting tasks / invoking apps All 0 tasks submitted ... waiting for completion Submission took 2.1457672119140625e-06 seconds = 0.0 tasks/second Runtime: 1.049041748046875e-05s vs target 20.0 Tasks per second: 0.0 --- parsl/benchmark/perf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/benchmark/perf.py b/parsl/benchmark/perf.py index 9325c24004..5740f2841d 100644 --- a/parsl/benchmark/perf.py +++ b/parsl/benchmark/perf.py @@ -62,7 +62,7 @@ def performance(*, resources: dict, target_t: float): print(f"Runtime: actual {delta_t:.3f}s vs target {target_t}s") print(f"Tasks per second: {rate:.3f}") - n = int(target_t * rate) + n = max(1, int(target_t * rate)) iteration += 1 From b32147f40ee0de76b20d9f4237aa3eaa4d3c7a88 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Tue, 3 Oct 2023 12:05:42 +0100 Subject: [PATCH 38/43] Fix flake8 error from upcoming Python 3.12 (#2893) In Python 3.12, f-strings vs flake8 has more whitespace checking, in line with whitespace checking in other Python expressions, as part of a broader rationalisation of f-string implementation: https://docs.python.org/3.12/whatsnew/3.12.html#whatsnew312-pep701 --- parsl/benchmark/perf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parsl/benchmark/perf.py b/parsl/benchmark/perf.py index 5740f2841d..d92b9105ad 100644 --- a/parsl/benchmark/perf.py +++ b/parsl/benchmark/perf.py @@ -48,7 +48,7 @@ def performance(*, resources: dict, target_t: float): submitted_t = time.time() print(f"All {n} tasks submitted ... waiting for completion") - print(f"Submission took {submitted_t - start_t:.3f} seconds = {n/(submitted_t - start_t):.3f} tasks/second") + print(f"Submission took {submitted_t - start_t:.3f} seconds = {n / (submitted_t - start_t):.3f} tasks/second") for f in concurrent.futures.as_completed(fs): assert f.result() == 7 From 95231e27d59e722352fa66705de52a2c2419bcdb Mon Sep 17 00:00:00 2001 From: Sophie Bui Date: Tue, 3 Oct 2023 15:23:23 -0500 Subject: [PATCH 39/43] Remove historical info and update roadmap (#2897) Updated the roadmap using content/copy from our working Google Doc --- docs/devguide/roadmap.rst | 174 ++++++++++---------------------------- 1 file changed, 45 insertions(+), 129 deletions(-) diff --git a/docs/devguide/roadmap.rst b/docs/devguide/roadmap.rst index 1c146c49d5..a1fe8e44e0 100644 --- a/docs/devguide/roadmap.rst +++ b/docs/devguide/roadmap.rst @@ -1,135 +1,51 @@ -Historical: Roadmap -=================== +Roadmap +======= -.. note:: - This roadmap has not been current since version 0.9.0 in 2019, and does - not reflect changes in project direction then. For this reason, this - roadmap is marked as historical. +**OVERVIEW** -Before diving into the roadmap, a quick retrospective look at the evolution of workflow -solutions that came before Parsl from the workflows group at UChicago and Argonne National Laboratory. +While we follow best practices in software development processes (e.g., CI, flake8, code review), there are opportunities to make our code more maintainable and accessible. This roadmap, written in the fall of 2023, covers our major activities planned through 2025 to increase efficiency, productivity, user experience, and community building. -.. image:: ../images/swift-e-timeline_trimmed.png - - -Sufficient capabilities to use Parsl in many common situations already exist. This document indicates where Parsl is going; -it contains a list of features that Parsl has or will have. Features that exist today are marked in bold, with the release -in which they were added marked for releases since 0.3.0. Help in providing any of the yet-to-be-developed capabilities is welcome. - -Features in preparation are documented via Github +Features and improvements are documented via GitHub `issues `_ and `pull requests `_. -Core Functionality ---------------------- - -* **Parsl has the ability to execute standard python code and to asynchronously execute tasks, called Apps.** - * **Any Python function annotated with "@App" is an App.** - * **Apps can be Python functions or bash scripts that wrap external applications.** -* **Asynchronous tasks return futures, which other tasks can use as inputs.** - * **This builds an implicit data flow graph.** -* **Asynchronous tasks can execute locally on threads or as separate processes.** -* **Asynchronous tasks can execute on a remote resource.** - * **libsubmit (to be renamed) provides this functionality.** - * **A shared filesystem is assumed; data staging (of files) is not yet supported.** -* **The Data Flow Kernel (DFK) schedules Parsl task execution (based on dataflow).** -* **Class-based config definition (v0.6.0)** -* **Singleton config, and separate DFK from app definitions (v0.6.0)** -* Class-based app definition - -Data management ---------------- - -* **File abstraction to support representation of local and remote files.** -* **Support for a variety of common data access protocols (e.g., FTP, HTTP, Globus) (v0.6.0)**. -* **Input/output staging models that support transparent movement of data from source to a location on which it is accessible for compute. This includes staging to/from the client (script execution location) and worker node (v0.6.0)**. -* Support for creation of a sandbox and execution within the sandbox. -* Multi-site support including transparent movement between sites. -* **Support for systems without a shared file system (point-to-point staging). (Partial support in v0.9.0)** -* Support for data caching at multiple levels and across sites. - -TODO: Add diagram for staging - - -Execution core and parallelism (DFK) ------------------------------------- - -* **Support for application and data futures within scripts.** -* **Internal (dynamically created/updated) task/data dependency graph that enables asynchronous execution ordered by data dependencies and throttled by resource limits.** -* **Well-defined state transition model for task lifecycle. (v0.5.0)** -* Add data staging to task state transition model. -* **More efficient algorithms for managing dependency resolution. (v0.7.0)** -* Scheduling and allocation algorithms that determine job placement based on job and data requirements (including deadlines) as well as site capabilities. -* **Directing jobs to a specific set of sites.(v0.4.0)** -* **Logic to manage (provision, resize) execution resource block based on job requirements, and running multiple tasks per resource block (v0.4.0).** -* **Retry logic to support recovery and fault tolerance** -* **Workflow level checkpointing and restart (v0.4.0)** -* **Transition away from IPP to in-house executors (HighThroughputExecutor and ExtremeScaleExecutor v0.7.0)** - -Resource provisioning and execution ------------------------------------ - -* **Uniform abstraction for execution resources (to support resource provisioning, job submission, allocation management) on cluster, cloud, and supercomputing resources** -* **Support for different execution models on any execution provider (e.g., pilot jobs using Ipython parallel on clusters and extreme-scale execution using Swift/T on supercomputers)** - * **Slurm** - * **HTCondor** - * **Cobalt** - * **GridEngine** - * **PBS/Torque** - * **AWS** - * **GoogleCloud** - * **Azure** - * **Nova/OpenStack/Jetstream (partial support)** - * **Kubernetes (v0.6.0)** -* **Support for launcher mechanisms** - * **srun** - * **aprun (Complete support 0.6.0)** - * **Various MPI launch mechanisms (Mpiexec, mpirun..)** -* **Support for remote execution using SSH (from v0.3.0)and OAuth-based authentication (from v0.9.0)** -* **Utilizing multiple sites for a single script’s execution (v0.4.0)** -* Cloud-hosted site configuration repository that stores configurations for resource authentication, data staging, and job submission endpoints -* **IPP workers to support multiple threads of execution per node. (v0.7.0 adds support via replacement executors)** -* Smarter serialization with caching frequently used objects. -* **Support for user-defined containers as Parsl apps and orchestration of workflows comprised of containers (v0.5.0)** - * **Docker (locally)** - * Shifter (NERSC, Blue Waters) - * Singularity (ALCF) - -Visualization, debugging, fault tolerance ------------------------------------------ - -* **Support for exception handling**. -* **Interface for accessing real-time state (v0.6.0)**. -* **Visualization library that enables users to introspect graph, task, and data dependencies, as well as observe state of executed/executing tasks (from v0.9.0)** -* Integration of visualization into jupyter -* Support for visualizing dead/dying parts of the task graph and retrying with updates to the task. -* **Retry model to selectively re-execute only the failed branches of a workflow graph** -* **Fault tolerance support for individual task execution** -* **Support for saving monitoring information to local DB (sqlite) and remote DB (elasticsearch) (v0.6.0 and v0.7.0)** - -Authentication and authorization --------------------------------- - -* **Seamless authentication using OAuth-based methods within Parsl scripts (e.g., native app grants) (v0.6.0)** -* Support for arbitrary identity providers and pass through to execution resources -* Support for transparent/scoped access to external services **(e.g., Globus transfer) (v0.6.0)** - -Ecosystem ---------- - -* Support for CWL, ability to execute CWL workflows and use CWL app descriptions -* Creation of library of Parsl apps and workflows -* Provenance capture/export in standard formats -* Automatic metrics capture and reporting to understand Parsl usage -* **Anonymous Usage Tracking (v0.4.0)** - -Documentation / Tutorials: --------------------------- - -* **Documentation about Parsl and its features** -* **Documentation about supported sites (v0.6.0)** -* **Self-guided Jupyter notebook tutorials on Parsl features** -* **Hands-on tutorial suitable for webinars and meetings** - - - +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. +* **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 +----------------------------- + +* **Enhanced MPI Support**: Extend Parsl’s MPI model with MPI apps and runtime support capable of running MPI apps in different environments (MPI flavor and launcher). +* **Serialization Configuration**: Enable users to select what serialization methods are used and enable users to supply their own serializer. +* **PSI/J integration**: Integrate PSI/J as a common interface for schedulers. +* **Internal Concurrency Model**: Revisit and rearchitect the concurrency model to reduce areas that are not well understood and reduce the likelihood of errors. +* **Common Model for Errors**: Make Parsl errors self-describing and understandable by users. +* **Plug-in Model for External Components**: Extend Parsl to implement interfaces defined above. +* **User Configuration Validation Tool**: Provide tooling to help users configure Parsl and diagnose and resolve errors. +* **Anonymized Usage Tracking**: Usage tracking is crucial for our data-oriented approach to understand the adoption of Parsl, which components are used, and where errors occur. This allows us to prioritize investment in components, progress components through the maturity levels, and identify bugs. Revisit prior usage tracking and develop a service that enables users to control tracking information. +* **Support for Globus Compute**: Enable execution of Parsl tasks using Globus Compute as an executor. +* **Update Globus Data Management**: Update Globus integration to use the new Globus Connect v5 model (i.e., needing specific scopes for individual endpoints). +* **Performance Measurement**: Improve ability to measure performance metrics and report to users. +* **Enhanced Debugging**: Application-level `logging `_ to understand app execution. + +Tutorials, Training, and User Support +------------------------------------- + +* **Configuration and Debugging**: Tutorials showing how to configure Parsl for different resources and debug execution. +* **Functional Serialization 101**: Tutorial describing how serialization works and how you can integrate custom serializers. +* **ProxyStore Data Management**: Tutorial showing how you can use ProxyStore to manage data for both inter and intra-site scenarios. +* **Open Dev Calls on Zoom**: The internal core team holds an open dev call/office hours every other Thursday to help users troubleshoot issues, present and share their work, connect with each other, and provide community updates. +* **Project Documentation**: is maintained and updated in `Read the Docs `_. + +Longer-term Objectives +---------------------- + +* **Globus Compute Integration**: Once Globus Compute supports multi-tenancy, Parsl will be able to use it to run remote tasks on initially one and then later multiple resources. +* **Multi-System Optimization**: Once Globus Compute integration is complete, it is best to use multiple systems for multiple tasks as part of a single workflow. +* **HPC Checkpointing and Job Migration**: As new resources become available, HPC tasks will be able to be checkpointed and moved to the system with more resources. From bd4175414c91a90694ad17fb96eb435da9093af8 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Wed, 4 Oct 2023 13:51:59 +0100 Subject: [PATCH 40/43] Do not return a value from a pytest case, to fix a deprecation warning (#2898) This fixes this pytest warning: PytestReturnNotNoneWarning: Expected None, but test_bash_apps/test_error_codes.py::test_div_0 returned True, which will be an error in a future version of pytest. Did you mean to use `assert` instead of `return`? --- parsl/tests/test_bash_apps/test_error_codes.py | 1 - 1 file changed, 1 deletion(-) diff --git a/parsl/tests/test_bash_apps/test_error_codes.py b/parsl/tests/test_bash_apps/test_error_codes.py index 38b2b9533d..5060655076 100644 --- a/parsl/tests/test_bash_apps/test_error_codes.py +++ b/parsl/tests/test_bash_apps/test_error_codes.py @@ -74,7 +74,6 @@ def test_div_0(test_fn=div_0): print(os.listdir('.')) os.remove('std.err') os.remove('std.out') - return True @pytest.mark.issue363 From d155786c0d3d817bcf3b0979d1017cdedd8839c8 Mon Sep 17 00:00:00 2001 From: "Andrew S. Rosen" Date: Sun, 8 Oct 2023 23:43:10 -0700 Subject: [PATCH 41/43] Make `max_threads` type `Optional[int]` in `ThreadPoolExecutor` (#2902) The ThreadPoolExecutor forces the max_threads kwarg to be of type int. However, since max_threads inevitably makes its way into concurrent.futures.ThreadPoolExecutor, which can take max_workers=None, I have modified the type hint to be Optional[int]. --- parsl/executors/threads.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parsl/executors/threads.py b/parsl/executors/threads.py index cb4c695cd6..cdd44637f2 100644 --- a/parsl/executors/threads.py +++ b/parsl/executors/threads.py @@ -18,7 +18,7 @@ class ThreadPoolExecutor(ParslExecutor, RepresentationMixin): Parameters ---------- - max_threads : int + max_threads : Optional[int] Number of threads. Default is 2. thread_name_prefix : string Thread name prefix @@ -27,7 +27,7 @@ class ThreadPoolExecutor(ParslExecutor, RepresentationMixin): """ @typeguard.typechecked - def __init__(self, label: str = 'threads', max_threads: int = 2, + def __init__(self, label: str = 'threads', max_threads: Optional[int] = 2, thread_name_prefix: str = '', storage_access: Optional[List[Staging]] = None, working_dir: Optional[str] = None): ParslExecutor.__init__(self) From 3badf9d1a0eda4c291d75963567ecd663b6b7089 Mon Sep 17 00:00:00 2001 From: Ben Clifford Date: Mon, 9 Oct 2023 09:26:24 +0200 Subject: [PATCH 42/43] Remove duplicated test that gives a deprecation warning (#2899) This test tests that multiple apps can be submitted and then return. This is tested better in, for example, test_python_apps/test_mapred.py This test has started to raise a deprecation warning due to returning a value (rather than returning None by falling off the end). Removing it seems to me tidier than fixing that warning. --- parsl/tests/test_docs/test_workflow3.py | 23 ----------------------- 1 file changed, 23 deletions(-) delete mode 100644 parsl/tests/test_docs/test_workflow3.py diff --git a/parsl/tests/test_docs/test_workflow3.py b/parsl/tests/test_docs/test_workflow3.py deleted file mode 100644 index 7ed52c8cb3..0000000000 --- a/parsl/tests/test_docs/test_workflow3.py +++ /dev/null @@ -1,23 +0,0 @@ -import parsl - -from parsl.app.app import python_app -from parsl.tests.configs.local_threads import config - - -@python_app -def generate(limit): - from random import randint - """Generate a random integer and return it""" - return randint(1, limit) - - -def test_parallel_for(N=2): - """Test parallel workflows from docs on Composing workflows - """ - rand_nums = [] - for i in range(1, 5): - rand_nums.append(generate(i)) - - # wait for all apps to finish and collect the results - outputs = [i.result() for i in rand_nums] - return outputs From 6b963ba8b099891fe4384c8fcb317fcbbc26a8f3 Mon Sep 17 00:00:00 2001 From: Thanh Son Phung Date: Mon, 9 Oct 2023 14:48:28 -0400 Subject: [PATCH 43/43] TaskVineExecutor: Improve serverless (#2894) This PR brings changes that addresses concerns in PR #2849. * Added and edited comments and debug messages so they are accurate or more meaningful. * Fixed the issue where function and arguments are serialized twice. Now they are serialized once. * Added stronger type hints and checking in several places. * Added a default function execution mode option in the Executor interface to support for applications that want to exclusively run functions in either mode. Users can still override this global option by specifying the execution mode at the function level via parsl_resource_specification. * Use multiprocessing.Event to signal the stopping condition instead of Value. * Other minor fixes. --- .../executors/taskvine/exec_parsl_function.py | 45 ++++++++---------- parsl/executors/taskvine/executor.py | 46 ++++++++++--------- parsl/executors/taskvine/factory.py | 8 +--- parsl/executors/taskvine/manager.py | 12 +++-- 4 files changed, 54 insertions(+), 57 deletions(-) diff --git a/parsl/executors/taskvine/exec_parsl_function.py b/parsl/executors/taskvine/exec_parsl_function.py index 0a408e1c03..59cca96bab 100644 --- a/parsl/executors/taskvine/exec_parsl_function.py +++ b/parsl/executors/taskvine/exec_parsl_function.py @@ -1,13 +1,15 @@ -from parsl.app.errors import RemoteExceptionWrapper -from parsl.data_provider.files import File -from parsl.utils import get_std_fname_mode import traceback import sys + import pickle +from parsl.app.errors import RemoteExceptionWrapper +from parsl.data_provider.files import File +from parsl.utils import get_std_fname_mode +from parsl.serialize import deserialize -# This scripts executes a parsl function which is pickled in 3 files: +# This scripts executes a parsl function which is pickled in 4 files: # -# exec_parsl_function.py map_file function_file result_file +# exec_parsl_function.py map_file function_file argument_file result_file # # map_file: Contains a pickled dictionary that indicates which local_paths the # parsl Files should take. @@ -15,6 +17,8 @@ # function_file: Contains a pickle parsl function. Function might be serialized in advance. # See @parsl.serialize.concretes.py # +# argument_file: Contains the serialized arguments to the function call. +# # result_file: A file path, whose content will contain the result of the function, including any # exception generated. Exceptions will be wrapped with RemoteExceptionWrapper. # @@ -26,12 +30,6 @@ # -def load_pickled_file(filename: str): - """ Load a pickled file and return its pickled object.""" - with open(filename, "rb") as f_in: - return pickle.load(f_in) - - def dump_result_to_file(result_file: str, result_package): """ Dump a result to the given result file.""" with open(result_file, "wb") as f_out: @@ -78,9 +76,9 @@ def remap_all_files(mapping, fn_args, fn_kwargs): remap_location(mapping, maybe_file) -def unpack_object(serialized_obj, user_namespace): - from parsl.serialize import deserialize - obj = deserialize(serialized_obj) +def unpack_object_from_file(path): + with open(path, 'rb') as f: + obj = deserialize(f.read()) return obj @@ -115,26 +113,23 @@ def encode_byte_code_function(user_namespace, fn, fn_name, args_name, kwargs_nam def load_function(map_file, function_file, argument_file): # Decodes the function and its file arguments to be executed into # function_code, and updates a user namespace with the function name and - # the variable named result_name. When the function is executed, its result + # the variable named `result_name`. When the function is executed, its result # will be stored in this variable in the user namespace. # Returns (namespace, function_code, result_name) - # Create the namespace to isolate the function execution. - user_ns = locals() - user_ns.update({'__builtins__': __builtins__}) - - packed_function = load_pickled_file(function_file) - packed_argument = load_pickled_file(argument_file) - - fn = unpack_object(packed_function, user_ns) - args_dict = unpack_object(packed_argument, user_ns) + fn = unpack_object_from_file(function_file) + args_dict = unpack_object_from_file(argument_file) fn_args = args_dict['args'] fn_kwargs = args_dict['kwargs'] fn_name = 'parsl_tmp_func_name' - mapping = load_pickled_file(map_file) + mapping = unpack_object_from_file(map_file) remap_all_files(mapping, fn_args, fn_kwargs) + # Create the namespace to isolate the function execution. + user_ns = locals() + user_ns.update({'__builtins__': __builtins__}) + (code, result_name) = encode_function(user_ns, fn, fn_name, fn_args, fn_kwargs) return (user_ns, code, result_name) diff --git a/parsl/executors/taskvine/executor.py b/parsl/executors/taskvine/executor.py index 3d4669707a..a6ce3987bc 100644 --- a/parsl/executors/taskvine/executor.py +++ b/parsl/executors/taskvine/executor.py @@ -11,15 +11,13 @@ import hashlib import subprocess import os -import pickle import queue import inspect import shutil import itertools import uuid -from ctypes import c_bool from concurrent.futures import Future -from typing import List, Optional, Union +from typing import List, Optional, Union, Literal # Import Parsl constructs import parsl.utils as putils @@ -71,12 +69,17 @@ class TaskVineExecutor(BlockProviderExecutor, putils.RepresentationMixin): with respect to other executors. Default is "TaskVineExecutor". - worker_launch_method: str + worker_launch_method: Union[Literal['provider'], Literal['factory'], Literal['manual']] Choose to use Parsl provider, TaskVine factory, or manual user-provided workers to scale workers. Options are among {'provider', 'factory', 'manual'}. Default is 'factory'. + function_exec_mode: Union[Literal['regular'], Literal['serverless']] + Choose to execute functions with a regular fresh python process or a + pre-warmed forked python process. + Default is 'regular'. + manager_config: TaskVineManagerConfig Configuration for the TaskVine manager. Default @@ -98,7 +101,8 @@ class TaskVineExecutor(BlockProviderExecutor, putils.RepresentationMixin): @typeguard.typechecked def __init__(self, label: str = "TaskVineExecutor", - worker_launch_method: str = 'factory', + worker_launch_method: Union[Literal['provider'], Literal['factory'], Literal['manual']] = 'factory', + function_exec_mode: Union[Literal['regular'], Literal['serverless']] = 'regular', manager_config: TaskVineManagerConfig = TaskVineManagerConfig(), factory_config: TaskVineFactoryConfig = TaskVineFactoryConfig(), provider: Optional[ExecutionProvider] = LocalProvider(init_blocks=1), @@ -107,9 +111,6 @@ def __init__(self, # Set worker launch option for this executor if worker_launch_method == 'factory' or worker_launch_method == 'manual': provider = None - elif worker_launch_method != 'provider': - raise ExecutorError(self, "Worker launch option '{worker_launch_method}' \ - is not supported.") # Initialize the parent class with the execution provider and block error handling enabled. # If provider is None, then no worker is launched via the provider method. @@ -126,6 +127,7 @@ def __init__(self, # Executor configurations self.label = label self.worker_launch_method = worker_launch_method + self.function_exec_mode = function_exec_mode self.manager_config = manager_config self.factory_config = factory_config self.storage_access = storage_access @@ -136,8 +138,8 @@ def __init__(self, # Queue to send finished tasks from TaskVine manager process to TaskVine executor process self._finished_task_queue: multiprocessing.Queue = multiprocessing.Queue() - # Value to signal whether the manager and factory processes should stop running - self._should_stop = multiprocessing.Value(c_bool, False) + # Event to signal whether the manager and factory processes should stop running + self._should_stop = multiprocessing.Event() # TaskVine manager process self._submit_process = None @@ -267,10 +269,11 @@ def start(self): # Begin work self._submit_process.start() - # Run worker scaler either with Parsl provider or TaskVine factory + # Run worker scaler either with Parsl provider or TaskVine factory. + # Skip if workers are launched manually. if self.worker_launch_method == 'factory': self._factory_process.start() - else: + elif self.worker_launch_method == 'provider': self.initialize_scaling() self._collector_thread.start() @@ -314,7 +317,7 @@ def submit(self, func, resource_specification, *args, **kwargs): logger.debug(f'Got resource specification: {resource_specification}') # Default execution mode of apps is regular - exec_mode = resource_specification.get('exec_mode', 'regular') + exec_mode = resource_specification.get('exec_mode', self.function_exec_mode) # Detect resources and features of a submitted Parsl app cores = None @@ -391,9 +394,9 @@ def submit(self, func, resource_specification, *args, **kwargs): and result to be found at: {}".format(executor_task_id, function_file, argument_file, result_file)) # Serialize function object and arguments, separately - self._serialize_object(function_file, func) + self._serialize_object_to_file(function_file, func) args_dict = {'args': args, 'kwargs': kwargs} - self._serialize_object(argument_file, args_dict) + self._serialize_object_to_file(argument_file, args_dict) # Construct the map file of local filenames at worker self._construct_map_file(map_file, input_files, output_files) @@ -466,11 +469,13 @@ def _patch_providers(self): if self.project_password_file: self.provider.transfer_input_files.append(self.project_password_file) - def _serialize_object(self, path, obj): + def _serialize_object_to_file(self, path, obj): """Takes any object and serializes it to the file path.""" serialized_obj = serialize(obj, buffer_threshold=1024 * 1024) with open(path, 'wb') as f_out: - pickle.dump(serialized_obj, f_out) + written = 0 + while written < len(serialized_obj): + written += f_out.write(serialized_obj[written:]) def _construct_map_file(self, map_file, input_files, output_files): """ Map local filepath of parsl files to the filenames at the execution worker. @@ -485,8 +490,7 @@ def _construct_map_file(self, map_file, input_files, output_files): else: remote_name = local_name file_translation_map[local_name] = remote_name - with open(map_file, "wb") as f_out: - pickle.dump(file_translation_map, f_out) + self._serialize_object_to_file(map_file, file_translation_map) def _register_file(self, parsl_file): """Generates a tuple (parsl_file.filepath, stage, cache) to give to @@ -592,7 +596,7 @@ def shutdown(self, *args, **kwargs): collector thread, which shuts down the TaskVine system submission. """ logger.debug("TaskVine shutdown started") - self._should_stop.value = True + self._should_stop.set() # Remove the workers that are still going kill_ids = [self.blocks[block] for block in self.blocks.keys()] @@ -618,7 +622,7 @@ def _collect_taskvine_results(self): """ logger.debug("Starting Collector Thread") try: - while not self._should_stop.value: + while not self._should_stop.is_set(): if not self._submit_process.is_alive(): raise ExecutorError(self, "taskvine Submit Process is not alive") diff --git a/parsl/executors/taskvine/factory.py b/parsl/executors/taskvine/factory.py index 1967b89974..24f74f3fe3 100644 --- a/parsl/executors/taskvine/factory.py +++ b/parsl/executors/taskvine/factory.py @@ -1,5 +1,4 @@ import logging -import time from parsl.process_loggers import wrap_with_logs from parsl.executors.taskvine.errors import TaskVineFactoryFailure @@ -54,11 +53,8 @@ def _taskvine_factory(should_stop, factory_config): if factory_config.batch_options: factory.batch_options = factory_config.batch_options - # setup factory context and sleep for a second in every loop to - # avoid wasting CPU + # run factory through Python context and wait for signal to stop. with factory: - while not should_stop.value: - time.sleep(1) + should_stop.wait() logger.debug("Exiting TaskVine factory process") - return 0 diff --git a/parsl/executors/taskvine/manager.py b/parsl/executors/taskvine/manager.py index 3394ffce9e..2351a0a301 100644 --- a/parsl/executors/taskvine/manager.py +++ b/parsl/executors/taskvine/manager.py @@ -106,6 +106,8 @@ def _prepare_environment_regular(m, manager_config, t, task, poncho_env_to_file, env_tarball = str(uuid.uuid4()) + '.tar.gz' logger.debug(f'Creating a poncho environment at {env_tarball} from conda environment {manager_config.env_pack}') subprocess.run([poncho_create_script, manager_config.env_pack, env_tarball], stdout=subprocess.DEVNULL, check=True) + else: + env_tarball = manager_config.env_pack poncho_env_file = m.declare_poncho(env_tarball, cache=True, peer_transfer=True) poncho_env_to_file[manager_config.env_pack] = poncho_env_file else: @@ -194,15 +196,15 @@ def _taskvine_submit_wait(ready_task_queue=None, logger.debug("Entering main loop of TaskVine manager") - while not should_stop.value: - # Monitor the task queue + while not should_stop.is_set(): + # Check if executor process is still running ppid = os.getppid() if ppid != orig_ppid: - logger.debug("new Process") + logger.debug("Executor process is detected to have exited. Exiting..") break # Submit tasks - while ready_task_queue.qsize() > 0 and not should_stop.value: + while ready_task_queue.qsize() > 0 and not should_stop.is_set(): # Obtain task from ready_task_queue try: task = ready_task_queue.get(timeout=1) @@ -377,7 +379,7 @@ def _taskvine_submit_wait(ready_task_queue=None, # If the queue is not empty wait on the TaskVine queue for a task task_found = True if not m.empty(): - while task_found and not should_stop.value: + while task_found and not should_stop.is_set(): # Obtain the task from the queue t = m.wait(1) if t is None: