From 59e4be8679e20f3e0c05cc7ac59c43aa2b9619cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 08:34:00 +0200 Subject: [PATCH 01/14] refactor internal interfaces --- executorlib/base/executor.py | 8 +-- .../cache/{executor.py => task_scheduler.py} | 6 +-- executorlib/interactive/blockallocation.py | 8 +-- executorlib/interactive/dependency.py | 10 ++-- executorlib/interactive/onetoone.py | 8 +-- executorlib/interfaces/flux.py | 24 ++++----- executorlib/interfaces/single.py | 18 +++---- executorlib/interfaces/slurm.py | 24 ++++----- notebooks/4-developer.ipynb | 32 +++++------ tests/test_cache_fileexecutor_mpi.py | 4 +- tests/test_cache_fileexecutor_serial.py | 12 ++--- tests/test_fluxpythonspawner.py | 12 ++--- tests/test_interactive_dependencies.py | 10 ++-- tests/test_mpiexecspawner.py | 54 +++++++++---------- 14 files changed, 115 insertions(+), 115 deletions(-) rename executorlib/cache/{executor.py => task_scheduler.py} (97%) diff --git a/executorlib/base/executor.py b/executorlib/base/executor.py index 59a42322..0b946856 100644 --- a/executorlib/base/executor.py +++ b/executorlib/base/executor.py @@ -14,7 +14,7 @@ from executorlib.standalone.serialize import cloudpickle_register -class ExecutorBase(FutureExecutor): +class TaskSchedulerBase(FutureExecutor): """ Base class for the executor. @@ -181,15 +181,15 @@ def __del__(self): self.shutdown(wait=False) -class ExecutorInterface(FutureExecutor): +class ExecutorBase(FutureExecutor): """ Interface class for the executor. Args: - executor (ExecutorBase): internal executor + executor (TaskSchedulerBase): internal executor """ - def __init__(self, executor: ExecutorBase): + def __init__(self, executor: TaskSchedulerBase): self._task_scheduler = executor @property diff --git a/executorlib/cache/executor.py b/executorlib/cache/task_scheduler.py similarity index 97% rename from executorlib/cache/executor.py rename to executorlib/cache/task_scheduler.py index 9f9582c3..c8d4c900 100644 --- a/executorlib/cache/executor.py +++ b/executorlib/cache/task_scheduler.py @@ -2,7 +2,7 @@ from threading import Thread from typing import Callable, Optional -from executorlib.base.executor import ExecutorBase +from executorlib.base.executor import TaskSchedulerBase from executorlib.cache.shared import execute_tasks_h5 from executorlib.cache.subprocess_spawner import ( execute_in_subprocess, @@ -24,7 +24,7 @@ execute_with_pysqa = execute_in_subprocess # type: ignore -class FileExecutor(ExecutorBase): +class FileTaskScheduler(TaskSchedulerBase): def __init__( self, cache_directory: str = "cache", @@ -113,7 +113,7 @@ def create_file_executor( check_executor(executor=flux_executor) check_nested_flux_executor(nested_flux_executor=flux_executor_nesting) check_flux_log_files(flux_log_files=flux_log_files) - return FileExecutor( + return FileTaskScheduler( cache_directory=cache_directory, resource_dict=resource_dict, pysqa_config_directory=pysqa_config_directory, diff --git a/executorlib/interactive/blockallocation.py b/executorlib/interactive/blockallocation.py index 9e6500b7..a2f00493 100644 --- a/executorlib/interactive/blockallocation.py +++ b/executorlib/interactive/blockallocation.py @@ -3,7 +3,7 @@ from threading import Thread from typing import Callable, Optional -from executorlib.base.executor import ExecutorBase, cancel_items_in_queue +from executorlib.base.executor import TaskSchedulerBase, cancel_items_in_queue from executorlib.interactive.shared import execute_tasks from executorlib.standalone.inputcheck import ( check_resource_dict, @@ -12,7 +12,7 @@ from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner -class BlockAllocationExecutor(ExecutorBase): +class BlockAllocationTaskScheduler(TaskSchedulerBase): """ The executorlib.interactive.executor.InteractiveExecutor leverages the exeutorlib interfaces to distribute python tasks on a workstation or inside a queuing system allocation. In contrast to the mpi4py.futures.MPIPoolExecutor the @@ -28,7 +28,7 @@ class BlockAllocationExecutor(ExecutorBase): Examples: >>> import numpy as np - >>> from executorlib.interactive.blockallocation import BlockAllocationExecutor + >>> from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -39,7 +39,7 @@ class BlockAllocationExecutor(ExecutorBase): >>> def init_k(): >>> return {"k": 3} >>> - >>> with BlockAllocationExecutor(max_workers=2, executor_kwargs={"init_function": init_k}) as p: + >>> with BlockAllocationTaskScheduler(max_workers=2, executor_kwargs={"init_function": init_k}) as p: >>> fs = p.submit(calc, 2, j=4) >>> print(fs.result()) [(array([2, 4, 3]), 2, 0), (array([2, 4, 3]), 2, 1)] diff --git a/executorlib/interactive/dependency.py b/executorlib/interactive/dependency.py index 5f27ff4a..0dc01eb5 100644 --- a/executorlib/interactive/dependency.py +++ b/executorlib/interactive/dependency.py @@ -4,7 +4,7 @@ from time import sleep from typing import Any, Callable, Optional -from executorlib.base.executor import ExecutorBase +from executorlib.base.executor import TaskSchedulerBase from executorlib.standalone.interactive.arguments import ( check_exception_was_raised, get_exception_lst, @@ -18,7 +18,7 @@ ) -class DependencyExecutor(ExecutorBase): +class DependencyTaskScheduler(TaskSchedulerBase): """ ExecutorWithDependencies is a class that extends ExecutorBase and provides functionality for executing tasks with dependencies. @@ -38,7 +38,7 @@ class DependencyExecutor(ExecutorBase): def __init__( self, - executor: ExecutorBase, + executor: TaskSchedulerBase, max_cores: Optional[int] = None, refresh_rate: float = 0.01, plot_dependency_graph: bool = False, @@ -188,7 +188,7 @@ def __exit__( def _execute_tasks_with_dependencies( future_queue: queue.Queue, executor_queue: queue.Queue, - executor: ExecutorBase, + executor: TaskSchedulerBase, refresh_rate: float = 0.01, ): """ @@ -198,7 +198,7 @@ def _execute_tasks_with_dependencies( Args: future_queue (Queue): Queue for receiving new tasks. executor_queue (Queue): Queue for the internal executor. - executor (ExecutorBase): Executor to execute the tasks with after the dependencies are resolved. + executor (TaskSchedulerBase): Executor to execute the tasks with after the dependencies are resolved. refresh_rate (float): Set the refresh rate in seconds, how frequently the input queue is checked. """ wait_lst = [] diff --git a/executorlib/interactive/onetoone.py b/executorlib/interactive/onetoone.py index ca38ca52..822660c5 100644 --- a/executorlib/interactive/onetoone.py +++ b/executorlib/interactive/onetoone.py @@ -2,12 +2,12 @@ from threading import Thread from typing import Optional -from executorlib.base.executor import ExecutorBase +from executorlib.base.executor import TaskSchedulerBase from executorlib.interactive.shared import execute_tasks from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner -class OneTaskPerProcessExecutor(ExecutorBase): +class OneProcessTaskScheduler(TaskSchedulerBase): """ The executorlib.interactive.executor.InteractiveStepExecutor leverages the executorlib interfaces to distribute python tasks. In contrast to the mpi4py.futures.MPIPoolExecutor the executorlib.interactive.executor.InteractiveStepExecutor @@ -23,7 +23,7 @@ class OneTaskPerProcessExecutor(ExecutorBase): Examples: >>> import numpy as np - >>> from executorlib.interactive.onetoone import OneTaskPerProcessExecutor + >>> from executorlib.interactive.onetoone import OneProcessTaskScheduler >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -31,7 +31,7 @@ class OneTaskPerProcessExecutor(ExecutorBase): >>> rank = MPI.COMM_WORLD.Get_rank() >>> return np.array([i, j, k]), size, rank >>> - >>> with OneTaskPerProcessExecutor(max_cores=2) as p: + >>> with OneProcessTaskScheduler(max_cores=2) as p: >>> fs = p.submit(calc, 2, j=4, k=3, resource_dict={"cores": 2}) >>> print(fs.result()) diff --git a/executorlib/interfaces/flux.py b/executorlib/interfaces/flux.py index 0d85b5b3..bd9add97 100644 --- a/executorlib/interfaces/flux.py +++ b/executorlib/interfaces/flux.py @@ -1,10 +1,10 @@ import contextlib from typing import Callable, Optional, Union -from executorlib.base.executor import ExecutorInterface -from executorlib.interactive.blockallocation import BlockAllocationExecutor -from executorlib.interactive.dependency import DependencyExecutor -from executorlib.interactive.onetoone import OneTaskPerProcessExecutor +from executorlib.base.executor import ExecutorBase +from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.interactive.dependency import DependencyTaskScheduler +from executorlib.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.inputcheck import ( check_command_line_argument_lst, check_init_function, @@ -22,7 +22,7 @@ ) -class FluxJobExecutor(ExecutorInterface): +class FluxJobExecutor(ExecutorBase): """ The executorlib.Executor leverages either the message passing interface (MPI), the SLURM workload manager or preferable the flux framework for distributing python functions within a given resource allocation. In contrast to @@ -167,7 +167,7 @@ def __init__( ) if not disable_dependencies: super().__init__( - executor=DependencyExecutor( + executor=DependencyTaskScheduler( executor=create_flux_executor( max_workers=max_workers, cache_directory=cache_directory, @@ -207,7 +207,7 @@ def __init__( ) -class FluxClusterExecutor(ExecutorInterface): +class FluxClusterExecutor(ExecutorBase): """ The executorlib.Executor leverages either the message passing interface (MPI), the SLURM workload manager or preferable the flux framework for distributing python functions within a given resource allocation. In contrast to @@ -341,7 +341,7 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.cache.executor import create_file_executor + from executorlib.cache.task_scheduler import create_file_executor super().__init__( executor=create_file_executor( @@ -363,7 +363,7 @@ def __init__( ) else: super().__init__( - executor=DependencyExecutor( + executor=DependencyTaskScheduler( executor=create_flux_executor( max_workers=max_workers, cache_directory=cache_directory, @@ -397,7 +397,7 @@ def create_flux_executor( hostname_localhost: Optional[bool] = None, block_allocation: bool = False, init_function: Optional[Callable] = None, -) -> Union[OneTaskPerProcessExecutor, BlockAllocationExecutor]: +) -> Union[OneProcessTaskScheduler, BlockAllocationTaskScheduler]: """ Create a flux executor @@ -468,13 +468,13 @@ def create_flux_executor( cores=cores_per_worker, threads_per_core=resource_dict.get("threads_per_core", 1), ) - return BlockAllocationExecutor( + return BlockAllocationTaskScheduler( max_workers=max_workers, executor_kwargs=resource_dict, spawner=FluxPythonSpawner, ) else: - return OneTaskPerProcessExecutor( + return OneProcessTaskScheduler( max_cores=max_cores, max_workers=max_workers, executor_kwargs=resource_dict, diff --git a/executorlib/interfaces/single.py b/executorlib/interfaces/single.py index 667104a8..aca2faeb 100644 --- a/executorlib/interfaces/single.py +++ b/executorlib/interfaces/single.py @@ -1,9 +1,9 @@ from typing import Callable, Optional, Union -from executorlib.base.executor import ExecutorInterface -from executorlib.interactive.blockallocation import BlockAllocationExecutor -from executorlib.interactive.dependency import DependencyExecutor -from executorlib.interactive.onetoone import OneTaskPerProcessExecutor +from executorlib.base.executor import ExecutorBase +from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.interactive.dependency import DependencyTaskScheduler +from executorlib.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.inputcheck import ( check_command_line_argument_lst, check_gpus_per_worker, @@ -15,7 +15,7 @@ from executorlib.standalone.interactive.spawner import MpiExecSpawner -class SingleNodeExecutor(ExecutorInterface): +class SingleNodeExecutor(ExecutorBase): """ The executorlib.Executor leverages either the message passing interface (MPI), the SLURM workload manager or preferable the flux framework for distributing python functions within a given resource allocation. In contrast to @@ -147,7 +147,7 @@ def __init__( ) if not disable_dependencies: super().__init__( - executor=DependencyExecutor( + executor=DependencyTaskScheduler( executor=create_single_node_executor( max_workers=max_workers, cache_directory=cache_directory, @@ -187,7 +187,7 @@ def create_single_node_executor( hostname_localhost: Optional[bool] = None, block_allocation: bool = False, init_function: Optional[Callable] = None, -) -> Union[OneTaskPerProcessExecutor, BlockAllocationExecutor]: +) -> Union[OneProcessTaskScheduler, BlockAllocationTaskScheduler]: """ Create a single node executor @@ -241,7 +241,7 @@ def create_single_node_executor( del resource_dict["slurm_cmd_args"] if block_allocation: resource_dict["init_function"] = init_function - return BlockAllocationExecutor( + return BlockAllocationTaskScheduler( max_workers=validate_number_of_cores( max_cores=max_cores, max_workers=max_workers, @@ -252,7 +252,7 @@ def create_single_node_executor( spawner=MpiExecSpawner, ) else: - return OneTaskPerProcessExecutor( + return OneProcessTaskScheduler( max_cores=max_cores, max_workers=max_workers, executor_kwargs=resource_dict, diff --git a/executorlib/interfaces/slurm.py b/executorlib/interfaces/slurm.py index d9365fd2..a6cbe094 100644 --- a/executorlib/interfaces/slurm.py +++ b/executorlib/interfaces/slurm.py @@ -1,9 +1,9 @@ from typing import Callable, Optional, Union -from executorlib.base.executor import ExecutorInterface -from executorlib.interactive.blockallocation import BlockAllocationExecutor -from executorlib.interactive.dependency import DependencyExecutor -from executorlib.interactive.onetoone import OneTaskPerProcessExecutor +from executorlib.base.executor import ExecutorBase +from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.interactive.dependency import DependencyTaskScheduler +from executorlib.interactive.onetoone import OneProcessTaskScheduler from executorlib.interactive.slurmspawner import SrunSpawner, validate_max_workers from executorlib.standalone.inputcheck import ( check_init_function, @@ -13,7 +13,7 @@ ) -class SlurmClusterExecutor(ExecutorInterface): +class SlurmClusterExecutor(ExecutorBase): """ The executorlib.Executor leverages either the message passing interface (MPI), the SLURM workload manager or preferable the flux framework for distributing python functions within a given resource allocation. In contrast to @@ -147,7 +147,7 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.cache.executor import create_file_executor + from executorlib.cache.task_scheduler import create_file_executor super().__init__( executor=create_file_executor( @@ -169,7 +169,7 @@ def __init__( ) else: super().__init__( - executor=DependencyExecutor( + executor=DependencyTaskScheduler( executor=create_slurm_executor( max_workers=max_workers, cache_directory=cache_directory, @@ -187,7 +187,7 @@ def __init__( ) -class SlurmJobExecutor(ExecutorInterface): +class SlurmJobExecutor(ExecutorBase): """ The executorlib.Executor leverages either the message passing interface (MPI), the SLURM workload manager or preferable the flux framework for distributing python functions within a given resource allocation. In contrast to @@ -327,7 +327,7 @@ def __init__( ) if not disable_dependencies: super().__init__( - executor=DependencyExecutor( + executor=DependencyTaskScheduler( executor=create_slurm_executor( max_workers=max_workers, cache_directory=cache_directory, @@ -367,7 +367,7 @@ def create_slurm_executor( hostname_localhost: Optional[bool] = None, block_allocation: bool = False, init_function: Optional[Callable] = None, -) -> Union[OneTaskPerProcessExecutor, BlockAllocationExecutor]: +) -> Union[OneProcessTaskScheduler, BlockAllocationTaskScheduler]: """ Create a SLURM executor @@ -425,13 +425,13 @@ def create_slurm_executor( cores=cores_per_worker, threads_per_core=resource_dict.get("threads_per_core", 1), ) - return BlockAllocationExecutor( + return BlockAllocationTaskScheduler( max_workers=max_workers, executor_kwargs=resource_dict, spawner=SrunSpawner, ) else: - return OneTaskPerProcessExecutor( + return OneProcessTaskScheduler( max_cores=max_cores, max_workers=max_workers, executor_kwargs=resource_dict, diff --git a/notebooks/4-developer.ipynb b/notebooks/4-developer.ipynb index 97f7b0c9..a0abcf05 100644 --- a/notebooks/4-developer.ipynb +++ b/notebooks/4-developer.ipynb @@ -80,23 +80,23 @@ "Given the level of separation the integration of submodules from the standalone module in external software packages should be the easiest way to benefit from the developments in executorlib beyond just using the `Executor` class. \n", "\n", "## Interface Class Hierarchy\n", - "executorlib provides five different interfaces, namely `SingleNodeExecutor`, `SlurmClusterExecutor`, `SlurmJobExecutor`, `FluxClusterExecutor` and `FluxJobExecutor`, internally these are mapped to four types of task schedulers `Executor._task_scheduler`, namely `BlockAllocationExecutor`, `DependencyExecutor`, `FileExecutor` and `OneTaskPerProcessExecutor` depending on which options are selected. The dependence is illustrated in the following table:\n", + "executorlib provides five different interfaces, namely `SingleNodeExecutor`, `SlurmClusterExecutor`, `SlurmJobExecutor`, `FluxClusterExecutor` and `FluxJobExecutor`, internally these are mapped to four types of task schedulers `Executor._task_scheduler`, namely `BlockAllocationTaskScheduler`, `DependencyTaskScheduler`, `FileTaskScheduler` and `OneProcessTaskScheduler` depending on which options are selected. The dependence is illustrated in the following table:\n", "\n", - "| | `BlockAllocationExecutor` | `DependencyExecutor` | `FileExecutor` | `OneTaskPerProcessExecutor` |\n", - "|-------------------------------------------------------------------------|---------------------------|--------------------------|----------------|-----------------------------|\n", - "| `SingleNodeExecutor(disable_dependencies=False)` | | with `MpiExecSpawner` | | |\n", - "| `SingleNodeExecutor(disable_dependencies=True, block_allocation=False)` | | | | with `MpiExecSpawner` |\n", - "| `SingleNodeExecutor(disable_dependencies=True, block_allocation=True)` | with `MpiExecSpawner` | | | |\n", - "| `SlurmClusterExecutor(plot_dependency_graph=False)` | | | with `pysqa` | |\n", - "| `SlurmClusterExecutor(plot_dependency_graph=True)` | | with `SrunSpawner` | | |\n", - "| `SlurmJobExecutor(disable_dependencies=False)` | | with `SrunSpawner` | | |\n", - "| `SlurmJobExecutor(disable_dependencies=True, block_allocation=False)` | | | | with `SrunSpawner` |\n", - "| `SlurmJobExecutor(disable_dependencies=True, block_allocation=True)` | with `SrunSpawner` | | | |\n", - "| `FluxClusterExecutor(plot_dependency_graph=False)` | | | with `pysqa` | |\n", - "| `FluxClusterExecutor(plot_dependency_graph=True)` | | with `FluxPythonSpawner` | | |\n", - "| `FluxJobExecutor(disable_dependencies=False)` | | with `FluxPythonSpawner` | | |\n", - "| `FluxJobExecutor(disable_dependencies=True, block_allocation=False)` | | | | with `FluxPythonSpawner` |\n", - "| `FluxJobExecutor(disable_dependencies=True, block_allocation=True)` | with `FluxPythonSpawner` | | | |" + "| | `BlockAllocationTaskScheduler` | `DependencyTaskScheduler` | `FileTaskScheduler` | `OneProcessTaskScheduler` |\n", + "|-------------------------------------------------------------------------|--------------------------------|---------------------------|---------------------|---------------------------|\n", + "| `SingleNodeExecutor(disable_dependencies=False)` | | with `MpiExecSpawner` | | |\n", + "| `SingleNodeExecutor(disable_dependencies=True, block_allocation=False)` | | | | with `MpiExecSpawner` |\n", + "| `SingleNodeExecutor(disable_dependencies=True, block_allocation=True)` | with `MpiExecSpawner` | | | |\n", + "| `SlurmClusterExecutor(plot_dependency_graph=False)` | | | with `pysqa` | |\n", + "| `SlurmClusterExecutor(plot_dependency_graph=True)` | | with `SrunSpawner` | | |\n", + "| `SlurmJobExecutor(disable_dependencies=False)` | | with `SrunSpawner` | | |\n", + "| `SlurmJobExecutor(disable_dependencies=True, block_allocation=False)` | | | | with `SrunSpawner` |\n", + "| `SlurmJobExecutor(disable_dependencies=True, block_allocation=True)` | with `SrunSpawner` | | | |\n", + "| `FluxClusterExecutor(plot_dependency_graph=False)` | | | with `pysqa` | |\n", + "| `FluxClusterExecutor(plot_dependency_graph=True)` | | with `FluxPythonSpawner` | | |\n", + "| `FluxJobExecutor(disable_dependencies=False)` | | with `FluxPythonSpawner` | | |\n", + "| `FluxJobExecutor(disable_dependencies=True, block_allocation=False)` | | | | with `FluxPythonSpawner` |\n", + "| `FluxJobExecutor(disable_dependencies=True, block_allocation=True)` | with `FluxPythonSpawner` | | | |" ], "metadata": {} }, diff --git a/tests/test_cache_fileexecutor_mpi.py b/tests/test_cache_fileexecutor_mpi.py index e30dd699..18817f84 100644 --- a/tests/test_cache_fileexecutor_mpi.py +++ b/tests/test_cache_fileexecutor_mpi.py @@ -7,7 +7,7 @@ try: - from executorlib.cache.executor import FileExecutor + from executorlib.cache.task_scheduler import FileTaskScheduler skip_h5py_test = False except ImportError: @@ -31,7 +31,7 @@ def mpi_funct(i): ) class TestCacheExecutorMPI(unittest.TestCase): def test_executor(self): - with FileExecutor( + with FileTaskScheduler( resource_dict={"cores": 2}, execute_function=execute_in_subprocess ) as exe: fs1 = exe.submit(mpi_funct, 1) diff --git a/tests/test_cache_fileexecutor_serial.py b/tests/test_cache_fileexecutor_serial.py index c6ac1b1a..c660eb33 100644 --- a/tests/test_cache_fileexecutor_serial.py +++ b/tests/test_cache_fileexecutor_serial.py @@ -11,7 +11,7 @@ ) try: - from executorlib.cache.executor import FileExecutor, create_file_executor + from executorlib.cache.task_scheduler import FileTaskScheduler, create_file_executor from executorlib.cache.shared import execute_tasks_h5 skip_h5py_test = False @@ -36,14 +36,14 @@ def get_error(a): ) class TestCacheExecutorSerial(unittest.TestCase): def test_executor_mixed(self): - with FileExecutor(execute_function=execute_in_subprocess) as exe: + with FileTaskScheduler(execute_function=execute_in_subprocess) as exe: fs1 = exe.submit(my_funct, 1, b=2) self.assertFalse(fs1.done()) self.assertEqual(fs1.result(), 3) self.assertTrue(fs1.done()) def test_executor_dependence_mixed(self): - with FileExecutor(execute_function=execute_in_subprocess) as exe: + with FileTaskScheduler(execute_function=execute_in_subprocess) as exe: fs1 = exe.submit(my_funct, 1, b=2) fs2 = exe.submit(my_funct, 1, b=fs1) self.assertFalse(fs2.done()) @@ -58,7 +58,7 @@ def test_create_file_executor_error(self): def test_executor_dependence_error(self): with self.assertRaises(ValueError): - with FileExecutor( + with FileTaskScheduler( execute_function=execute_in_subprocess, disable_dependencies=True ) as exe: fs = exe.submit(my_funct, 1, b=exe.submit(my_funct, 1, b=2)) @@ -66,7 +66,7 @@ def test_executor_dependence_error(self): def test_executor_working_directory(self): cwd = os.path.join(os.path.dirname(__file__), "executables") - with FileExecutor( + with FileTaskScheduler( resource_dict={"cwd": cwd}, execute_function=execute_in_subprocess ) as exe: fs1 = exe.submit(list_files_in_working_directory) @@ -74,7 +74,7 @@ def test_executor_working_directory(self): def test_executor_error(self): cwd = os.path.join(os.path.dirname(__file__), "executables") - with FileExecutor( + with FileTaskScheduler( resource_dict={"cwd": cwd}, execute_function=execute_in_subprocess ) as exe: fs1 = exe.submit(get_error, a=1) diff --git a/tests/test_fluxpythonspawner.py b/tests/test_fluxpythonspawner.py index 7cf0ad89..5c5c52eb 100644 --- a/tests/test_fluxpythonspawner.py +++ b/tests/test_fluxpythonspawner.py @@ -6,7 +6,7 @@ import numpy as np from executorlib.interactive.shared import execute_tasks -from executorlib.interactive.blockallocation import BlockAllocationExecutor +from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler from executorlib.standalone.serialize import cloudpickle_register @@ -48,7 +48,7 @@ def setUp(self): self.flux_executor = flux.job.FluxExecutor() def test_flux_executor_serial(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=2, executor_kwargs={"flux_executor": self.flux_executor, "priority": 20}, spawner=FluxPythonSpawner, @@ -61,7 +61,7 @@ def test_flux_executor_serial(self): self.assertTrue(fs_2.done()) def test_flux_executor_threads(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={ "flux_executor": self.flux_executor, @@ -77,7 +77,7 @@ def test_flux_executor_threads(self): self.assertTrue(fs_2.done()) def test_flux_executor_parallel(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={ "flux_executor": self.flux_executor, @@ -91,7 +91,7 @@ def test_flux_executor_parallel(self): self.assertTrue(fs_1.done()) def test_single_task(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={ "flux_executor": self.flux_executor, @@ -138,7 +138,7 @@ def test_execute_task_threads(self): q.join() def test_internal_memory(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={ "flux_executor": self.flux_executor, diff --git a/tests/test_interactive_dependencies.py b/tests/test_interactive_dependencies.py index 24ed3c04..e662cfa0 100644 --- a/tests/test_interactive_dependencies.py +++ b/tests/test_interactive_dependencies.py @@ -5,7 +5,7 @@ import numpy as np -from executorlib.interactive.blockallocation import BlockAllocationExecutor +from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler from executorlib.standalone.interactive.spawner import MpiExecSpawner @@ -18,7 +18,7 @@ def calc(i): class TestFuture(unittest.TestCase): def test_pool_serial(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -34,7 +34,7 @@ def test_pool_serial(self): skip_mpi4py_test, "mpi4py is not installed, so the mpi4py tests are skipped." ) def test_pool_serial_multi_core(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -67,7 +67,7 @@ def callback(future): def submit(): # Executor only exists in this scope and can get garbage collected after # this function is exits - future = BlockAllocationExecutor( + future = BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={}, spawner=MpiExecSpawner, @@ -108,7 +108,7 @@ def __init__(self): def run(self): self.running = True - future = BlockAllocationExecutor( + future = BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={}, spawner=MpiExecSpawner, diff --git a/tests/test_mpiexecspawner.py b/tests/test_mpiexecspawner.py index 4d36fb86..9de3b7a2 100644 --- a/tests/test_mpiexecspawner.py +++ b/tests/test_mpiexecspawner.py @@ -7,11 +7,11 @@ import numpy as np -from executorlib.base.executor import ExecutorBase +from executorlib.base.executor import TaskSchedulerBase from executorlib.standalone.interactive.spawner import MpiExecSpawner from executorlib.interactive.shared import execute_tasks -from executorlib.interactive.blockallocation import BlockAllocationExecutor -from executorlib.interactive.onetoone import OneTaskPerProcessExecutor +from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.interactive.backend import call_funct from executorlib.standalone.serialize import cloudpickle_register @@ -64,7 +64,7 @@ def sleep_one(i): class TestPyMpiExecutorSerial(unittest.TestCase): def test_pympiexecutor_two_workers(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=2, executor_kwargs={}, spawner=MpiExecSpawner, @@ -78,7 +78,7 @@ def test_pympiexecutor_two_workers(self): self.assertTrue(fs_2.done()) def test_max_workers(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=2, executor_kwargs={}, spawner=MpiExecSpawner, @@ -86,7 +86,7 @@ def test_max_workers(self): self.assertEqual(exe.max_workers, 2) def test_pympiexecutor_one_worker(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={}, spawner=MpiExecSpawner, @@ -102,7 +102,7 @@ def test_pympiexecutor_one_worker(self): class TestPyMpiExecutorStepSerial(unittest.TestCase): def test_pympiexecutor_two_workers(self): - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_cores=2, executor_kwargs={}, spawner=MpiExecSpawner, @@ -116,7 +116,7 @@ def test_pympiexecutor_two_workers(self): self.assertTrue(fs_2.done()) def test_max_workers(self): - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_workers=2, executor_kwargs={}, spawner=MpiExecSpawner, @@ -124,7 +124,7 @@ def test_max_workers(self): self.assertEqual(exe.max_workers, 2) def test_pympiexecutor_one_worker(self): - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_cores=1, executor_kwargs={}, spawner=MpiExecSpawner, @@ -143,7 +143,7 @@ def test_pympiexecutor_one_worker(self): ) class TestPyMpiExecutorMPI(unittest.TestCase): def test_pympiexecutor_one_worker_with_mpi(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -154,7 +154,7 @@ def test_pympiexecutor_one_worker_with_mpi(self): self.assertTrue(fs_1.done()) def test_pympiexecutor_one_worker_with_mpi_multiple_submissions(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -174,7 +174,7 @@ def test_pympiexecutor_one_worker_with_mpi_multiple_submissions(self): ) def test_pympiexecutor_one_worker_with_mpi_echo(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -189,7 +189,7 @@ def test_pympiexecutor_one_worker_with_mpi_echo(self): ) class TestPyMpiStepExecutorMPI(unittest.TestCase): def test_pympiexecutor_one_worker_with_mpi(self): - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_cores=2, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -200,7 +200,7 @@ def test_pympiexecutor_one_worker_with_mpi(self): self.assertTrue(fs_1.done()) def test_pympiexecutor_one_worker_with_mpi_multiple_submissions(self): - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_cores=2, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -220,7 +220,7 @@ def test_pympiexecutor_one_worker_with_mpi_multiple_submissions(self): ) def test_pympiexecutor_one_worker_with_mpi_echo(self): - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_cores=2, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -232,7 +232,7 @@ def test_pympiexecutor_one_worker_with_mpi_echo(self): class TestPyMpiExecutorInitFunction(unittest.TestCase): def test_internal_memory(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={ "cores": 1, @@ -273,7 +273,7 @@ def test_execute_task(self): class TestFuturePool(unittest.TestCase): def test_pool_serial(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -288,7 +288,7 @@ def test_pool_serial(self): self.assertEqual(output.result(), np.array(4)) def test_executor_multi_submission(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -301,7 +301,7 @@ def test_executor_multi_submission(self): self.assertTrue(fs_2.done()) def test_shutdown(self): - p = BlockAllocationExecutor( + p = BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -317,7 +317,7 @@ def test_shutdown(self): fs2.result() def test_pool_serial_map(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -327,7 +327,7 @@ def test_pool_serial_map(self): def test_executor_exception(self): with self.assertRaises(RuntimeError): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -337,7 +337,7 @@ def test_executor_exception(self): def test_executor_exception_future(self): with self.assertRaises(RuntimeError): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 1}, spawner=MpiExecSpawner, @@ -358,7 +358,7 @@ def test_meta(self): "openmpi_oversubscribe": False, "max_workers": 1, } - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={ "cores": 2, @@ -374,7 +374,7 @@ def test_meta(self): self.assertEqual(exe.info[k], v) else: self.assertEqual(str(exe.info[k]), v) - with ExecutorBase() as exe: + with TaskSchedulerBase() as exe: self.assertIsNone(exe.info) def test_meta_step(self): @@ -386,7 +386,7 @@ def test_meta_step(self): "openmpi_oversubscribe": False, "max_cores": 2, } - with OneTaskPerProcessExecutor( + with OneProcessTaskScheduler( max_cores=2, executor_kwargs={ "cores": 2, @@ -406,7 +406,7 @@ def test_meta_step(self): skip_mpi4py_test, "mpi4py is not installed, so the mpi4py tests are skipped." ) def test_pool_multi_core(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, @@ -424,7 +424,7 @@ def test_pool_multi_core(self): skip_mpi4py_test, "mpi4py is not installed, so the mpi4py tests are skipped." ) def test_pool_multi_core_map(self): - with BlockAllocationExecutor( + with BlockAllocationTaskScheduler( max_workers=1, executor_kwargs={"cores": 2}, spawner=MpiExecSpawner, From 6ae7b7e0f704a7a0bc8c3ac955940eb47185712f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:05:00 +0200 Subject: [PATCH 02/14] split task scheduler and executor --- executorlib/base/executor.py | 175 +------------------- executorlib/base/task_scheduler.py | 181 +++++++++++++++++++++ executorlib/cache/task_scheduler.py | 2 +- executorlib/interactive/blockallocation.py | 3 +- executorlib/interactive/dependency.py | 2 +- executorlib/interactive/onetoone.py | 2 +- tests/test_mpiexecspawner.py | 2 +- 7 files changed, 189 insertions(+), 178 deletions(-) create mode 100644 executorlib/base/task_scheduler.py diff --git a/executorlib/base/executor.py b/executorlib/base/executor.py index 0b946856..edf59ca2 100644 --- a/executorlib/base/executor.py +++ b/executorlib/base/executor.py @@ -1,4 +1,3 @@ -import contextlib import queue from concurrent.futures import ( Executor as FutureExecutor, @@ -6,179 +5,9 @@ from concurrent.futures import ( Future, ) -from threading import Thread -from typing import Callable, Optional, Union +from typing import Callable, Optional -from executorlib.standalone.inputcheck import check_resource_dict -from executorlib.standalone.queue import cancel_items_in_queue -from executorlib.standalone.serialize import cloudpickle_register - - -class TaskSchedulerBase(FutureExecutor): - """ - Base class for the executor. - - Args: - max_cores (int): defines the number cores which can be used in parallel - """ - - def __init__(self, max_cores: Optional[int] = None): - """ - Initialize the ExecutorBase class. - """ - cloudpickle_register(ind=3) - self._process_kwargs: dict = {} - self._max_cores = max_cores - self._future_queue: Optional[queue.Queue] = queue.Queue() - self._process: Optional[Union[Thread, list[Thread]]] = None - - @property - def max_workers(self) -> Optional[int]: - return self._process_kwargs.get("max_workers") - - @max_workers.setter - def max_workers(self, max_workers: int): - raise NotImplementedError("The max_workers setter is not implemented.") - - @property - def info(self) -> Optional[dict]: - """ - Get the information about the executor. - - Returns: - Optional[dict]: Information about the executor. - """ - meta_data_dict = self._process_kwargs.copy() - if "future_queue" in meta_data_dict: - del meta_data_dict["future_queue"] - if self._process is not None and isinstance(self._process, list): - meta_data_dict["max_workers"] = len(self._process) - return meta_data_dict - elif self._process is not None: - return meta_data_dict - else: - return None - - @property - def future_queue(self) -> Optional[queue.Queue]: - """ - Get the future queue. - - Returns: - queue.Queue: The future queue. - """ - return self._future_queue - - def submit( # type: ignore - self, - fn: Callable, - /, - *args, - resource_dict: Optional[dict] = None, - **kwargs, - ) -> Future: - """ - Submits a callable to be executed with the given arguments. - - Schedules the callable to be executed as fn(*args, **kwargs) and returns - a Future instance representing the execution of the callable. - - Args: - fn (callable): function to submit for execution - args: arguments for the submitted function - kwargs: keyword arguments for the submitted function - resource_dict (dict): resource dictionary, which defines the resources used for the execution of the - function. Example resource dictionary: { - cores: 1, - threads_per_core: 1, - gpus_per_worker: 0, - oversubscribe: False, - cwd: None, - executor: None, - hostname_localhost: False, - } - - Returns: - Future: A Future representing the given call. - """ - if resource_dict is None: - resource_dict = {} - cores = resource_dict.get("cores") - if ( - cores is not None - and self._max_cores is not None - and cores > self._max_cores - ): - raise ValueError( - "The specified number of cores is larger than the available number of cores." - ) - check_resource_dict(function=fn) - f: Future = Future() - if self._future_queue is not None: - self._future_queue.put( - { - "fn": fn, - "args": args, - "kwargs": kwargs, - "future": f, - "resource_dict": resource_dict, - } - ) - return f - - def shutdown(self, wait: bool = True, *, cancel_futures: bool = False): - """ - Clean-up the resources associated with the Executor. - - It is safe to call this method several times. Otherwise, no other - methods can be called after this one. - - Args: - wait (bool): If True then shutdown will not return until all running - futures have finished executing and the resources used by the - parallel_executors have been reclaimed. - cancel_futures (bool): If True then shutdown will cancel all pending - futures. Futures that are completed or running will not be - cancelled. - """ - if cancel_futures and self._future_queue is not None: - cancel_items_in_queue(que=self._future_queue) - if self._process is not None and self._future_queue is not None: - self._future_queue.put({"shutdown": True, "wait": wait}) - if wait and isinstance(self._process, Thread): - self._process.join() - self._future_queue.join() - self._process = None - self._future_queue = None - - def _set_process(self, process: Thread): - """ - Set the process for the executor. - - Args: - process (RaisingThread): The process for the executor. - """ - self._process = process - self._process.start() - - def __len__(self) -> int: - """ - Get the length of the executor. - - Returns: - int: The length of the executor. - """ - queue_size = 0 - if self._future_queue is not None: - queue_size = self._future_queue.qsize() - return queue_size - - def __del__(self): - """ - Clean-up the resources associated with the Executor. - """ - with contextlib.suppress(AttributeError, RuntimeError): - self.shutdown(wait=False) +from executorlib.base.task_scheduler import TaskSchedulerBase class ExecutorBase(FutureExecutor): diff --git a/executorlib/base/task_scheduler.py b/executorlib/base/task_scheduler.py new file mode 100644 index 00000000..36c46c21 --- /dev/null +++ b/executorlib/base/task_scheduler.py @@ -0,0 +1,181 @@ +import contextlib +import queue +from concurrent.futures import ( + Executor as FutureExecutor, +) +from concurrent.futures import ( + Future, +) +from threading import Thread +from typing import Callable, Optional, Union + +from executorlib.standalone.inputcheck import check_resource_dict +from executorlib.standalone.queue import cancel_items_in_queue +from executorlib.standalone.serialize import cloudpickle_register + + +class TaskSchedulerBase(FutureExecutor): + """ + Base class for the executor. + + Args: + max_cores (int): defines the number cores which can be used in parallel + """ + + def __init__(self, max_cores: Optional[int] = None): + """ + Initialize the ExecutorBase class. + """ + cloudpickle_register(ind=3) + self._process_kwargs: dict = {} + self._max_cores = max_cores + self._future_queue: Optional[queue.Queue] = queue.Queue() + self._process: Optional[Union[Thread, list[Thread]]] = None + + @property + def max_workers(self) -> Optional[int]: + return self._process_kwargs.get("max_workers") + + @max_workers.setter + def max_workers(self, max_workers: int): + raise NotImplementedError("The max_workers setter is not implemented.") + + @property + def info(self) -> Optional[dict]: + """ + Get the information about the executor. + + Returns: + Optional[dict]: Information about the executor. + """ + meta_data_dict = self._process_kwargs.copy() + if "future_queue" in meta_data_dict: + del meta_data_dict["future_queue"] + if self._process is not None and isinstance(self._process, list): + meta_data_dict["max_workers"] = len(self._process) + return meta_data_dict + elif self._process is not None: + return meta_data_dict + else: + return None + + @property + def future_queue(self) -> Optional[queue.Queue]: + """ + Get the future queue. + + Returns: + queue.Queue: The future queue. + """ + return self._future_queue + + def submit( # type: ignore + self, + fn: Callable, + /, + *args, + resource_dict: Optional[dict] = None, + **kwargs, + ) -> Future: + """ + Submits a callable to be executed with the given arguments. + + Schedules the callable to be executed as fn(*args, **kwargs) and returns + a Future instance representing the execution of the callable. + + Args: + fn (callable): function to submit for execution + args: arguments for the submitted function + kwargs: keyword arguments for the submitted function + resource_dict (dict): resource dictionary, which defines the resources used for the execution of the + function. Example resource dictionary: { + cores: 1, + threads_per_core: 1, + gpus_per_worker: 0, + oversubscribe: False, + cwd: None, + executor: None, + hostname_localhost: False, + } + + Returns: + Future: A Future representing the given call. + """ + if resource_dict is None: + resource_dict = {} + cores = resource_dict.get("cores") + if ( + cores is not None + and self._max_cores is not None + and cores > self._max_cores + ): + raise ValueError( + "The specified number of cores is larger than the available number of cores." + ) + check_resource_dict(function=fn) + f: Future = Future() + if self._future_queue is not None: + self._future_queue.put( + { + "fn": fn, + "args": args, + "kwargs": kwargs, + "future": f, + "resource_dict": resource_dict, + } + ) + return f + + def shutdown(self, wait: bool = True, *, cancel_futures: bool = False): + """ + Clean-up the resources associated with the Executor. + + It is safe to call this method several times. Otherwise, no other + methods can be called after this one. + + Args: + wait (bool): If True then shutdown will not return until all running + futures have finished executing and the resources used by the + parallel_executors have been reclaimed. + cancel_futures (bool): If True then shutdown will cancel all pending + futures. Futures that are completed or running will not be + cancelled. + """ + if cancel_futures and self._future_queue is not None: + cancel_items_in_queue(que=self._future_queue) + if self._process is not None and self._future_queue is not None: + self._future_queue.put({"shutdown": True, "wait": wait}) + if wait and isinstance(self._process, Thread): + self._process.join() + self._future_queue.join() + self._process = None + self._future_queue = None + + def _set_process(self, process: Thread): + """ + Set the process for the executor. + + Args: + process (RaisingThread): The process for the executor. + """ + self._process = process + self._process.start() + + def __len__(self) -> int: + """ + Get the length of the executor. + + Returns: + int: The length of the executor. + """ + queue_size = 0 + if self._future_queue is not None: + queue_size = self._future_queue.qsize() + return queue_size + + def __del__(self): + """ + Clean-up the resources associated with the Executor. + """ + with contextlib.suppress(AttributeError, RuntimeError): + self.shutdown(wait=False) diff --git a/executorlib/cache/task_scheduler.py b/executorlib/cache/task_scheduler.py index c8d4c900..88044d9b 100644 --- a/executorlib/cache/task_scheduler.py +++ b/executorlib/cache/task_scheduler.py @@ -2,7 +2,7 @@ from threading import Thread from typing import Callable, Optional -from executorlib.base.executor import TaskSchedulerBase +from executorlib.base.task_scheduler import TaskSchedulerBase from executorlib.cache.shared import execute_tasks_h5 from executorlib.cache.subprocess_spawner import ( execute_in_subprocess, diff --git a/executorlib/interactive/blockallocation.py b/executorlib/interactive/blockallocation.py index a2f00493..9c5aeeef 100644 --- a/executorlib/interactive/blockallocation.py +++ b/executorlib/interactive/blockallocation.py @@ -3,13 +3,14 @@ from threading import Thread from typing import Callable, Optional -from executorlib.base.executor import TaskSchedulerBase, cancel_items_in_queue +from executorlib.base.task_scheduler import TaskSchedulerBase from executorlib.interactive.shared import execute_tasks from executorlib.standalone.inputcheck import ( check_resource_dict, check_resource_dict_is_empty, ) from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner +from executorlib.standalone.queue import cancel_items_in_queue class BlockAllocationTaskScheduler(TaskSchedulerBase): diff --git a/executorlib/interactive/dependency.py b/executorlib/interactive/dependency.py index 0dc01eb5..c7c7a5df 100644 --- a/executorlib/interactive/dependency.py +++ b/executorlib/interactive/dependency.py @@ -4,7 +4,7 @@ from time import sleep from typing import Any, Callable, Optional -from executorlib.base.executor import TaskSchedulerBase +from executorlib.base.task_scheduler import TaskSchedulerBase from executorlib.standalone.interactive.arguments import ( check_exception_was_raised, get_exception_lst, diff --git a/executorlib/interactive/onetoone.py b/executorlib/interactive/onetoone.py index 822660c5..0eeec255 100644 --- a/executorlib/interactive/onetoone.py +++ b/executorlib/interactive/onetoone.py @@ -2,7 +2,7 @@ from threading import Thread from typing import Optional -from executorlib.base.executor import TaskSchedulerBase +from executorlib.base.task_scheduler import TaskSchedulerBase from executorlib.interactive.shared import execute_tasks from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner diff --git a/tests/test_mpiexecspawner.py b/tests/test_mpiexecspawner.py index 9de3b7a2..f00f7ecf 100644 --- a/tests/test_mpiexecspawner.py +++ b/tests/test_mpiexecspawner.py @@ -7,7 +7,7 @@ import numpy as np -from executorlib.base.executor import TaskSchedulerBase +from executorlib.base.task_scheduler import TaskSchedulerBase from executorlib.standalone.interactive.spawner import MpiExecSpawner from executorlib.interactive.shared import execute_tasks from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler From adbdb600b33b0d0d39b4b61d076ccc9dc0c0f12e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:11:32 +0200 Subject: [PATCH 03/14] major refactoring --- executorlib/__init__.py | 6 +- executorlib/backend/cache_parallel.py | 2 +- executorlib/backend/cache_serial.py | 2 +- executorlib/{base => executor}/__init__.py | 0 .../{base/executor.py => executor/base.py} | 2 +- executorlib/{interfaces => executor}/flux.py | 26 +- .../{interfaces => executor}/single.py | 16 +- executorlib/{interfaces => executor}/slurm.py | 26 +- executorlib/standalone/hdf.py | 2 +- .../{cache => task_scheduler}/__init__.py | 0 .../base.py} | 0 .../file}/__init__.py | 0 .../{cache => task_scheduler/file}/backend.py | 2 +- .../file}/queue_spawner.py | 0 .../{cache => task_scheduler/file}/shared.py | 8 +- .../file}/subprocess_spawner.py | 0 .../file}/task_scheduler.py | 12 +- .../interactive}/__init__.py | 0 .../interactive/blockallocation.py | 6 +- .../interactive/dependency.py | 2 +- .../interactive/fluxspawner.py | 0 .../interactive/onetoone.py | 6 +- .../interactive/shared.py | 10 +- .../interactive/slurmspawner.py | 0 notebooks/1-single-node.ipynb | 8 +- notebooks/2-hpc-cluster.ipynb | 195 ++++++++++- notebooks/3-hpc-job.ipynb | 323 +++++++++++++++++- tests/test_base_executor_queue.py | 2 +- tests/test_cache_backend_execute.py | 40 +-- tests/test_cache_fileexecutor_mpi.py | 8 +- tests/test_cache_fileexecutor_serial.py | 16 +- tests/test_fluxclusterexecutor.py | 8 +- tests/test_fluxjobexecutor.py | 2 +- tests/test_fluxjobexecutor_plot.py | 2 +- tests/test_fluxpythonspawner.py | 6 +- tests/test_interactive_dependencies.py | 2 +- tests/test_interactive_slurmspawner.py | 4 +- tests/test_mpiexecspawner.py | 14 +- tests/test_singlenodeexecutor_cache.py | 6 +- tests/test_singlenodeexecutor_dependencies.py | 4 +- tests/test_singlenodeexecutor_mpi.py | 4 +- .../test_singlenodeexecutor_shell_executor.py | 2 +- ...st_singlenodeexecutor_shell_interactive.py | 2 +- tests/test_standalone_hdf.py | 14 +- tests/test_standalone_interactive_backend.py | 2 +- 45 files changed, 653 insertions(+), 139 deletions(-) rename executorlib/{base => executor}/__init__.py (100%) rename executorlib/{base/executor.py => executor/base.py} (98%) rename executorlib/{interfaces => executor}/flux.py (97%) rename executorlib/{interfaces => executor}/single.py (96%) rename executorlib/{interfaces => executor}/slurm.py (97%) rename executorlib/{cache => task_scheduler}/__init__.py (100%) rename executorlib/{base/task_scheduler.py => task_scheduler/base.py} (100%) rename executorlib/{interactive => task_scheduler/file}/__init__.py (100%) rename executorlib/{cache => task_scheduler/file}/backend.py (97%) rename executorlib/{cache => task_scheduler/file}/queue_spawner.py (100%) rename executorlib/{cache => task_scheduler/file}/shared.py (97%) rename executorlib/{cache => task_scheduler/file}/subprocess_spawner.py (100%) rename executorlib/{cache => task_scheduler/file}/task_scheduler.py (93%) rename executorlib/{interfaces => task_scheduler/interactive}/__init__.py (100%) rename executorlib/{ => task_scheduler}/interactive/blockallocation.py (97%) rename executorlib/{ => task_scheduler}/interactive/dependency.py (99%) rename executorlib/{ => task_scheduler}/interactive/fluxspawner.py (100%) rename executorlib/{ => task_scheduler}/interactive/onetoone.py (98%) rename executorlib/{ => task_scheduler}/interactive/shared.py (95%) rename executorlib/{ => task_scheduler}/interactive/slurmspawner.py (100%) diff --git a/executorlib/__init__.py b/executorlib/__init__.py index 127fd879..7b47910b 100644 --- a/executorlib/__init__.py +++ b/executorlib/__init__.py @@ -1,10 +1,10 @@ from executorlib._version import get_versions as _get_versions -from executorlib.interfaces.flux import ( +from executorlib.executor.flux import ( FluxClusterExecutor, FluxJobExecutor, ) -from executorlib.interfaces.single import SingleNodeExecutor -from executorlib.interfaces.slurm import ( +from executorlib.executor.single import SingleNodeExecutor +from executorlib.executor.slurm import ( SlurmClusterExecutor, SlurmJobExecutor, ) diff --git a/executorlib/backend/cache_parallel.py b/executorlib/backend/cache_parallel.py index f094446d..1e47c7ed 100644 --- a/executorlib/backend/cache_parallel.py +++ b/executorlib/backend/cache_parallel.py @@ -4,7 +4,7 @@ import cloudpickle -from executorlib.cache.backend import backend_load_file, backend_write_file +from executorlib.task_scheduler.file import backend_load_file, backend_write_file def main() -> None: diff --git a/executorlib/backend/cache_serial.py b/executorlib/backend/cache_serial.py index daf5a520..1a0b487d 100644 --- a/executorlib/backend/cache_serial.py +++ b/executorlib/backend/cache_serial.py @@ -1,6 +1,6 @@ import sys -from executorlib.cache.backend import backend_execute_task_in_file +from executorlib.task_scheduler.file import backend_execute_task_in_file if __name__ == "__main__": backend_execute_task_in_file(file_name=sys.argv[1]) diff --git a/executorlib/base/__init__.py b/executorlib/executor/__init__.py similarity index 100% rename from executorlib/base/__init__.py rename to executorlib/executor/__init__.py diff --git a/executorlib/base/executor.py b/executorlib/executor/base.py similarity index 98% rename from executorlib/base/executor.py rename to executorlib/executor/base.py index edf59ca2..78a410a4 100644 --- a/executorlib/base/executor.py +++ b/executorlib/executor/base.py @@ -7,7 +7,7 @@ ) from typing import Callable, Optional -from executorlib.base.task_scheduler import TaskSchedulerBase +from executorlib.task_scheduler.base import TaskSchedulerBase class ExecutorBase(FutureExecutor): diff --git a/executorlib/interfaces/flux.py b/executorlib/executor/flux.py similarity index 97% rename from executorlib/interfaces/flux.py rename to executorlib/executor/flux.py index bd9add97..e45b0f9c 100644 --- a/executorlib/interfaces/flux.py +++ b/executorlib/executor/flux.py @@ -1,10 +1,10 @@ import contextlib from typing import Callable, Optional, Union -from executorlib.base.executor import ExecutorBase -from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.interactive.dependency import DependencyTaskScheduler -from executorlib.interactive.onetoone import OneProcessTaskScheduler +from executorlib.executor.base import ExecutorBase +from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.inputcheck import ( check_command_line_argument_lst, check_init_function, @@ -16,7 +16,7 @@ ) with contextlib.suppress(ImportError): - from executorlib.interactive.fluxspawner import ( + from executorlib.task_scheduler.interactive.fluxspawner import ( FluxPythonSpawner, validate_max_workers, ) @@ -34,7 +34,7 @@ class FluxJobExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -70,7 +70,7 @@ class FluxJobExecutor(ExecutorBase): Examples: ``` >>> import numpy as np - >>> from executorlib.interfaces.flux import FluxJobExecutor + >>> from executorlib.executor.flux import FluxJobExecutor >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -118,7 +118,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -219,7 +219,7 @@ class FluxClusterExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -251,7 +251,7 @@ class FluxClusterExecutor(ExecutorBase): Examples: ``` >>> import numpy as np - >>> from executorlib.interfaces.flux import FluxClusterExecutor + >>> from executorlib.executor.flux import FluxClusterExecutor >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -296,7 +296,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -341,7 +341,7 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.cache.task_scheduler import create_file_executor + from executorlib.task_scheduler.file import create_file_executor super().__init__( executor=create_file_executor( @@ -406,7 +406,7 @@ def create_flux_executor( number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. max_cores (int): defines the number cores which can be used in parallel - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - threads_per_core (int): number of OpenMP threads to be used for each function call diff --git a/executorlib/interfaces/single.py b/executorlib/executor/single.py similarity index 96% rename from executorlib/interfaces/single.py rename to executorlib/executor/single.py index aca2faeb..4de30d8b 100644 --- a/executorlib/interfaces/single.py +++ b/executorlib/executor/single.py @@ -1,9 +1,9 @@ from typing import Callable, Optional, Union -from executorlib.base.executor import ExecutorBase -from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.interactive.dependency import DependencyTaskScheduler -from executorlib.interactive.onetoone import OneProcessTaskScheduler +from executorlib.executor.base import ExecutorBase +from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.inputcheck import ( check_command_line_argument_lst, check_gpus_per_worker, @@ -27,7 +27,7 @@ class SingleNodeExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -58,7 +58,7 @@ class SingleNodeExecutor(ExecutorBase): Examples: ``` >>> import numpy as np - >>> from executorlib.interfaces.single import SingleNodeExecutor + >>> from executorlib.executor.single import SingleNodeExecutor >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -102,7 +102,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -196,7 +196,7 @@ def create_single_node_executor( number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. max_cores (int): defines the number cores which can be used in parallel - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - threads_per_core (int): number of OpenMP threads to be used for each function call diff --git a/executorlib/interfaces/slurm.py b/executorlib/executor/slurm.py similarity index 97% rename from executorlib/interfaces/slurm.py rename to executorlib/executor/slurm.py index a6cbe094..c93062dc 100644 --- a/executorlib/interfaces/slurm.py +++ b/executorlib/executor/slurm.py @@ -1,10 +1,10 @@ from typing import Callable, Optional, Union -from executorlib.base.executor import ExecutorBase -from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.interactive.dependency import DependencyTaskScheduler -from executorlib.interactive.onetoone import OneProcessTaskScheduler -from executorlib.interactive.slurmspawner import SrunSpawner, validate_max_workers +from executorlib.executor.base import ExecutorBase +from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler +from executorlib.task_scheduler.interactive.slurmspawner import SrunSpawner, validate_max_workers from executorlib.standalone.inputcheck import ( check_init_function, check_plot_dependency_graph, @@ -25,7 +25,7 @@ class SlurmClusterExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -57,7 +57,7 @@ class SlurmClusterExecutor(ExecutorBase): Examples: ``` >>> import numpy as np - >>> from executorlib.interfaces.slurm import SlurmClusterExecutor + >>> from executorlib.executor.slurm import SlurmClusterExecutor >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -102,7 +102,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -147,7 +147,7 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.cache.task_scheduler import create_file_executor + from executorlib.task_scheduler.file import create_file_executor super().__init__( executor=create_file_executor( @@ -199,7 +199,7 @@ class SlurmJobExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -234,7 +234,7 @@ class SlurmJobExecutor(ExecutorBase): Examples: ``` >>> import numpy as np - >>> from executorlib.interfaces.slurm import SlurmJobExecutor + >>> from executorlib.executor.slurm import SlurmJobExecutor >>> >>> def calc(i, j, k): >>> from mpi4py import MPI @@ -278,7 +278,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -376,7 +376,7 @@ def create_slurm_executor( number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. max_cores (int): defines the number cores which can be used in parallel - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - threads_per_core (int): number of OpenMP threads to be used for each function call diff --git a/executorlib/standalone/hdf.py b/executorlib/standalone/hdf.py index 8fb26f72..b782b5c0 100644 --- a/executorlib/standalone/hdf.py +++ b/executorlib/standalone/hdf.py @@ -108,7 +108,7 @@ def get_queue_id(file_name: Optional[str]) -> Optional[int]: def get_cache_data(cache_directory: str) -> list[dict]: file_lst = [] for task_key in os.listdir(cache_directory): - file_name = os.path.join(cache_directory, task_key, "cache.h5out") + file_name = os.path.join(cache_directory, task_key, "file.h5out") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) if os.path.exists(file_name): with h5py.File(file_name, "r") as hdf: diff --git a/executorlib/cache/__init__.py b/executorlib/task_scheduler/__init__.py similarity index 100% rename from executorlib/cache/__init__.py rename to executorlib/task_scheduler/__init__.py diff --git a/executorlib/base/task_scheduler.py b/executorlib/task_scheduler/base.py similarity index 100% rename from executorlib/base/task_scheduler.py rename to executorlib/task_scheduler/base.py diff --git a/executorlib/interactive/__init__.py b/executorlib/task_scheduler/file/__init__.py similarity index 100% rename from executorlib/interactive/__init__.py rename to executorlib/task_scheduler/file/__init__.py diff --git a/executorlib/cache/backend.py b/executorlib/task_scheduler/file/backend.py similarity index 97% rename from executorlib/cache/backend.py rename to executorlib/task_scheduler/file/backend.py index cbb649e8..8d54e19f 100644 --- a/executorlib/cache/backend.py +++ b/executorlib/task_scheduler/file/backend.py @@ -2,7 +2,7 @@ import time from typing import Any -from executorlib.cache.shared import FutureItem +from executorlib.task_scheduler.file.shared import FutureItem from executorlib.standalone.hdf import dump, load diff --git a/executorlib/cache/queue_spawner.py b/executorlib/task_scheduler/file/queue_spawner.py similarity index 100% rename from executorlib/cache/queue_spawner.py rename to executorlib/task_scheduler/file/queue_spawner.py diff --git a/executorlib/cache/shared.py b/executorlib/task_scheduler/file/shared.py similarity index 97% rename from executorlib/cache/shared.py rename to executorlib/task_scheduler/file/shared.py index 36c3f693..deb59374 100644 --- a/executorlib/cache/shared.py +++ b/executorlib/task_scheduler/file/shared.py @@ -110,11 +110,11 @@ def execute_tasks_h5( if task_key not in memory_dict: if not ( task_key in os.listdir(cache_directory) - and "cache.h5out" + and "file.h5out" in os.listdir(os.path.join(cache_directory, task_key)) ): os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) - file_name = os.path.join(cache_directory, task_key, "cache.h5in") + file_name = os.path.join(cache_directory, task_key, "file.h5in") dump(file_name=file_name, data_dict=data_dict) if not disable_dependencies: task_dependent_lst = [ @@ -141,7 +141,7 @@ def execute_tasks_h5( cache_directory=os.path.join(cache_directory, task_key), ) file_name_dict[task_key] = os.path.join( - cache_directory, task_key, "cache.h5out" + cache_directory, task_key, "file.h5out" ) memory_dict[task_key] = task_dict["future"] future_queue.task_done() @@ -197,7 +197,7 @@ def _check_task_output( Future: The updated future object. """ - file_name = os.path.join(cache_directory, task_key, "cache.h5out") + file_name = os.path.join(cache_directory, task_key, "file.h5out") if not os.path.exists(file_name): return future_obj exec_flag, no_error_flag, result = get_output(file_name=file_name) diff --git a/executorlib/cache/subprocess_spawner.py b/executorlib/task_scheduler/file/subprocess_spawner.py similarity index 100% rename from executorlib/cache/subprocess_spawner.py rename to executorlib/task_scheduler/file/subprocess_spawner.py diff --git a/executorlib/cache/task_scheduler.py b/executorlib/task_scheduler/file/task_scheduler.py similarity index 93% rename from executorlib/cache/task_scheduler.py rename to executorlib/task_scheduler/file/task_scheduler.py index 88044d9b..9993d97f 100644 --- a/executorlib/cache/task_scheduler.py +++ b/executorlib/task_scheduler/file/task_scheduler.py @@ -2,9 +2,9 @@ from threading import Thread from typing import Callable, Optional -from executorlib.base.task_scheduler import TaskSchedulerBase -from executorlib.cache.shared import execute_tasks_h5 -from executorlib.cache.subprocess_spawner import ( +from executorlib.task_scheduler.base import TaskSchedulerBase +from executorlib.task_scheduler.file.shared import execute_tasks_h5 +from executorlib.task_scheduler.file.subprocess_spawner import ( execute_in_subprocess, terminate_subprocess, ) @@ -18,7 +18,7 @@ ) try: - from executorlib.cache.queue_spawner import execute_with_pysqa + from executorlib.task_scheduler.file.queue_spawner import execute_with_pysqa except ImportError: # If pysqa is not available fall back to executing tasks in a subprocess execute_with_pysqa = execute_in_subprocess # type: ignore @@ -27,7 +27,7 @@ class FileTaskScheduler(TaskSchedulerBase): def __init__( self, - cache_directory: str = "cache", + cache_directory: str = "file", resource_dict: Optional[dict] = None, execute_function: Callable = execute_with_pysqa, terminate_function: Optional[Callable] = None, @@ -39,7 +39,7 @@ def __init__( Initialize the FileExecutor. Args: - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - cwd (str/None): current working directory where the parallel python task is executed diff --git a/executorlib/interfaces/__init__.py b/executorlib/task_scheduler/interactive/__init__.py similarity index 100% rename from executorlib/interfaces/__init__.py rename to executorlib/task_scheduler/interactive/__init__.py diff --git a/executorlib/interactive/blockallocation.py b/executorlib/task_scheduler/interactive/blockallocation.py similarity index 97% rename from executorlib/interactive/blockallocation.py rename to executorlib/task_scheduler/interactive/blockallocation.py index 9c5aeeef..967e5c0d 100644 --- a/executorlib/interactive/blockallocation.py +++ b/executorlib/task_scheduler/interactive/blockallocation.py @@ -3,8 +3,8 @@ from threading import Thread from typing import Callable, Optional -from executorlib.base.task_scheduler import TaskSchedulerBase -from executorlib.interactive.shared import execute_tasks +from executorlib.task_scheduler.base import TaskSchedulerBase +from executorlib.task_scheduler.interactive.shared import execute_tasks from executorlib.standalone.inputcheck import ( check_resource_dict, check_resource_dict_is_empty, @@ -15,7 +15,7 @@ class BlockAllocationTaskScheduler(TaskSchedulerBase): """ - The executorlib.interactive.executor.InteractiveExecutor leverages the exeutorlib interfaces to distribute python + The executorlib.interactive.executor.InteractiveExecutor leverages the exeutorlib executor to distribute python tasks on a workstation or inside a queuing system allocation. In contrast to the mpi4py.futures.MPIPoolExecutor the executorlib.interactive.executor.InteractiveExecutor can be executed in a serial python process and does not require the python script to be executed with MPI. Consequently, it is primarily an abstraction of its functionality to diff --git a/executorlib/interactive/dependency.py b/executorlib/task_scheduler/interactive/dependency.py similarity index 99% rename from executorlib/interactive/dependency.py rename to executorlib/task_scheduler/interactive/dependency.py index c7c7a5df..8e3821bf 100644 --- a/executorlib/interactive/dependency.py +++ b/executorlib/task_scheduler/interactive/dependency.py @@ -4,7 +4,7 @@ from time import sleep from typing import Any, Callable, Optional -from executorlib.base.task_scheduler import TaskSchedulerBase +from executorlib.task_scheduler.base import TaskSchedulerBase from executorlib.standalone.interactive.arguments import ( check_exception_was_raised, get_exception_lst, diff --git a/executorlib/interactive/fluxspawner.py b/executorlib/task_scheduler/interactive/fluxspawner.py similarity index 100% rename from executorlib/interactive/fluxspawner.py rename to executorlib/task_scheduler/interactive/fluxspawner.py diff --git a/executorlib/interactive/onetoone.py b/executorlib/task_scheduler/interactive/onetoone.py similarity index 98% rename from executorlib/interactive/onetoone.py rename to executorlib/task_scheduler/interactive/onetoone.py index 0eeec255..6285a813 100644 --- a/executorlib/interactive/onetoone.py +++ b/executorlib/task_scheduler/interactive/onetoone.py @@ -2,14 +2,14 @@ from threading import Thread from typing import Optional -from executorlib.base.task_scheduler import TaskSchedulerBase -from executorlib.interactive.shared import execute_tasks +from executorlib.task_scheduler.base import TaskSchedulerBase +from executorlib.task_scheduler.interactive.shared import execute_tasks from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner class OneProcessTaskScheduler(TaskSchedulerBase): """ - The executorlib.interactive.executor.InteractiveStepExecutor leverages the executorlib interfaces to distribute python + The executorlib.interactive.executor.InteractiveStepExecutor leverages the executorlib executor to distribute python tasks. In contrast to the mpi4py.futures.MPIPoolExecutor the executorlib.interactive.executor.InteractiveStepExecutor can be executed in a serial python process and does not require the python script to be executed with MPI. Consequently, it is primarily an abstraction of its functionality to improve the usability in particular when used diff --git a/executorlib/interactive/shared.py b/executorlib/task_scheduler/interactive/shared.py similarity index 95% rename from executorlib/interactive/shared.py rename to executorlib/task_scheduler/interactive/shared.py index cc582347..6e82b1a0 100644 --- a/executorlib/interactive/shared.py +++ b/executorlib/task_scheduler/interactive/shared.py @@ -40,7 +40,7 @@ def execute_tasks( this look up for security reasons. So on MacOS it is required to set this option to true init_function (Callable): optional function to preset arguments for functions which are submitted later - cache_directory (str, optional): The directory to store cache files. Defaults to "cache". + cache_directory (str, optional): The directory to store file files. Defaults to "file". queue_join_on_shutdown (bool): Join communication queue when thread is closed. Defaults to True. """ interface = interface_bootup( @@ -131,14 +131,14 @@ def _execute_task_with_cache( cache_directory: str, ): """ - Execute the task in the task_dict by communicating it via the interface using the cache in the cache directory. + Execute the task in the task_dict by communicating it via the interface using the file in the file directory. Args: interface (SocketInterface): socket interface for zmq communication task_dict (dict): task submitted to the executor as dictionary. This dictionary has the following keys {"fn": Callable, "args": (), "kwargs": {}, "resource_dict": {}} future_queue (Queue): Queue for receiving new tasks. - cache_directory (str): The directory to store cache files. + cache_directory (str): The directory to store file files. """ from executorlib.standalone.hdf import dump, get_output @@ -149,10 +149,10 @@ def _execute_task_with_cache( resource_dict=task_dict.get("resource_dict", {}), ) os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) - file_name = os.path.join(cache_directory, task_key, "cache.h5out") + file_name = os.path.join(cache_directory, task_key, "file.h5out") if not ( task_key in os.listdir(cache_directory) - and "cache.h5out" in os.listdir(os.path.join(cache_directory, task_key)) + and "file.h5out" in os.listdir(os.path.join(cache_directory, task_key)) ): f = task_dict.pop("future") if f.set_running_or_notify_cancel(): diff --git a/executorlib/interactive/slurmspawner.py b/executorlib/task_scheduler/interactive/slurmspawner.py similarity index 100% rename from executorlib/interactive/slurmspawner.py rename to executorlib/task_scheduler/interactive/slurmspawner.py diff --git a/notebooks/1-single-node.ipynb b/notebooks/1-single-node.ipynb index 91fd7349..60689144 100644 --- a/notebooks/1-single-node.ipynb +++ b/notebooks/1-single-node.ipynb @@ -530,7 +530,7 @@ ], "source": [ "%%time\n", - "with SingleNodeExecutor(cache_directory=\"./cache\") as exe:\n", + "with SingleNodeExecutor(cache_directory=\"./file\") as exe:\n", " future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)]\n", " print([f.result() for f in future_lst])" ] @@ -563,7 +563,7 @@ ], "source": [ "%%time\n", - "with SingleNodeExecutor(cache_directory=\"./cache\") as exe:\n", + "with SingleNodeExecutor(cache_directory=\"./file\") as exe:\n", " future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)]\n", " print([f.result() for f in future_lst])" ] @@ -664,7 +664,7 @@ "import pandas\n", "from executorlib import get_cache_data\n", "\n", - "df = pandas.DataFrame(get_cache_data(cache_directory=\"./cache\"))\n", + "df = pandas.DataFrame(get_cache_data(cache_directory=\"./file\"))\n", "df" ] }, @@ -694,7 +694,7 @@ "import os\n", "import shutil\n", "\n", - "cache_dir = \"./cache\"\n", + "cache_dir = \"./file\"\n", "if os.path.exists(cache_dir):\n", " print(os.listdir(cache_dir))\n", " try:\n", diff --git a/notebooks/2-hpc-cluster.ipynb b/notebooks/2-hpc-cluster.ipynb index ec3d7007..8bc85182 100644 --- a/notebooks/2-hpc-cluster.ipynb +++ b/notebooks/2-hpc-cluster.ipynb @@ -1 +1,194 @@ -{"metadata":{"kernelspec":{"display_name":"Flux","language":"python","name":"flux"},"language_info":{"codemirror_mode":{"name":"ipython","version":3},"file_extension":".py","mimetype":"text/x-python","name":"python","nbconvert_exporter":"python","pygments_lexer":"ipython3","version":"3.12.9"}},"nbformat_minor":5,"nbformat":4,"cells":[{"id":"ddf66f38-dc4a-4306-8b1c-b923fdb76922","cell_type":"markdown","source":"# HPC Cluster Executor\nIn contrast to the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) and the [HPC Job Executor](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html) the HPC Submission Executors do not communicate via the [zero message queue](https://zeromq.org) but instead store the python functions on the file system and uses the job scheduler to handle the dependencies of the Python functions. Consequently, the block allocation `block_allocation` and the init function `init_function` are not available in the HPC Cluster Executors. At the same time it is possible to close the Python process which created the `Executor`, wait until the execution of the submitted Python functions is completed and afterwards reload the results from the cache.\n\nInternally the HPC submission mode is using the [Python simple queuing system adatper (pysqa)](https://pysqa.readthedocs.io) to connect to HPC job schedulers and the [h5py](https://www.h5py.org) package for serializing the Python functions to store them on the file system. Both packages are optional dependency of executorlib. The installation of the [pysqa](https://pysqa.readthedocs.io) package and the [h5py](https://www.h5py.org) package are covered in the installation section. ","metadata":{}},{"id":"d56862a6-8279-421d-a090-7ca2a3c4d416","cell_type":"markdown","source":"## SLURM\nThe [Simple Linux Utility for Resource Management (SLURM)](https://slurm.schedmd.com) job scheduler is currently the most commonly used job scheduler for HPC clusters. In the HPC submission mode executorlib internally uses the [sbatch](https://slurm.schedmd.com/sbatch.html) command this is in contrast to the [HPC allocatiom mode] which internally uses the [srun](https://slurm.schedmd.com/srun.html) command. \n\nThe connection to the job scheduler is based on the [Python simple queuing system adatper (pysqa)](https://pysqa.readthedocs.io). It provides a default configuration for most commonly used job schedulers including SLURM, in addition it is also possible to provide the submission template as part of the resource dictionary `resource_dict` or via the path to the configuration directory with the `pysqa_config_directory` parameter. All three options are covered in more detail on the [pysqa documentation](https://pysqa.readthedocs.io).","metadata":{}},{"id":"db7760e8-35a6-4a1c-8b0f-410b536c3835","cell_type":"markdown","source":"```python\nfrom executorlib import SlurmClusterExecutor\n```","metadata":{}},{"id":"b20913f3-59e4-418c-a399-866124f8e497","cell_type":"markdown","source":"In comparison to the [SingleNodeExecutor](https://executorlib.readthedocs.io/en/latest/1-single-node.html), the only parameter which is changed in the `SlurmClusterExecutor` is the requirement to specify the cache directory using the `cache_directory=\"./cache\"`. The rest of the syntax remains exactly the same, to simplify the up-scaling of simulation workflows.","metadata":{}},{"id":"0b8f3b77-6199-4736-9f28-3058c5230777","cell_type":"markdown","source":"```python\nwith SlurmClusterExecutor(cache_directory=\"./cache\") as exe:\n future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)]\n print([f.result() for f in future_lst])\n```","metadata":{}},{"id":"37bef7ac-ce3e-4d8a-b848-b1474c370bca","cell_type":"markdown","source":"Specific parameters for `SlurmClusterExecutor` like the maximum run time `\"run_time_max\"`, the maximum memory `\"memory_max\"` or the submission template for the job submission script `\"submission_template\"` can be specified as part of the resource dictionary. Again it is possible to specify the resource dictonary `resource_dicionary` either for each function in the `submit()` function or during the initialization of the `SlurmClusterExecutor`.","metadata":{}},{"id":"658781de-f222-4235-8c26-b0f77a0831b3","cell_type":"markdown","source":"```python\nsubmission_template = \"\"\"\\\n#!/bin/bash\n#SBATCH --output=time.out\n#SBATCH --job-name={{job_name}}\n#SBATCH --chdir={{working_directory}}\n#SBATCH --get-user-env=L\n#SBATCH --partition={{partition}}\n{%- if run_time_max %}\n#SBATCH --time={{ [1, run_time_max // 60]|max }}\n{%- endif %}\n{%- if dependency %}\n#SBATCH --dependency=afterok:{{ dependency | join(',') }}\n{%- endif %}\n{%- if memory_max %}\n#SBATCH --mem={{memory_max}}G\n{%- endif %}\n#SBATCH --cpus-per-task={{cores}}\n\n{{command}}\n\"\"\"\n\nwith SlurmClusterExecutor(cache_directory=\"./cache\") as exe:\n future = exe.submit(\n sum, [4, 4], \n resource_dict={\n \"submission_template\": submission_template, \n \"run_time_max\": 180, # in seconds \n })\n print(future.result())\n```","metadata":{}},{"id":"f7ad9c97-7743-4f87-9344-4299b2b31a56","cell_type":"markdown","source":"With these options executorlib in combination with the SLURM job scheduler provides a lot flexibility to configure the submission of Python functions depending on the specific configuration of the job scheduler. ","metadata":{}},{"id":"2a814efb-2fbc-41ba-98df-cf121d19ea66","cell_type":"markdown","source":"## Flux\nWhile most HPC job schedulers require extensive configuration before they can be tested, the [flux framework](http://flux-framework.org) can be installed with the conda package manager, as explained in the [installation section](https://executorlib.readthedocs.io/en/latest/installation.html#alternative-installations). This simple installation makes the flux framework especially suitable for demonstrations, testing and continous integration. So below a number of features for the HPC submission mode are demonstrated based on the example of the [flux framework](http://flux-framework.org) still the same applies to other job schedulers like SLURM introduced above.","metadata":{}},{"id":"29d7aa18-357e-416e-805c-1322b59abec1","cell_type":"markdown","source":"### Dependencies\nAs already demonstrated for the [SingleNodeExecutor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) the `Executor` classes from executorlib are capable of resolving the dependencies of serial functions, when [concurrent futures Future](https://docs.python.org/3/library/concurrent.futures.html#future-objects) objects are used as inputs for subsequent function calls. For the case of the HPC submission these dependencies are communicated to the job scheduler, which allows to stop the Python process which created the `Executor` class, wait until the execution of the submitted Python functions is completed and afterwards restart the Python process for the `Executor` class and reload the calculation results from the cache defined by the `cache_directory` parameter.","metadata":{}},{"id":"0f7fc37a-1248-492d-91ab-9db1d737eaee","cell_type":"code","source":"def add_funct(a, b):\n return a + b","metadata":{"trusted":false},"outputs":[],"execution_count":1},{"id":"ae308683-6083-4e78-afc2-bff6c6dc297b","cell_type":"code","source":"from executorlib import FluxClusterExecutor\n\nwith FluxClusterExecutor(cache_directory=\"./cache\") as exe:\n future = 0\n for i in range(4, 8):\n future = exe.submit(add_funct, i, future)\n print(future.result())","metadata":{"trusted":false},"outputs":[{"name":"stdout","output_type":"stream","text":"22\n"}],"execution_count":2},{"id":"ca75cb6c-c50f-4bee-9b09-d8d29d6c263b","cell_type":"markdown","source":"### Resource Assignment\nIn analogy to the [SingleNodeExecutor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) the resource assignment for the `FluxClusterExecutor` is handled by either including the resource dictionary parameter `resource_dict` in the initialization of the `FluxClusterExecutor` class or in every call of the `submit()` function.\n\nBelow this is demonstrated once for the assignment of multiple CPU cores for the execution of a Python function which internally uses the message passing interface (MPI) via the [mpi4py](https://mpi4py.readthedocs.io) package.","metadata":{}},{"id":"eded3a0f-e54f-44f6-962f-eedde4bd2158","cell_type":"code","source":"def calc(i):\n from mpi4py import MPI\n\n size = MPI.COMM_WORLD.Get_size()\n rank = MPI.COMM_WORLD.Get_rank()\n return i, size, rank\n","metadata":{"trusted":false},"outputs":[],"execution_count":3},{"id":"669b05df-3cb2-4f69-9d94-8b2442745ebb","cell_type":"code","source":"with FluxClusterExecutor(cache_directory=\"./cache\") as exe:\n fs = exe.submit(calc, 3, resource_dict={\"cores\": 2})\n print(fs.result())","metadata":{"trusted":false},"outputs":[{"name":"stdout","output_type":"stream","text":"[(3, 2, 0), (3, 2, 1)]\n"}],"execution_count":4},{"id":"d91499d7-5c6c-4c10-b7b7-bfc4b87ddaa8","cell_type":"markdown","source":"Beyond CPU cores and threads which were previously also introduced for the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) the HPC Cluster Executors also provide the option to select the available accelerator cards or GPUs, by specifying the `\"gpus_per_core\"` parameter in the resource dictionary `resource_dict`. For demonstration we create a Python function which reads the GPU device IDs and submit it to the `FluxClusterExecutor` class:\n```python\ndef get_available_gpus():\n import socket\n from tensorflow.python.client import device_lib\n local_device_protos = device_lib.list_local_devices()\n return [\n (x.name, x.physical_device_desc, socket.gethostname()) \n for x in local_device_protos if x.device_type == 'GPU'\n ]\n```\n\n```python\nwith FluxClusterExecutor(\n cache_directory=\"./cache\",\n resource_dict={\"gpus_per_core\": 1}\n) as exe:\n fs_1 = exe.submit(get_available_gpus)\n fs_2 = exe.submit(get_available_gpus)\n print(fs_1.result(), fs_2.result())\n```","metadata":{}},{"id":"3f47fd34-04d1-42a7-bb06-6821dc99a648","cell_type":"markdown","source":"### Cleaning Cache\nFinally, as the HPC Cluster Executors leverage the file system to communicate serialized Python functions, it is important to clean up the cache directory specified by the `cache_directory` parameter once the results of the submitted Python functions are no longer needed. The serialized Python functions are stored in binary format using the [cloudpickle](https://github.com/cloudpipe/cloudpickle) library for serialization. This format is design for caching but not for long-term storage. The user is responsible for the long-term storage of their data.","metadata":{}},{"id":"f537b4f6-cc98-43da-8aca-94a823bcbcbd","cell_type":"code","source":"import os\nimport shutil\n\ncache_dir = \"./cache\"\nif os.path.exists(cache_dir):\n print(os.listdir(cache_dir))\n try:\n shutil.rmtree(cache_dir)\n except OSError:\n pass","metadata":{"trusted":false},"outputs":[{"name":"stdout","output_type":"stream","text":"['add_functdce32a0e7f6eac9e4e19fec335b79726', 'calc76234667eef65c770fecf54645ef8ada', 'add_functee0545e0d3edb8a4a6ceb6d5ae712d39', 'add_funct3263a1038c0d088677685b6eccd9f7b7', 'add_funct6034ded02bdb3ff97695f3a94455ca4d']\n"}],"execution_count":5}]} \ No newline at end of file +{ + "metadata": { + "kernelspec": { + "display_name": "Flux", + "language": "python", + "name": "flux" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.12.9" + } + }, + "nbformat_minor": 5, + "nbformat": 4, + "cells": [ + { + "id": "ddf66f38-dc4a-4306-8b1c-b923fdb76922", + "cell_type": "markdown", + "source": "# HPC Cluster Executor\nIn contrast to the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) and the [HPC Job Executor](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html) the HPC Submission Executors do not communicate via the [zero message queue](https://zeromq.org) but instead store the python functions on the file system and uses the job scheduler to handle the dependencies of the Python functions. Consequently, the block allocation `block_allocation` and the init function `init_function` are not available in the HPC Cluster Executors. At the same time it is possible to close the Python process which created the `Executor`, wait until the execution of the submitted Python functions is completed and afterwards reload the results from the cache.\n\nInternally the HPC submission mode is using the [Python simple queuing system adatper (pysqa)](https://pysqa.readthedocs.io) to connect to HPC job schedulers and the [h5py](https://www.h5py.org) package for serializing the Python functions to store them on the file system. Both packages are optional dependency of executorlib. The installation of the [pysqa](https://pysqa.readthedocs.io) package and the [h5py](https://www.h5py.org) package are covered in the installation section. ", + "metadata": {} + }, + { + "id": "d56862a6-8279-421d-a090-7ca2a3c4d416", + "cell_type": "markdown", + "source": "## SLURM\nThe [Simple Linux Utility for Resource Management (SLURM)](https://slurm.schedmd.com) job scheduler is currently the most commonly used job scheduler for HPC clusters. In the HPC submission mode executorlib internally uses the [sbatch](https://slurm.schedmd.com/sbatch.html) command this is in contrast to the [HPC allocatiom mode] which internally uses the [srun](https://slurm.schedmd.com/srun.html) command. \n\nThe connection to the job scheduler is based on the [Python simple queuing system adatper (pysqa)](https://pysqa.readthedocs.io). It provides a default configuration for most commonly used job schedulers including SLURM, in addition it is also possible to provide the submission template as part of the resource dictionary `resource_dict` or via the path to the configuration directory with the `pysqa_config_directory` parameter. All three options are covered in more detail on the [pysqa documentation](https://pysqa.readthedocs.io).", + "metadata": {} + }, + { + "id": "db7760e8-35a6-4a1c-8b0f-410b536c3835", + "cell_type": "markdown", + "source": "```python\nfrom executorlib import SlurmClusterExecutor\n```", + "metadata": {} + }, + { + "id": "b20913f3-59e4-418c-a399-866124f8e497", + "cell_type": "markdown", + "source": "In comparison to the [SingleNodeExecutor](https://executorlib.readthedocs.io/en/latest/1-single-node.html), the only parameter which is changed in the `SlurmClusterExecutor` is the requirement to specify the cache directory using the `cache_directory=\"./cache\"`. The rest of the syntax remains exactly the same, to simplify the up-scaling of simulation workflows.", + "metadata": {} + }, + { + "id": "0b8f3b77-6199-4736-9f28-3058c5230777", + "cell_type": "markdown", + "source": "```python\nwith SlurmClusterExecutor(cache_directory=\"./cache\") as exe:\n future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)]\n print([f.result() for f in future_lst])\n```", + "metadata": {} + }, + { + "id": "37bef7ac-ce3e-4d8a-b848-b1474c370bca", + "cell_type": "markdown", + "source": "Specific parameters for `SlurmClusterExecutor` like the maximum run time `\"run_time_max\"`, the maximum memory `\"memory_max\"` or the submission template for the job submission script `\"submission_template\"` can be specified as part of the resource dictionary. Again it is possible to specify the resource dictonary `resource_dicionary` either for each function in the `submit()` function or during the initialization of the `SlurmClusterExecutor`.", + "metadata": {} + }, + { + "id": "658781de-f222-4235-8c26-b0f77a0831b3", + "cell_type": "markdown", + "source": "```python\nsubmission_template = \"\"\"\\\n#!/bin/bash\n#SBATCH --output=time.out\n#SBATCH --job-name={{job_name}}\n#SBATCH --chdir={{working_directory}}\n#SBATCH --get-user-env=L\n#SBATCH --partition={{partition}}\n{%- if run_time_max %}\n#SBATCH --time={{ [1, run_time_max // 60]|max }}\n{%- endif %}\n{%- if dependency %}\n#SBATCH --dependency=afterok:{{ dependency | join(',') }}\n{%- endif %}\n{%- if memory_max %}\n#SBATCH --mem={{memory_max}}G\n{%- endif %}\n#SBATCH --cpus-per-task={{cores}}\n\n{{command}}\n\"\"\"\n\nwith SlurmClusterExecutor(cache_directory=\"./cache\") as exe:\n future = exe.submit(\n sum, [4, 4], \n resource_dict={\n \"submission_template\": submission_template, \n \"run_time_max\": 180, # in seconds \n })\n print(future.result())\n```", + "metadata": {} + }, + { + "id": "f7ad9c97-7743-4f87-9344-4299b2b31a56", + "cell_type": "markdown", + "source": "With these options executorlib in combination with the SLURM job scheduler provides a lot flexibility to configure the submission of Python functions depending on the specific configuration of the job scheduler. ", + "metadata": {} + }, + { + "id": "2a814efb-2fbc-41ba-98df-cf121d19ea66", + "cell_type": "markdown", + "source": "## Flux\nWhile most HPC job schedulers require extensive configuration before they can be tested, the [flux framework](http://flux-framework.org) can be installed with the conda package manager, as explained in the [installation section](https://executorlib.readthedocs.io/en/latest/installation.html#alternative-installations). This simple installation makes the flux framework especially suitable for demonstrations, testing and continous integration. So below a number of features for the HPC submission mode are demonstrated based on the example of the [flux framework](http://flux-framework.org) still the same applies to other job schedulers like SLURM introduced above.", + "metadata": {} + }, + { + "id": "29d7aa18-357e-416e-805c-1322b59abec1", + "cell_type": "markdown", + "source": "### Dependencies\nAs already demonstrated for the [SingleNodeExecutor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) the `Executor` classes from executorlib are capable of resolving the dependencies of serial functions, when [concurrent futures Future](https://docs.python.org/3/library/concurrent.futures.html#future-objects) objects are used as inputs for subsequent function calls. For the case of the HPC submission these dependencies are communicated to the job scheduler, which allows to stop the Python process which created the `Executor` class, wait until the execution of the submitted Python functions is completed and afterwards restart the Python process for the `Executor` class and reload the calculation results from the cache defined by the `cache_directory` parameter.", + "metadata": {} + }, + { + "id": "0f7fc37a-1248-492d-91ab-9db1d737eaee", + "cell_type": "code", + "source": "def add_funct(a, b):\n return a + b", + "metadata": { + "trusted": false + }, + "outputs": [], + "execution_count": 1 + }, + { + "id": "ae308683-6083-4e78-afc2-bff6c6dc297b", + "cell_type": "code", + "source": [ + "from executorlib import FluxClusterExecutor\n", + "\n", + "with FluxClusterExecutor(cache_directory=\"./file\") as exe:\n", + " future = 0\n", + " for i in range(4, 8):\n", + " future = exe.submit(add_funct, i, future)\n", + " print(future.result())" + ], + "metadata": { + "trusted": false + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "22\n" + } + ], + "execution_count": 2 + }, + { + "id": "ca75cb6c-c50f-4bee-9b09-d8d29d6c263b", + "cell_type": "markdown", + "source": "### Resource Assignment\nIn analogy to the [SingleNodeExecutor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) the resource assignment for the `FluxClusterExecutor` is handled by either including the resource dictionary parameter `resource_dict` in the initialization of the `FluxClusterExecutor` class or in every call of the `submit()` function.\n\nBelow this is demonstrated once for the assignment of multiple CPU cores for the execution of a Python function which internally uses the message passing interface (MPI) via the [mpi4py](https://mpi4py.readthedocs.io) package.", + "metadata": {} + }, + { + "id": "eded3a0f-e54f-44f6-962f-eedde4bd2158", + "cell_type": "code", + "source": "def calc(i):\n from mpi4py import MPI\n\n size = MPI.COMM_WORLD.Get_size()\n rank = MPI.COMM_WORLD.Get_rank()\n return i, size, rank\n", + "metadata": { + "trusted": false + }, + "outputs": [], + "execution_count": 3 + }, + { + "id": "669b05df-3cb2-4f69-9d94-8b2442745ebb", + "cell_type": "code", + "source": [ + "with FluxClusterExecutor(cache_directory=\"./file\") as exe:\n", + " fs = exe.submit(calc, 3, resource_dict={\"cores\": 2})\n", + " print(fs.result())" + ], + "metadata": { + "trusted": false + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "[(3, 2, 0), (3, 2, 1)]\n" + } + ], + "execution_count": 4 + }, + { + "id": "d91499d7-5c6c-4c10-b7b7-bfc4b87ddaa8", + "cell_type": "markdown", + "source": "Beyond CPU cores and threads which were previously also introduced for the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) the HPC Cluster Executors also provide the option to select the available accelerator cards or GPUs, by specifying the `\"gpus_per_core\"` parameter in the resource dictionary `resource_dict`. For demonstration we create a Python function which reads the GPU device IDs and submit it to the `FluxClusterExecutor` class:\n```python\ndef get_available_gpus():\n import socket\n from tensorflow.python.client import device_lib\n local_device_protos = device_lib.list_local_devices()\n return [\n (x.name, x.physical_device_desc, socket.gethostname()) \n for x in local_device_protos if x.device_type == 'GPU'\n ]\n```\n\n```python\nwith FluxClusterExecutor(\n cache_directory=\"./cache\",\n resource_dict={\"gpus_per_core\": 1}\n) as exe:\n fs_1 = exe.submit(get_available_gpus)\n fs_2 = exe.submit(get_available_gpus)\n print(fs_1.result(), fs_2.result())\n```", + "metadata": {} + }, + { + "id": "3f47fd34-04d1-42a7-bb06-6821dc99a648", + "cell_type": "markdown", + "source": "### Cleaning Cache\nFinally, as the HPC Cluster Executors leverage the file system to communicate serialized Python functions, it is important to clean up the cache directory specified by the `cache_directory` parameter once the results of the submitted Python functions are no longer needed. The serialized Python functions are stored in binary format using the [cloudpickle](https://github.com/cloudpipe/cloudpickle) library for serialization. This format is design for caching but not for long-term storage. The user is responsible for the long-term storage of their data.", + "metadata": {} + }, + { + "id": "f537b4f6-cc98-43da-8aca-94a823bcbcbd", + "cell_type": "code", + "source": [ + "import os\n", + "import shutil\n", + "\n", + "cache_dir = \"./file\"\n", + "if os.path.exists(cache_dir):\n", + " print(os.listdir(cache_dir))\n", + " try:\n", + " shutil.rmtree(cache_dir)\n", + " except OSError:\n", + " pass" + ], + "metadata": { + "trusted": false + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "['add_functdce32a0e7f6eac9e4e19fec335b79726', 'calc76234667eef65c770fecf54645ef8ada', 'add_functee0545e0d3edb8a4a6ceb6d5ae712d39', 'add_funct3263a1038c0d088677685b6eccd9f7b7', 'add_funct6034ded02bdb3ff97695f3a94455ca4d']\n" + } + ], + "execution_count": 5 + } + ] +} diff --git a/notebooks/3-hpc-job.ipynb b/notebooks/3-hpc-job.ipynb index e21fdaf8..dd4e5f3b 100644 --- a/notebooks/3-hpc-job.ipynb +++ b/notebooks/3-hpc-job.ipynb @@ -1 +1,322 @@ -{"metadata":{"kernelspec":{"name":"flux","display_name":"Flux","language":"python"},"language_info":{"name":"python","version":"3.12.9","mimetype":"text/x-python","codemirror_mode":{"name":"ipython","version":3},"pygments_lexer":"ipython3","nbconvert_exporter":"python","file_extension":".py"}},"nbformat_minor":5,"nbformat":4,"cells":[{"id":"87c3425d-5abe-4e0b-a948-e371808c322c","cell_type":"markdown","source":"# HPC Job Executor\nIn contrast to the [HPC Cluster Executor](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html) which submits individual Python functions to HPC job schedulers, the HPC Job Executors take a given job allocation of the HPC job scheduler and executes Python functions with the resources available in this job allocation. In this regard it is similar to the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) as it communicates with the individual Python processes using the [zero message queue](https://zeromq.org/), still it is more advanced as it can access the computational resources of all compute nodes of the given HPC job allocation and also provides the option to assign GPUs as accelerators for parallel execution.\n\nAvailable Functionality: \n* Submit Python functions with the [submit() function or the map() function](https://executorlib.readthedocs.io/en/latest/1-single-node.html#basic-functionality).\n* Support for parallel execution, either using the [message passing interface (MPI)](https://executorlib.readthedocs.io/en/latest/1-single-node.html#mpi-parallel-functions), [thread based parallelism](https://executorlib.readthedocs.io/en/latest/1-single-node.html#thread-parallel-functions) or by [assigning dedicated GPUs](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html#resource-assignment) to selected Python functions. All these resources assignments are handled via the [resource dictionary parameter resource_dict](https://executorlib.readthedocs.io/en/latest/trouble_shooting.html#resource-dictionary).\n* Performance optimization features, like [block allocation](https://executorlib.readthedocs.io/en/latest/1-single-node.html#block-allocation), [dependency resolution](https://executorlib.readthedocs.io/en/latest/1-single-node.html#dependencies) and [caching](https://executorlib.readthedocs.io/en/latest/1-single-node.html#cache).\n\nThe only parameter the user has to change is the `backend` parameter. ","metadata":{}},{"id":"8c788b9f-6b54-4ce0-a864-4526b7f6f170","cell_type":"markdown","source":"## SLURM\nWith the [Simple Linux Utility for Resource Management (SLURM)](https://slurm.schedmd.com/) currently being the most commonly used job scheduler, executorlib provides an interface to submit Python functions to SLURM. Internally, this is based on the [srun](https://slurm.schedmd.com/srun.html) command of the SLURM scheduler, which creates job steps in a given allocation. Given that all resource requests in SLURM are communicated via a central database a large number of submitted Python functions and resulting job steps can slow down the performance of SLURM. To address this limitation it is recommended to install the hierarchical job scheduler [flux](https://flux-framework.org/) in addition to SLURM, to use flux for distributing the resources within a given allocation. This configuration is discussed in more detail below in the section [SLURM with flux](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html#slurm-with-flux).","metadata":{}},{"id":"133b751f-0925-4d11-99f0-3f8dd9360b54","cell_type":"code","source":"from executorlib import SlurmJobExecutor","metadata":{"trusted":true},"outputs":[],"execution_count":1},{"id":"9b74944e-2ccd-4cb0-860a-d876310ea870","cell_type":"markdown","source":"```python\nwith SlurmAllocationExecutor() as exe:\n future = exe.submit(sum, [1, 1])\n print(future.result())\n```","metadata":{}},{"id":"36e2d68a-f093-4082-933a-d95bfe7a60c6","cell_type":"markdown","source":"## SLURM with Flux \nAs discussed in the installation section it is important to select the [flux](https://flux-framework.org/) version compatible to the installation of a given HPC cluster. Which GPUs are available? Who manufactured these GPUs? Does the HPC use [mpich](https://www.mpich.org/) or [OpenMPI](https://www.open-mpi.org/) or one of their commercial counter parts like cray MPI or intel MPI? Depending on the configuration different installation options can be choosen, as explained in the [installation section](https://executorlib.readthedocs.io/en/latest/installation.html#hpc-job-executor).\n\nAfterwards flux can be started in an [sbatch](https://slurm.schedmd.com/sbatch.html) submission script using:\n```\nsrun flux start python \n```\nIn this Python script `` the `\"flux_allocation\"` backend can be used.","metadata":{}},{"id":"68be70c3-af18-4165-862d-7022d35bf9e4","cell_type":"markdown","source":"### Resource Assignment\nIndependent of the selected Executor [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html), [HPC Cluster Executor](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html) or HPC job executor the assignment of the computational resources remains the same. They can either be specified in the `submit()` function by adding the resource dictionary parameter [resource_dict](https://executorlib.readthedocs.io/en/latest/trouble_shooting.html#resource-dictionary) or alternatively during the initialization of the `Executor` class by adding the resource dictionary parameter [resource_dict](https://executorlib.readthedocs.io/en/latest/trouble_shooting.html#resource-dictionary) there.\n\nThis functionality of executorlib is commonly used to rewrite individual Python functions to use MPI while the rest of the Python program remains serial.","metadata":{}},{"id":"8a2c08df-cfea-4783-ace6-68fcd8ebd330","cell_type":"code","source":"def calc_mpi(i):\n from mpi4py import MPI\n\n size = MPI.COMM_WORLD.Get_size()\n rank = MPI.COMM_WORLD.Get_rank()\n return i, size, rank","metadata":{"trusted":true},"outputs":[],"execution_count":2},{"id":"715e0c00-7b17-40bb-bd55-b0e097bfef07","cell_type":"markdown","source":"Depending on the choice of MPI version, it is recommended to specify the pmi standard which [flux](https://flux-framework.org/) should use internally for the resource assignment. For example for OpenMPI >=5 `\"pmix\"` is the recommended pmi standard.","metadata":{}},{"id":"5802c7d7-9560-4909-9d30-a915a91ac0a1","cell_type":"code","source":"from executorlib import FluxJobExecutor\n\nwith FluxJobExecutor(flux_executor_pmi_mode=\"pmix\") as exe:\n fs = exe.submit(calc_mpi, 3, resource_dict={\"cores\": 2})\n print(fs.result())","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":"[(3, 2, 0), (3, 2, 1)]\n"}],"execution_count":3},{"id":"da862425-08b6-4ced-999f-89a74e85f410","cell_type":"markdown","source":"### Block Allocation\nThe block allocation for the HPC allocation mode follows the same implementation as the [block allocation for the Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html#block-allocation). It starts by defining the initialization function `init_function()` which returns a dictionary which is internally used to look up input parameters for Python functions submitted to the `FluxJobExecutor` class. Commonly this functionality is used to store large data objects inside the Python process created for the block allocation, rather than reloading these Python objects for each submitted function.","metadata":{}},{"id":"cdc742c0-35f7-47ff-88c0-1b0dbeabe51b","cell_type":"code","source":"def init_function():\n return {\"j\": 4, \"k\": 3, \"l\": 2}","metadata":{"trusted":true},"outputs":[],"execution_count":4},{"id":"5ddf8343-ab2c-4469-ac9f-ee568823d4ad","cell_type":"code","source":"def calc_with_preload(i, j, k):\n return i + j + k","metadata":{"trusted":true},"outputs":[],"execution_count":5},{"id":"0da13efa-1941-416f-b9e6-bba15b5cdfa2","cell_type":"code","source":"with FluxJobExecutor(\n flux_executor_pmi_mode=\"pmix\",\n max_workers=2,\n init_function=init_function,\n block_allocation=True,\n) as exe:\n fs = exe.submit(calc_with_preload, 2, j=5)\n print(fs.result())","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":"10\n"}],"execution_count":6},{"id":"82f3b947-e662-4a0d-b590-9475e0b4f7dd","cell_type":"markdown","source":"In this example the parameter `k` is used from the dataset created by the initialization function while the parameters `i` and `j` are specified by the call of the `submit()` function. \n\nWhen using the block allocation mode, it is recommended to set either the maxium number of workers using the `max_workers` parameter or the maximum number of CPU cores using the `max_cores` parameter to prevent oversubscribing the available resources. ","metadata":{}},{"id":"8ced8359-8ecb-480b-966b-b85d8446d85c","cell_type":"markdown","source":"### Dependencies\nPython functions with rather different computational resource requirements should not be merged into a single function. So to able to execute a series of Python functions which each depend on the output of the previous Python function executorlib internally handles the dependencies based on the [concurrent futures future](https://docs.python.org/3/library/concurrent.futures.html#future-objects) objects from the Python standard library. This implementation is independent of the selected backend and works for HPC allocation mode just like explained in the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html#dependencies) section.","metadata":{}},{"id":"bd26d97b-46fd-4786-9ad1-1e534b31bf36","cell_type":"code","source":"def add_funct(a, b):\n return a + b","metadata":{"trusted":true},"outputs":[],"execution_count":7},{"id":"1a2d440f-3cfc-4ff2-b74d-e21823c65f69","cell_type":"code","source":"with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\") as exe:\n future = 0\n for i in range(1, 4):\n future = exe.submit(add_funct, i, future)\n print(future.result())","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":"6\n"}],"execution_count":8},{"id":"f526c2bf-fdf5-463b-a955-020753138415","cell_type":"markdown","source":"### Caching\nFinally, also the caching is available for HPC allocation mode, in analogy to the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html#cache). Again this functionality is not designed to identify function calls with the same parameters, but rather provides the option to reload previously cached results even after the Python processes which contained the executorlib `Executor` class is closed. As the cache is stored on the file system, this option can decrease the performance of executorlib. Consequently the caching option should primarily be used during the prototyping phase.","metadata":{}},{"id":"dcba63e0-72f5-49d1-ab04-2092fccc1c47","cell_type":"code","source":"with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\", cache_directory=\"./cache\") as exe:\n future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)]\n print([f.result() for f in future_lst])","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":"[2, 4, 6]\n"}],"execution_count":9},{"id":"c3958a14-075b-4c10-9729-d1c559a9231c","cell_type":"code","source":"import os\nimport shutil\n\ncache_dir = \"./cache\"\nif os.path.exists(cache_dir):\n print(os.listdir(cache_dir))\n try:\n shutil.rmtree(cache_dir)\n except OSError:\n pass","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":"['sum0d968285d17368d1c34ea7392309bcc5', 'sum6270955d7c8022a0c1027aafaee64439', 'sum0102e33bb2921ae07a3bbe3db5d3dec9']\n"}],"execution_count":10},{"id":"c24ca82d-60bd-4fb9-a082-bf9a81e838bf","cell_type":"markdown","source":"### Nested executors\nThe hierarchical nature of the [flux](https://flux-framework.org/) job scheduler allows the creation of additional executorlib Executors inside the functions submitted to the Executor. This hierarchy can be beneficial to separate the logic to saturate the available computational resources. ","metadata":{}},{"id":"06fb2d1f-65fc-4df6-9402-5e9837835484","cell_type":"code","source":"def calc_nested():\n from executorlib import FluxJobExecutor\n\n with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\") as exe:\n fs = exe.submit(sum, [1, 1])\n return fs.result()","metadata":{"trusted":true},"outputs":[],"execution_count":11},{"id":"89b7d0fd-5978-4913-a79a-f26cc8047445","cell_type":"code","source":"with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\", flux_executor_nesting=True) as exe:\n fs = exe.submit(calc_nested)\n print(fs.result())","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":"2\n"}],"execution_count":12},{"id":"34a8c690-ca5a-41d1-b38f-c67eff085750","cell_type":"markdown","source":"### Resource Monitoring\nFor debugging it is commonly helpful to keep track of the computational resources. [flux](https://flux-framework.org/) provides a number of features to analyse the resource utilization, so here only the two most commonly used ones are introduced. Starting with the option to list all the resources available in a given allocation with the `flux resource list` command:","metadata":{}},{"id":"7481eb0a-a41b-4d46-bb48-b4db299fcd86","cell_type":"code","source":"! flux resource list","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":" STATE NNODES NCORES NGPUS NODELIST\n free 1 24 0 jupyter-pyiron-executorlib-slqpe5j5\n allocated 0 0 0 \n down 0 0 0 \n"}],"execution_count":13},{"id":"08d98134-a0e0-4841-be82-e09e1af29e7f","cell_type":"markdown","source":"Followed by the list of jobs which were executed in a given flux session. This can be retrieved using the `flux jobs -a` command:","metadata":{}},{"id":"1ee6e147-f53a-4526-8ed0-fd036f2ee6bf","cell_type":"code","source":"! flux jobs -a","metadata":{"trusted":true},"outputs":[{"name":"stdout","output_type":"stream","text":" JOBID USER NAME ST NTASKS NNODES TIME INFO\n\u001b[01;32m Æ’5c7bbtT jovyan flux CD 1 1 4.227s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’47tyNMM jovyan python CD 1 1 2.982s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’47sVP51 jovyan python CD 1 1 2.902s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’427vAfR jovyan python CD 1 1 2.986s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’3jUnECw jovyan python CD 1 1 0.455s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’3P1G9Uj jovyan python CD 1 1 0.643s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’38sQze3 jovyan python CD 1 1 0.606s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’2HHH1w5 jovyan python CD 1 1 1.665s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’2EvtA1M jovyan python CD 1 1 1.734s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m\u001b[01;32m Æ’V4qQRd jovyan python CD 2 1 1.463s jupyter-pyiron-executorlib-slqpe5j5\n\u001b[0;0m"}],"execution_count":14},{"id":"021f165b-27cc-4676-968b-cbcfd1f0210a","cell_type":"markdown","source":"## Flux\nWhile the number of HPC clusters which use [flux](https://flux-framework.org/) as primary job scheduler is currently still limited the setup and functionality provided by executorlib for running [SLURM with flux](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html#slurm-with-flux) also applies to HPCs which use [flux](https://flux-framework.org/) as primary job scheduler.","metadata":{}},{"id":"04f03ebb-3f9e-4738-b9d2-5cb0db9b63c3","cell_type":"code","source":"","metadata":{"trusted":true},"outputs":[],"execution_count":null}]} \ No newline at end of file +{ + "metadata": { + "kernelspec": { + "name": "flux", + "display_name": "Flux", + "language": "python" + }, + "language_info": { + "name": "python", + "version": "3.12.9", + "mimetype": "text/x-python", + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "pygments_lexer": "ipython3", + "nbconvert_exporter": "python", + "file_extension": ".py" + } + }, + "nbformat_minor": 5, + "nbformat": 4, + "cells": [ + { + "id": "87c3425d-5abe-4e0b-a948-e371808c322c", + "cell_type": "markdown", + "source": "# HPC Job Executor\nIn contrast to the [HPC Cluster Executor](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html) which submits individual Python functions to HPC job schedulers, the HPC Job Executors take a given job allocation of the HPC job scheduler and executes Python functions with the resources available in this job allocation. In this regard it is similar to the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) as it communicates with the individual Python processes using the [zero message queue](https://zeromq.org/), still it is more advanced as it can access the computational resources of all compute nodes of the given HPC job allocation and also provides the option to assign GPUs as accelerators for parallel execution.\n\nAvailable Functionality: \n* Submit Python functions with the [submit() function or the map() function](https://executorlib.readthedocs.io/en/latest/1-single-node.html#basic-functionality).\n* Support for parallel execution, either using the [message passing interface (MPI)](https://executorlib.readthedocs.io/en/latest/1-single-node.html#mpi-parallel-functions), [thread based parallelism](https://executorlib.readthedocs.io/en/latest/1-single-node.html#thread-parallel-functions) or by [assigning dedicated GPUs](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html#resource-assignment) to selected Python functions. All these resources assignments are handled via the [resource dictionary parameter resource_dict](https://executorlib.readthedocs.io/en/latest/trouble_shooting.html#resource-dictionary).\n* Performance optimization features, like [block allocation](https://executorlib.readthedocs.io/en/latest/1-single-node.html#block-allocation), [dependency resolution](https://executorlib.readthedocs.io/en/latest/1-single-node.html#dependencies) and [caching](https://executorlib.readthedocs.io/en/latest/1-single-node.html#cache).\n\nThe only parameter the user has to change is the `backend` parameter. ", + "metadata": {} + }, + { + "id": "8c788b9f-6b54-4ce0-a864-4526b7f6f170", + "cell_type": "markdown", + "source": "## SLURM\nWith the [Simple Linux Utility for Resource Management (SLURM)](https://slurm.schedmd.com/) currently being the most commonly used job scheduler, executorlib provides an interface to submit Python functions to SLURM. Internally, this is based on the [srun](https://slurm.schedmd.com/srun.html) command of the SLURM scheduler, which creates job steps in a given allocation. Given that all resource requests in SLURM are communicated via a central database a large number of submitted Python functions and resulting job steps can slow down the performance of SLURM. To address this limitation it is recommended to install the hierarchical job scheduler [flux](https://flux-framework.org/) in addition to SLURM, to use flux for distributing the resources within a given allocation. This configuration is discussed in more detail below in the section [SLURM with flux](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html#slurm-with-flux).", + "metadata": {} + }, + { + "id": "133b751f-0925-4d11-99f0-3f8dd9360b54", + "cell_type": "code", + "source": "from executorlib import SlurmJobExecutor", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": 1 + }, + { + "id": "9b74944e-2ccd-4cb0-860a-d876310ea870", + "cell_type": "markdown", + "source": "```python\nwith SlurmAllocationExecutor() as exe:\n future = exe.submit(sum, [1, 1])\n print(future.result())\n```", + "metadata": {} + }, + { + "id": "36e2d68a-f093-4082-933a-d95bfe7a60c6", + "cell_type": "markdown", + "source": "## SLURM with Flux \nAs discussed in the installation section it is important to select the [flux](https://flux-framework.org/) version compatible to the installation of a given HPC cluster. Which GPUs are available? Who manufactured these GPUs? Does the HPC use [mpich](https://www.mpich.org/) or [OpenMPI](https://www.open-mpi.org/) or one of their commercial counter parts like cray MPI or intel MPI? Depending on the configuration different installation options can be choosen, as explained in the [installation section](https://executorlib.readthedocs.io/en/latest/installation.html#hpc-job-executor).\n\nAfterwards flux can be started in an [sbatch](https://slurm.schedmd.com/sbatch.html) submission script using:\n```\nsrun flux start python \n```\nIn this Python script `` the `\"flux_allocation\"` backend can be used.", + "metadata": {} + }, + { + "id": "68be70c3-af18-4165-862d-7022d35bf9e4", + "cell_type": "markdown", + "source": "### Resource Assignment\nIndependent of the selected Executor [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html), [HPC Cluster Executor](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html) or HPC job executor the assignment of the computational resources remains the same. They can either be specified in the `submit()` function by adding the resource dictionary parameter [resource_dict](https://executorlib.readthedocs.io/en/latest/trouble_shooting.html#resource-dictionary) or alternatively during the initialization of the `Executor` class by adding the resource dictionary parameter [resource_dict](https://executorlib.readthedocs.io/en/latest/trouble_shooting.html#resource-dictionary) there.\n\nThis functionality of executorlib is commonly used to rewrite individual Python functions to use MPI while the rest of the Python program remains serial.", + "metadata": {} + }, + { + "id": "8a2c08df-cfea-4783-ace6-68fcd8ebd330", + "cell_type": "code", + "source": "def calc_mpi(i):\n from mpi4py import MPI\n\n size = MPI.COMM_WORLD.Get_size()\n rank = MPI.COMM_WORLD.Get_rank()\n return i, size, rank", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": 2 + }, + { + "id": "715e0c00-7b17-40bb-bd55-b0e097bfef07", + "cell_type": "markdown", + "source": "Depending on the choice of MPI version, it is recommended to specify the pmi standard which [flux](https://flux-framework.org/) should use internally for the resource assignment. For example for OpenMPI >=5 `\"pmix\"` is the recommended pmi standard.", + "metadata": {} + }, + { + "id": "5802c7d7-9560-4909-9d30-a915a91ac0a1", + "cell_type": "code", + "source": "from executorlib import FluxJobExecutor\n\nwith FluxJobExecutor(flux_executor_pmi_mode=\"pmix\") as exe:\n fs = exe.submit(calc_mpi, 3, resource_dict={\"cores\": 2})\n print(fs.result())", + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "[(3, 2, 0), (3, 2, 1)]\n" + } + ], + "execution_count": 3 + }, + { + "id": "da862425-08b6-4ced-999f-89a74e85f410", + "cell_type": "markdown", + "source": "### Block Allocation\nThe block allocation for the HPC allocation mode follows the same implementation as the [block allocation for the Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html#block-allocation). It starts by defining the initialization function `init_function()` which returns a dictionary which is internally used to look up input parameters for Python functions submitted to the `FluxJobExecutor` class. Commonly this functionality is used to store large data objects inside the Python process created for the block allocation, rather than reloading these Python objects for each submitted function.", + "metadata": {} + }, + { + "id": "cdc742c0-35f7-47ff-88c0-1b0dbeabe51b", + "cell_type": "code", + "source": "def init_function():\n return {\"j\": 4, \"k\": 3, \"l\": 2}", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": 4 + }, + { + "id": "5ddf8343-ab2c-4469-ac9f-ee568823d4ad", + "cell_type": "code", + "source": "def calc_with_preload(i, j, k):\n return i + j + k", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": 5 + }, + { + "id": "0da13efa-1941-416f-b9e6-bba15b5cdfa2", + "cell_type": "code", + "source": "with FluxJobExecutor(\n flux_executor_pmi_mode=\"pmix\",\n max_workers=2,\n init_function=init_function,\n block_allocation=True,\n) as exe:\n fs = exe.submit(calc_with_preload, 2, j=5)\n print(fs.result())", + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "10\n" + } + ], + "execution_count": 6 + }, + { + "id": "82f3b947-e662-4a0d-b590-9475e0b4f7dd", + "cell_type": "markdown", + "source": "In this example the parameter `k` is used from the dataset created by the initialization function while the parameters `i` and `j` are specified by the call of the `submit()` function. \n\nWhen using the block allocation mode, it is recommended to set either the maxium number of workers using the `max_workers` parameter or the maximum number of CPU cores using the `max_cores` parameter to prevent oversubscribing the available resources. ", + "metadata": {} + }, + { + "id": "8ced8359-8ecb-480b-966b-b85d8446d85c", + "cell_type": "markdown", + "source": "### Dependencies\nPython functions with rather different computational resource requirements should not be merged into a single function. So to able to execute a series of Python functions which each depend on the output of the previous Python function executorlib internally handles the dependencies based on the [concurrent futures future](https://docs.python.org/3/library/concurrent.futures.html#future-objects) objects from the Python standard library. This implementation is independent of the selected backend and works for HPC allocation mode just like explained in the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html#dependencies) section.", + "metadata": {} + }, + { + "id": "bd26d97b-46fd-4786-9ad1-1e534b31bf36", + "cell_type": "code", + "source": "def add_funct(a, b):\n return a + b", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": 7 + }, + { + "id": "1a2d440f-3cfc-4ff2-b74d-e21823c65f69", + "cell_type": "code", + "source": "with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\") as exe:\n future = 0\n for i in range(1, 4):\n future = exe.submit(add_funct, i, future)\n print(future.result())", + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "6\n" + } + ], + "execution_count": 8 + }, + { + "id": "f526c2bf-fdf5-463b-a955-020753138415", + "cell_type": "markdown", + "source": "### Caching\nFinally, also the caching is available for HPC allocation mode, in analogy to the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html#cache). Again this functionality is not designed to identify function calls with the same parameters, but rather provides the option to reload previously cached results even after the Python processes which contained the executorlib `Executor` class is closed. As the cache is stored on the file system, this option can decrease the performance of executorlib. Consequently the caching option should primarily be used during the prototyping phase.", + "metadata": {} + }, + { + "id": "dcba63e0-72f5-49d1-ab04-2092fccc1c47", + "cell_type": "code", + "source": [ + "with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\", cache_directory=\"./file\") as exe:\n", + " future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)]\n", + " print([f.result() for f in future_lst])" + ], + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "[2, 4, 6]\n" + } + ], + "execution_count": 9 + }, + { + "id": "c3958a14-075b-4c10-9729-d1c559a9231c", + "cell_type": "code", + "source": [ + "import os\n", + "import shutil\n", + "\n", + "cache_dir = \"./file\"\n", + "if os.path.exists(cache_dir):\n", + " print(os.listdir(cache_dir))\n", + " try:\n", + " shutil.rmtree(cache_dir)\n", + " except OSError:\n", + " pass" + ], + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "['sum0d968285d17368d1c34ea7392309bcc5', 'sum6270955d7c8022a0c1027aafaee64439', 'sum0102e33bb2921ae07a3bbe3db5d3dec9']\n" + } + ], + "execution_count": 10 + }, + { + "id": "c24ca82d-60bd-4fb9-a082-bf9a81e838bf", + "cell_type": "markdown", + "source": "### Nested executors\nThe hierarchical nature of the [flux](https://flux-framework.org/) job scheduler allows the creation of additional executorlib Executors inside the functions submitted to the Executor. This hierarchy can be beneficial to separate the logic to saturate the available computational resources. ", + "metadata": {} + }, + { + "id": "06fb2d1f-65fc-4df6-9402-5e9837835484", + "cell_type": "code", + "source": "def calc_nested():\n from executorlib import FluxJobExecutor\n\n with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\") as exe:\n fs = exe.submit(sum, [1, 1])\n return fs.result()", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": 11 + }, + { + "id": "89b7d0fd-5978-4913-a79a-f26cc8047445", + "cell_type": "code", + "source": "with FluxJobExecutor(flux_executor_pmi_mode=\"pmix\", flux_executor_nesting=True) as exe:\n fs = exe.submit(calc_nested)\n print(fs.result())", + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": "2\n" + } + ], + "execution_count": 12 + }, + { + "id": "34a8c690-ca5a-41d1-b38f-c67eff085750", + "cell_type": "markdown", + "source": "### Resource Monitoring\nFor debugging it is commonly helpful to keep track of the computational resources. [flux](https://flux-framework.org/) provides a number of features to analyse the resource utilization, so here only the two most commonly used ones are introduced. Starting with the option to list all the resources available in a given allocation with the `flux resource list` command:", + "metadata": {} + }, + { + "id": "7481eb0a-a41b-4d46-bb48-b4db299fcd86", + "cell_type": "code", + "source": "! flux resource list", + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": " STATE NNODES NCORES NGPUS NODELIST\n free 1 24 0 jupyter-pyiron-executorlib-slqpe5j5\n allocated 0 0 0 \n down 0 0 0 \n" + } + ], + "execution_count": 13 + }, + { + "id": "08d98134-a0e0-4841-be82-e09e1af29e7f", + "cell_type": "markdown", + "source": "Followed by the list of jobs which were executed in a given flux session. This can be retrieved using the `flux jobs -a` command:", + "metadata": {} + }, + { + "id": "1ee6e147-f53a-4526-8ed0-fd036f2ee6bf", + "cell_type": "code", + "source": "! flux jobs -a", + "metadata": { + "trusted": true + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": " JOBID USER NAME ST NTASKS NNODES TIME INFO\n\u001B[01;32m Æ’5c7bbtT jovyan flux CD 1 1 4.227s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’47tyNMM jovyan python CD 1 1 2.982s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’47sVP51 jovyan python CD 1 1 2.902s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’427vAfR jovyan python CD 1 1 2.986s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’3jUnECw jovyan python CD 1 1 0.455s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’3P1G9Uj jovyan python CD 1 1 0.643s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’38sQze3 jovyan python CD 1 1 0.606s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’2HHH1w5 jovyan python CD 1 1 1.665s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’2EvtA1M jovyan python CD 1 1 1.734s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m\u001B[01;32m Æ’V4qQRd jovyan python CD 2 1 1.463s jupyter-pyiron-executorlib-slqpe5j5\n\u001B[0;0m" + } + ], + "execution_count": 14 + }, + { + "id": "021f165b-27cc-4676-968b-cbcfd1f0210a", + "cell_type": "markdown", + "source": "## Flux\nWhile the number of HPC clusters which use [flux](https://flux-framework.org/) as primary job scheduler is currently still limited the setup and functionality provided by executorlib for running [SLURM with flux](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html#slurm-with-flux) also applies to HPCs which use [flux](https://flux-framework.org/) as primary job scheduler.", + "metadata": {} + }, + { + "id": "04f03ebb-3f9e-4738-b9d2-5cb0db9b63c3", + "cell_type": "code", + "source": "", + "metadata": { + "trusted": true + }, + "outputs": [], + "execution_count": null + } + ] +} diff --git a/tests/test_base_executor_queue.py b/tests/test_base_executor_queue.py index d27840dd..fccd1833 100644 --- a/tests/test_base_executor_queue.py +++ b/tests/test_base_executor_queue.py @@ -2,7 +2,7 @@ from queue import Queue import unittest -from executorlib.base.executor import cancel_items_in_queue +from executorlib.executor.base import cancel_items_in_queue class TestQueue(unittest.TestCase): diff --git a/tests/test_cache_backend_execute.py b/tests/test_cache_backend_execute.py index 0dce06c6..32bc05df 100644 --- a/tests/test_cache_backend_execute.py +++ b/tests/test_cache_backend_execute.py @@ -5,8 +5,8 @@ try: - from executorlib.cache.backend import backend_execute_task_in_file - from executorlib.cache.shared import _check_task_output, FutureItem + from executorlib.task_scheduler.file import backend_execute_task_in_file + from executorlib.task_scheduler.file import _check_task_output, FutureItem from executorlib.standalone.hdf import dump, get_runtime from executorlib.standalone.serialize import serialize_funct_h5 @@ -28,14 +28,14 @@ def get_error(a): ) class TestSharedFunctions(unittest.TestCase): def test_execute_function_mixed(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=my_funct, fn_args=[1], fn_kwargs={"b": 2}, ) - file_name = os.path.join(cache_directory, task_key, "cache.h5in") + file_name = os.path.join(cache_directory, task_key, "file.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -46,24 +46,24 @@ def test_execute_function_mixed(self): self.assertTrue(future_obj.done()) self.assertEqual(future_obj.result(), 3) self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "cache.h5out") + file_name=os.path.join(cache_directory, task_key, "file.h5out") ) self.assertTrue(future_file_obj.done()) self.assertEqual(future_file_obj.result(), 3) def test_execute_function_args(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=my_funct, fn_args=[1, 2], fn_kwargs={}, ) - file_name = os.path.join(cache_directory, task_key, "cache.h5in") + file_name = os.path.join(cache_directory, task_key, "file.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -74,24 +74,24 @@ def test_execute_function_args(self): self.assertTrue(future_obj.done()) self.assertEqual(future_obj.result(), 3) self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "cache.h5out") + file_name=os.path.join(cache_directory, task_key, "file.h5out") ) self.assertTrue(future_file_obj.done()) self.assertEqual(future_file_obj.result(), 3) def test_execute_function_kwargs(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=my_funct, fn_args=[], fn_kwargs={"a": 1, "b": 2}, ) - file_name = os.path.join(cache_directory, task_key, "cache.h5in") + file_name = os.path.join(cache_directory, task_key, "file.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -102,24 +102,24 @@ def test_execute_function_kwargs(self): self.assertTrue(future_obj.done()) self.assertEqual(future_obj.result(), 3) self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "cache.h5out") + file_name=os.path.join(cache_directory, task_key, "file.h5out") ) self.assertTrue(future_file_obj.done()) self.assertEqual(future_file_obj.result(), 3) def test_execute_function_error(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=get_error, fn_args=[], fn_kwargs={"a": 1}, ) - file_name = os.path.join(cache_directory, task_key, "cache.h5in") + file_name = os.path.join(cache_directory, task_key, "file.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -131,16 +131,16 @@ def test_execute_function_error(self): with self.assertRaises(ValueError): future_obj.result() self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "cache.h5out") + file_name=os.path.join(cache_directory, task_key, "file.h5out") ) self.assertTrue(future_file_obj.done()) with self.assertRaises(ValueError): future_file_obj.result() def tearDown(self): - if os.path.exists("cache"): - shutil.rmtree("cache") + if os.path.exists("file"): + shutil.rmtree("file") diff --git a/tests/test_cache_fileexecutor_mpi.py b/tests/test_cache_fileexecutor_mpi.py index 18817f84..db4edb31 100644 --- a/tests/test_cache_fileexecutor_mpi.py +++ b/tests/test_cache_fileexecutor_mpi.py @@ -3,11 +3,11 @@ import shutil import unittest -from executorlib.cache.subprocess_spawner import execute_in_subprocess +from executorlib.task_scheduler.file import execute_in_subprocess try: - from executorlib.cache.task_scheduler import FileTaskScheduler + from executorlib.task_scheduler.file import FileTaskScheduler skip_h5py_test = False except ImportError: @@ -40,5 +40,5 @@ def test_executor(self): self.assertTrue(fs1.done()) def tearDown(self): - if os.path.exists("cache"): - shutil.rmtree("cache") + if os.path.exists("file"): + shutil.rmtree("file") diff --git a/tests/test_cache_fileexecutor_serial.py b/tests/test_cache_fileexecutor_serial.py index c660eb33..aa1c300d 100644 --- a/tests/test_cache_fileexecutor_serial.py +++ b/tests/test_cache_fileexecutor_serial.py @@ -5,14 +5,14 @@ import unittest from threading import Thread -from executorlib.cache.subprocess_spawner import ( +from executorlib.task_scheduler.file import ( execute_in_subprocess, terminate_subprocess, ) try: - from executorlib.cache.task_scheduler import FileTaskScheduler, create_file_executor - from executorlib.cache.shared import execute_tasks_h5 + from executorlib.task_scheduler.file import FileTaskScheduler, create_file_executor + from executorlib.task_scheduler.file import execute_tasks_h5 skip_h5py_test = False except ImportError: @@ -93,7 +93,7 @@ def test_executor_function(self): "resource_dict": {}, } ) - cache_dir = os.path.abspath("cache") + cache_dir = os.path.abspath("file") os.makedirs(cache_dir, exist_ok=True) process = Thread( target=execute_tasks_h5, @@ -134,7 +134,7 @@ def test_executor_function_dependence_kwargs(self): "resource_dict": {}, } ) - cache_dir = os.path.abspath("cache") + cache_dir = os.path.abspath("file") os.makedirs(cache_dir, exist_ok=True) process = Thread( target=execute_tasks_h5, @@ -175,7 +175,7 @@ def test_executor_function_dependence_args(self): "resource_dict": {}, } ) - cache_dir = os.path.abspath("cache") + cache_dir = os.path.abspath("file") os.makedirs(cache_dir, exist_ok=True) process = Thread( target=execute_tasks_h5, @@ -203,5 +203,5 @@ def test_execute_in_subprocess_errors(self): execute_in_subprocess(file_name=__file__, command=[], backend="flux") def tearDown(self): - if os.path.exists("cache"): - shutil.rmtree("cache") + if os.path.exists("file"): + shutil.rmtree("file") diff --git a/tests/test_fluxclusterexecutor.py b/tests/test_fluxclusterexecutor.py index 73deb661..28262726 100644 --- a/tests/test_fluxclusterexecutor.py +++ b/tests/test_fluxclusterexecutor.py @@ -33,9 +33,9 @@ def mpi_funct(i): class TestCacheExecutorPysqa(unittest.TestCase): def test_executor(self): with FluxClusterExecutor( - resource_dict={"cores": 2, "cwd": "cache"}, + resource_dict={"cores": 2, "cwd": "file"}, block_allocation=False, - cache_directory="cache", + cache_directory="file", ) as exe: cloudpickle_register(ind=1) fs1 = exe.submit(mpi_funct, 1) @@ -44,5 +44,5 @@ def test_executor(self): self.assertTrue(fs1.done()) def tearDown(self): - if os.path.exists("cache"): - shutil.rmtree("cache") + if os.path.exists("file"): + shutil.rmtree("file") diff --git a/tests/test_fluxjobexecutor.py b/tests/test_fluxjobexecutor.py index d5505e68..976a9d61 100644 --- a/tests/test_fluxjobexecutor.py +++ b/tests/test_fluxjobexecutor.py @@ -8,7 +8,7 @@ try: import flux.job - from executorlib.interactive.fluxspawner import FluxPythonSpawner + from executorlib.task_scheduler.interactive import FluxPythonSpawner skip_flux_test = "FLUX_URI" not in os.environ pmi = os.environ.get("EXECUTORLIB_PMIX", None) diff --git a/tests/test_fluxjobexecutor_plot.py b/tests/test_fluxjobexecutor_plot.py index 74b920f6..a4ac643d 100644 --- a/tests/test_fluxjobexecutor_plot.py +++ b/tests/test_fluxjobexecutor_plot.py @@ -10,7 +10,7 @@ try: import pygraphviz import flux.job - from executorlib.interactive.fluxspawner import FluxPythonSpawner + from executorlib.task_scheduler.interactive import FluxPythonSpawner skip_graphviz_flux_test = "FLUX_URI" not in os.environ except ImportError: diff --git a/tests/test_fluxpythonspawner.py b/tests/test_fluxpythonspawner.py index 5c5c52eb..44e06ec7 100644 --- a/tests/test_fluxpythonspawner.py +++ b/tests/test_fluxpythonspawner.py @@ -5,14 +5,14 @@ import numpy as np -from executorlib.interactive.shared import execute_tasks -from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.shared import execute_tasks +from executorlib.task_scheduler.interactive import BlockAllocationTaskScheduler from executorlib.standalone.serialize import cloudpickle_register try: import flux.job - from executorlib.interactive.fluxspawner import FluxPythonSpawner + from executorlib.task_scheduler.interactive import FluxPythonSpawner skip_flux_test = "FLUX_URI" not in os.environ pmi = os.environ.get("EXECUTORLIB_PMIX", None) diff --git a/tests/test_interactive_dependencies.py b/tests/test_interactive_dependencies.py index e662cfa0..7cbb1148 100644 --- a/tests/test_interactive_dependencies.py +++ b/tests/test_interactive_dependencies.py @@ -5,7 +5,7 @@ import numpy as np -from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive import BlockAllocationTaskScheduler from executorlib.standalone.interactive.spawner import MpiExecSpawner diff --git a/tests/test_interactive_slurmspawner.py b/tests/test_interactive_slurmspawner.py index 147e871d..4f595181 100644 --- a/tests/test_interactive_slurmspawner.py +++ b/tests/test_interactive_slurmspawner.py @@ -1,8 +1,8 @@ import unittest -from executorlib.interactive.slurmspawner import generate_slurm_command +from executorlib.task_scheduler.interactive.slurmspawner import generate_slurm_command try: - from executorlib.cache.queue_spawner import _pysqa_execute_command + from executorlib.task_scheduler.file import _pysqa_execute_command skip_pysqa_test = False except ImportError: diff --git a/tests/test_mpiexecspawner.py b/tests/test_mpiexecspawner.py index f00f7ecf..6796a321 100644 --- a/tests/test_mpiexecspawner.py +++ b/tests/test_mpiexecspawner.py @@ -7,11 +7,11 @@ import numpy as np -from executorlib.base.task_scheduler import TaskSchedulerBase +from executorlib.task_scheduler.base import TaskSchedulerBase from executorlib.standalone.interactive.spawner import MpiExecSpawner -from executorlib.interactive.shared import execute_tasks -from executorlib.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.interactive.onetoone import OneProcessTaskScheduler +from executorlib.task_scheduler.interactive.shared import execute_tasks +from executorlib.task_scheduler.interactive import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive import OneProcessTaskScheduler from executorlib.standalone.interactive.backend import call_funct from executorlib.standalone.serialize import cloudpickle_register @@ -503,7 +503,7 @@ def test_execute_task_parallel(self): class TestFuturePoolCache(unittest.TestCase): def tearDown(self): - shutil.rmtree("./cache") + shutil.rmtree("./file") @unittest.skipIf( skip_h5py_test, "h5py is not installed, so the h5py tests are skipped." @@ -519,7 +519,7 @@ def test_execute_task_cache(self): cores=1, openmpi_oversubscribe=False, spawner=MpiExecSpawner, - cache_directory="./cache", + cache_directory="./file", ) self.assertEqual(f.result(), 1) q.join() @@ -538,6 +538,6 @@ def test_execute_task_cache_failed_no_argument(self): cores=1, openmpi_oversubscribe=False, spawner=MpiExecSpawner, - cache_directory="./cache", + cache_directory="./file", ) q.join() diff --git a/tests/test_singlenodeexecutor_cache.py b/tests/test_singlenodeexecutor_cache.py index bded9cab..6a81caaf 100644 --- a/tests/test_singlenodeexecutor_cache.py +++ b/tests/test_singlenodeexecutor_cache.py @@ -22,7 +22,7 @@ def get_error(a): ) class TestCacheFunctions(unittest.TestCase): def test_cache_data(self): - cache_directory = "./cache" + cache_directory = "./file" with SingleNodeExecutor(cache_directory=cache_directory) as exe: future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)] result_lst = [f.result() for f in future_lst] @@ -42,7 +42,7 @@ def test_cache_error(self): print(f.result()) def tearDown(self): - if os.path.exists("cache"): - shutil.rmtree("cache") + if os.path.exists("file"): + shutil.rmtree("file") if os.path.exists("cache_error"): shutil.rmtree("cache_error") diff --git a/tests/test_singlenodeexecutor_dependencies.py b/tests/test_singlenodeexecutor_dependencies.py index 4e4780d3..cd453bce 100644 --- a/tests/test_singlenodeexecutor_dependencies.py +++ b/tests/test_singlenodeexecutor_dependencies.py @@ -5,8 +5,8 @@ from threading import Thread from executorlib import SingleNodeExecutor -from executorlib.interfaces.single import create_single_node_executor -from executorlib.interactive.dependency import _execute_tasks_with_dependencies +from executorlib.executor.single import create_single_node_executor +from executorlib.task_scheduler.interactive.dependency import _execute_tasks_with_dependencies from executorlib.standalone.serialize import cloudpickle_register from executorlib.standalone.interactive.spawner import MpiExecSpawner diff --git a/tests/test_singlenodeexecutor_mpi.py b/tests/test_singlenodeexecutor_mpi.py index ca7abe39..7b238959 100644 --- a/tests/test_singlenodeexecutor_mpi.py +++ b/tests/test_singlenodeexecutor_mpi.py @@ -83,7 +83,7 @@ def test_errors(self): class TestExecutorBackendCache(unittest.TestCase): def tearDown(self): - shutil.rmtree("./cache") + shutil.rmtree("./file") @unittest.skipIf( skip_mpi4py_test, "mpi4py is not installed, so the mpi4py tests are skipped." @@ -93,7 +93,7 @@ def test_meta_executor_parallel_cache(self): max_workers=2, resource_dict={"cores": 2}, block_allocation=True, - cache_directory="./cache", + cache_directory="./file", ) as exe: cloudpickle_register(ind=1) time_1 = time.time() diff --git a/tests/test_singlenodeexecutor_shell_executor.py b/tests/test_singlenodeexecutor_shell_executor.py index 7a69cf22..df97ecd2 100644 --- a/tests/test_singlenodeexecutor_shell_executor.py +++ b/tests/test_singlenodeexecutor_shell_executor.py @@ -5,7 +5,7 @@ from executorlib import SingleNodeExecutor from executorlib.standalone.serialize import cloudpickle_register -from executorlib.interactive.shared import execute_tasks +from executorlib.task_scheduler.interactive.shared import execute_tasks from executorlib.standalone.interactive.spawner import MpiExecSpawner diff --git a/tests/test_singlenodeexecutor_shell_interactive.py b/tests/test_singlenodeexecutor_shell_interactive.py index 6b4e2d77..0adc54bf 100644 --- a/tests/test_singlenodeexecutor_shell_interactive.py +++ b/tests/test_singlenodeexecutor_shell_interactive.py @@ -6,7 +6,7 @@ from executorlib import SingleNodeExecutor from executorlib.standalone.serialize import cloudpickle_register -from executorlib.interactive.shared import execute_tasks +from executorlib.task_scheduler.interactive.shared import execute_tasks from executorlib.standalone.interactive.spawner import MpiExecSpawner diff --git a/tests/test_standalone_hdf.py b/tests/test_standalone_hdf.py index 95d1aa6e..a130d112 100644 --- a/tests/test_standalone_hdf.py +++ b/tests/test_standalone_hdf.py @@ -26,7 +26,7 @@ def my_funct(a, b): ) class TestSharedFunctions(unittest.TestCase): def test_hdf_mixed(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_mixed.h5") a = 1 @@ -46,7 +46,7 @@ def test_hdf_mixed(self): self.assertIsNone(output) def test_hdf_args(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_args.h5") a = 1 @@ -63,7 +63,7 @@ def test_hdf_args(self): self.assertIsNone(output) def test_hdf_kwargs(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_kwargs.h5") a = 1 @@ -89,7 +89,7 @@ def test_hdf_kwargs(self): self.assertIsNone(output) def test_hdf_queue_id(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_queue.h5") queue_id = 123 @@ -105,7 +105,7 @@ def test_hdf_queue_id(self): self.assertIsNone(output) def test_hdf_error(self): - cache_directory = os.path.abspath("cache") + cache_directory = os.path.abspath("file") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_error.h5") error = ValueError() @@ -120,5 +120,5 @@ def test_hdf_error(self): self.assertTrue(isinstance(output, error.__class__)) def tearDown(self): - if os.path.exists("cache"): - shutil.rmtree("cache") + if os.path.exists("file"): + shutil.rmtree("file") diff --git a/tests/test_standalone_interactive_backend.py b/tests/test_standalone_interactive_backend.py index 40bda2e1..cfa961af 100644 --- a/tests/test_standalone_interactive_backend.py +++ b/tests/test_standalone_interactive_backend.py @@ -4,7 +4,7 @@ from executorlib.standalone.interactive.backend import parse_arguments from executorlib.standalone.interactive.spawner import MpiExecSpawner -from executorlib.interactive.slurmspawner import SrunSpawner +from executorlib.task_scheduler.interactive.slurmspawner import SrunSpawner class TestParser(unittest.TestCase): From d7bdad738fe880d1ceceb67136b5c480769f06a0 Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Sat, 26 Apr 2025 07:12:05 +0000 Subject: [PATCH 04/14] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- executorlib/executor/flux.py | 8 +++++--- executorlib/executor/single.py | 8 +++++--- executorlib/executor/slurm.py | 13 +++++++++---- executorlib/task_scheduler/file/backend.py | 2 +- executorlib/task_scheduler/file/task_scheduler.py | 12 ++++++------ .../task_scheduler/interactive/blockallocation.py | 4 ++-- .../task_scheduler/interactive/dependency.py | 2 +- executorlib/task_scheduler/interactive/onetoone.py | 2 +- 8 files changed, 30 insertions(+), 21 deletions(-) diff --git a/executorlib/executor/flux.py b/executorlib/executor/flux.py index e45b0f9c..36d584b7 100644 --- a/executorlib/executor/flux.py +++ b/executorlib/executor/flux.py @@ -2,9 +2,6 @@ from typing import Callable, Optional, Union from executorlib.executor.base import ExecutorBase -from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler -from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.inputcheck import ( check_command_line_argument_lst, check_init_function, @@ -14,6 +11,11 @@ check_refresh_rate, validate_number_of_cores, ) +from executorlib.task_scheduler.interactive.blockallocation import ( + BlockAllocationTaskScheduler, +) +from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler with contextlib.suppress(ImportError): from executorlib.task_scheduler.interactive.fluxspawner import ( diff --git a/executorlib/executor/single.py b/executorlib/executor/single.py index 4de30d8b..3435df06 100644 --- a/executorlib/executor/single.py +++ b/executorlib/executor/single.py @@ -1,9 +1,6 @@ from typing import Callable, Optional, Union from executorlib.executor.base import ExecutorBase -from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler -from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.inputcheck import ( check_command_line_argument_lst, check_gpus_per_worker, @@ -13,6 +10,11 @@ validate_number_of_cores, ) from executorlib.standalone.interactive.spawner import MpiExecSpawner +from executorlib.task_scheduler.interactive.blockallocation import ( + BlockAllocationTaskScheduler, +) +from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler class SingleNodeExecutor(ExecutorBase): diff --git a/executorlib/executor/slurm.py b/executorlib/executor/slurm.py index c93062dc..b9705689 100644 --- a/executorlib/executor/slurm.py +++ b/executorlib/executor/slurm.py @@ -1,16 +1,21 @@ from typing import Callable, Optional, Union from executorlib.executor.base import ExecutorBase -from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler -from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler -from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler -from executorlib.task_scheduler.interactive.slurmspawner import SrunSpawner, validate_max_workers from executorlib.standalone.inputcheck import ( check_init_function, check_plot_dependency_graph, check_refresh_rate, validate_number_of_cores, ) +from executorlib.task_scheduler.interactive.blockallocation import ( + BlockAllocationTaskScheduler, +) +from executorlib.task_scheduler.interactive.dependency import DependencyTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler +from executorlib.task_scheduler.interactive.slurmspawner import ( + SrunSpawner, + validate_max_workers, +) class SlurmClusterExecutor(ExecutorBase): diff --git a/executorlib/task_scheduler/file/backend.py b/executorlib/task_scheduler/file/backend.py index 8d54e19f..63fe6ea5 100644 --- a/executorlib/task_scheduler/file/backend.py +++ b/executorlib/task_scheduler/file/backend.py @@ -2,8 +2,8 @@ import time from typing import Any -from executorlib.task_scheduler.file.shared import FutureItem from executorlib.standalone.hdf import dump, load +from executorlib.task_scheduler.file.shared import FutureItem def backend_load_file(file_name: str) -> dict: diff --git a/executorlib/task_scheduler/file/task_scheduler.py b/executorlib/task_scheduler/file/task_scheduler.py index 9993d97f..3a220019 100644 --- a/executorlib/task_scheduler/file/task_scheduler.py +++ b/executorlib/task_scheduler/file/task_scheduler.py @@ -2,12 +2,6 @@ from threading import Thread from typing import Callable, Optional -from executorlib.task_scheduler.base import TaskSchedulerBase -from executorlib.task_scheduler.file.shared import execute_tasks_h5 -from executorlib.task_scheduler.file.subprocess_spawner import ( - execute_in_subprocess, - terminate_subprocess, -) from executorlib.standalone.inputcheck import ( check_executor, check_flux_executor_pmi_mode, @@ -16,6 +10,12 @@ check_max_workers_and_cores, check_nested_flux_executor, ) +from executorlib.task_scheduler.base import TaskSchedulerBase +from executorlib.task_scheduler.file.shared import execute_tasks_h5 +from executorlib.task_scheduler.file.subprocess_spawner import ( + execute_in_subprocess, + terminate_subprocess, +) try: from executorlib.task_scheduler.file.queue_spawner import execute_with_pysqa diff --git a/executorlib/task_scheduler/interactive/blockallocation.py b/executorlib/task_scheduler/interactive/blockallocation.py index 967e5c0d..7f474d6a 100644 --- a/executorlib/task_scheduler/interactive/blockallocation.py +++ b/executorlib/task_scheduler/interactive/blockallocation.py @@ -3,14 +3,14 @@ from threading import Thread from typing import Callable, Optional -from executorlib.task_scheduler.base import TaskSchedulerBase -from executorlib.task_scheduler.interactive.shared import execute_tasks from executorlib.standalone.inputcheck import ( check_resource_dict, check_resource_dict_is_empty, ) from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner from executorlib.standalone.queue import cancel_items_in_queue +from executorlib.task_scheduler.base import TaskSchedulerBase +from executorlib.task_scheduler.interactive.shared import execute_tasks class BlockAllocationTaskScheduler(TaskSchedulerBase): diff --git a/executorlib/task_scheduler/interactive/dependency.py b/executorlib/task_scheduler/interactive/dependency.py index 8e3821bf..07f9f2a3 100644 --- a/executorlib/task_scheduler/interactive/dependency.py +++ b/executorlib/task_scheduler/interactive/dependency.py @@ -4,7 +4,6 @@ from time import sleep from typing import Any, Callable, Optional -from executorlib.task_scheduler.base import TaskSchedulerBase from executorlib.standalone.interactive.arguments import ( check_exception_was_raised, get_exception_lst, @@ -16,6 +15,7 @@ generate_nodes_and_edges, generate_task_hash, ) +from executorlib.task_scheduler.base import TaskSchedulerBase class DependencyTaskScheduler(TaskSchedulerBase): diff --git a/executorlib/task_scheduler/interactive/onetoone.py b/executorlib/task_scheduler/interactive/onetoone.py index 6285a813..d28f014b 100644 --- a/executorlib/task_scheduler/interactive/onetoone.py +++ b/executorlib/task_scheduler/interactive/onetoone.py @@ -2,9 +2,9 @@ from threading import Thread from typing import Optional +from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner from executorlib.task_scheduler.base import TaskSchedulerBase from executorlib.task_scheduler.interactive.shared import execute_tasks -from executorlib.standalone.interactive.spawner import BaseSpawner, MpiExecSpawner class OneProcessTaskScheduler(TaskSchedulerBase): From 22eed9d06e4b79b427c3ab4dd03f242a351a4c13 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:14:04 +0200 Subject: [PATCH 05/14] fix backends --- executorlib/backend/cache_parallel.py | 2 +- executorlib/backend/cache_serial.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executorlib/backend/cache_parallel.py b/executorlib/backend/cache_parallel.py index 1e47c7ed..61cab654 100644 --- a/executorlib/backend/cache_parallel.py +++ b/executorlib/backend/cache_parallel.py @@ -4,7 +4,7 @@ import cloudpickle -from executorlib.task_scheduler.file import backend_load_file, backend_write_file +from executorlib.task_scheduler.file.backend import backend_load_file, backend_write_file def main() -> None: diff --git a/executorlib/backend/cache_serial.py b/executorlib/backend/cache_serial.py index 1a0b487d..c627fe8d 100644 --- a/executorlib/backend/cache_serial.py +++ b/executorlib/backend/cache_serial.py @@ -1,6 +1,6 @@ import sys -from executorlib.task_scheduler.file import backend_execute_task_in_file +from executorlib.task_scheduler.file.backend import backend_execute_task_in_file if __name__ == "__main__": backend_execute_task_in_file(file_name=sys.argv[1]) From ddbb6d270b105865c17686d31188b4e1c463cbc8 Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Sat, 26 Apr 2025 07:14:36 +0000 Subject: [PATCH 06/14] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- executorlib/backend/cache_parallel.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/executorlib/backend/cache_parallel.py b/executorlib/backend/cache_parallel.py index 61cab654..9b1d25d8 100644 --- a/executorlib/backend/cache_parallel.py +++ b/executorlib/backend/cache_parallel.py @@ -4,7 +4,10 @@ import cloudpickle -from executorlib.task_scheduler.file.backend import backend_load_file, backend_write_file +from executorlib.task_scheduler.file.backend import ( + backend_load_file, + backend_write_file, +) def main() -> None: From 0f054e317900799280ccebc7d4c532a1d3ceaf27 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:22:21 +0200 Subject: [PATCH 07/14] fix imports --- tests/test_base_executor_queue.py | 2 +- tests/test_cache_backend_execute.py | 4 ++-- tests/test_cache_fileexecutor_mpi.py | 4 ++-- tests/test_cache_fileexecutor_serial.py | 6 +++--- tests/test_fluxjobexecutor.py | 2 +- tests/test_fluxjobexecutor_plot.py | 2 +- tests/test_fluxpythonspawner.py | 4 ++-- tests/test_interactive_dependencies.py | 2 +- tests/test_interactive_slurmspawner.py | 2 +- tests/test_mpiexecspawner.py | 4 ++-- 10 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/test_base_executor_queue.py b/tests/test_base_executor_queue.py index fccd1833..7e6f40b3 100644 --- a/tests/test_base_executor_queue.py +++ b/tests/test_base_executor_queue.py @@ -2,7 +2,7 @@ from queue import Queue import unittest -from executorlib.executor.base import cancel_items_in_queue +from executorlib.standalone.queue import cancel_items_in_queue class TestQueue(unittest.TestCase): diff --git a/tests/test_cache_backend_execute.py b/tests/test_cache_backend_execute.py index 32bc05df..5e934404 100644 --- a/tests/test_cache_backend_execute.py +++ b/tests/test_cache_backend_execute.py @@ -5,8 +5,8 @@ try: - from executorlib.task_scheduler.file import backend_execute_task_in_file - from executorlib.task_scheduler.file import _check_task_output, FutureItem + from executorlib.task_scheduler.file.backend import backend_execute_task_in_file + from executorlib.task_scheduler.file.shared import _check_task_output, FutureItem from executorlib.standalone.hdf import dump, get_runtime from executorlib.standalone.serialize import serialize_funct_h5 diff --git a/tests/test_cache_fileexecutor_mpi.py b/tests/test_cache_fileexecutor_mpi.py index db4edb31..71efbf12 100644 --- a/tests/test_cache_fileexecutor_mpi.py +++ b/tests/test_cache_fileexecutor_mpi.py @@ -3,11 +3,11 @@ import shutil import unittest -from executorlib.task_scheduler.file import execute_in_subprocess +from executorlib.task_scheduler.file.subprocess_spawner import execute_in_subprocess try: - from executorlib.task_scheduler.file import FileTaskScheduler + from executorlib.task_scheduler.file.task_scheduler import FileTaskScheduler skip_h5py_test = False except ImportError: diff --git a/tests/test_cache_fileexecutor_serial.py b/tests/test_cache_fileexecutor_serial.py index aa1c300d..8479dccf 100644 --- a/tests/test_cache_fileexecutor_serial.py +++ b/tests/test_cache_fileexecutor_serial.py @@ -5,14 +5,14 @@ import unittest from threading import Thread -from executorlib.task_scheduler.file import ( +from executorlib.task_scheduler.file.subprocess_spawner import ( execute_in_subprocess, terminate_subprocess, ) try: - from executorlib.task_scheduler.file import FileTaskScheduler, create_file_executor - from executorlib.task_scheduler.file import execute_tasks_h5 + from executorlib.task_scheduler.file.task_scheduler import FileTaskScheduler, create_file_executor + from executorlib.task_scheduler.file.shared import execute_tasks_h5 skip_h5py_test = False except ImportError: diff --git a/tests/test_fluxjobexecutor.py b/tests/test_fluxjobexecutor.py index 976a9d61..8cfa8e9a 100644 --- a/tests/test_fluxjobexecutor.py +++ b/tests/test_fluxjobexecutor.py @@ -8,7 +8,7 @@ try: import flux.job - from executorlib.task_scheduler.interactive import FluxPythonSpawner + from executorlib.task_scheduler.interactive.fluxspawner import FluxPythonSpawner skip_flux_test = "FLUX_URI" not in os.environ pmi = os.environ.get("EXECUTORLIB_PMIX", None) diff --git a/tests/test_fluxjobexecutor_plot.py b/tests/test_fluxjobexecutor_plot.py index a4ac643d..1b71239e 100644 --- a/tests/test_fluxjobexecutor_plot.py +++ b/tests/test_fluxjobexecutor_plot.py @@ -10,7 +10,7 @@ try: import pygraphviz import flux.job - from executorlib.task_scheduler.interactive import FluxPythonSpawner + from executorlib.task_scheduler.interactive.fluxspawner import FluxPythonSpawner skip_graphviz_flux_test = "FLUX_URI" not in os.environ except ImportError: diff --git a/tests/test_fluxpythonspawner.py b/tests/test_fluxpythonspawner.py index 44e06ec7..bf8eb939 100644 --- a/tests/test_fluxpythonspawner.py +++ b/tests/test_fluxpythonspawner.py @@ -6,13 +6,13 @@ import numpy as np from executorlib.task_scheduler.interactive.shared import execute_tasks -from executorlib.task_scheduler.interactive import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler from executorlib.standalone.serialize import cloudpickle_register try: import flux.job - from executorlib.task_scheduler.interactive import FluxPythonSpawner + from executorlib.task_scheduler.interactive.fluxspawner import FluxPythonSpawner skip_flux_test = "FLUX_URI" not in os.environ pmi = os.environ.get("EXECUTORLIB_PMIX", None) diff --git a/tests/test_interactive_dependencies.py b/tests/test_interactive_dependencies.py index 7cbb1148..74d48d2b 100644 --- a/tests/test_interactive_dependencies.py +++ b/tests/test_interactive_dependencies.py @@ -5,7 +5,7 @@ import numpy as np -from executorlib.task_scheduler.interactive import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler from executorlib.standalone.interactive.spawner import MpiExecSpawner diff --git a/tests/test_interactive_slurmspawner.py b/tests/test_interactive_slurmspawner.py index 4f595181..a0af5b67 100644 --- a/tests/test_interactive_slurmspawner.py +++ b/tests/test_interactive_slurmspawner.py @@ -2,7 +2,7 @@ from executorlib.task_scheduler.interactive.slurmspawner import generate_slurm_command try: - from executorlib.task_scheduler.file import _pysqa_execute_command + from executorlib.task_scheduler.file.queue_spawner import _pysqa_execute_command skip_pysqa_test = False except ImportError: diff --git a/tests/test_mpiexecspawner.py b/tests/test_mpiexecspawner.py index 6796a321..79f14b9f 100644 --- a/tests/test_mpiexecspawner.py +++ b/tests/test_mpiexecspawner.py @@ -10,8 +10,8 @@ from executorlib.task_scheduler.base import TaskSchedulerBase from executorlib.standalone.interactive.spawner import MpiExecSpawner from executorlib.task_scheduler.interactive.shared import execute_tasks -from executorlib.task_scheduler.interactive import BlockAllocationTaskScheduler -from executorlib.task_scheduler.interactive import OneProcessTaskScheduler +from executorlib.task_scheduler.interactive.blockallocation import BlockAllocationTaskScheduler +from executorlib.task_scheduler.interactive.onetoone import OneProcessTaskScheduler from executorlib.standalone.interactive.backend import call_funct from executorlib.standalone.serialize import cloudpickle_register From b8f73a8c4eeaf5c28c85a3c0fcc0bff6769dee9c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:25:28 +0200 Subject: [PATCH 08/14] fix hidden imports --- executorlib/executor/flux.py | 2 +- executorlib/executor/slurm.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executorlib/executor/flux.py b/executorlib/executor/flux.py index 36d584b7..59587bab 100644 --- a/executorlib/executor/flux.py +++ b/executorlib/executor/flux.py @@ -343,7 +343,7 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.task_scheduler.file import create_file_executor + from executorlib.task_scheduler.file.task_scheduler import create_file_executor super().__init__( executor=create_file_executor( diff --git a/executorlib/executor/slurm.py b/executorlib/executor/slurm.py index b9705689..503505dc 100644 --- a/executorlib/executor/slurm.py +++ b/executorlib/executor/slurm.py @@ -152,7 +152,7 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.task_scheduler.file import create_file_executor + from executorlib.task_scheduler.file.task_scheduler import create_file_executor super().__init__( executor=create_file_executor( From 5e411b789edc8a2e088b740da663e58dd200f527 Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Sat, 26 Apr 2025 07:25:40 +0000 Subject: [PATCH 09/14] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- executorlib/executor/flux.py | 4 +++- executorlib/executor/slurm.py | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/executorlib/executor/flux.py b/executorlib/executor/flux.py index 59587bab..9147778e 100644 --- a/executorlib/executor/flux.py +++ b/executorlib/executor/flux.py @@ -343,7 +343,9 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.task_scheduler.file.task_scheduler import create_file_executor + from executorlib.task_scheduler.file.task_scheduler import ( + create_file_executor, + ) super().__init__( executor=create_file_executor( diff --git a/executorlib/executor/slurm.py b/executorlib/executor/slurm.py index 503505dc..1a557fc0 100644 --- a/executorlib/executor/slurm.py +++ b/executorlib/executor/slurm.py @@ -152,7 +152,9 @@ def __init__( {k: v for k, v in default_resource_dict.items() if k not in resource_dict} ) if not plot_dependency_graph: - from executorlib.task_scheduler.file.task_scheduler import create_file_executor + from executorlib.task_scheduler.file.task_scheduler import ( + create_file_executor, + ) super().__init__( executor=create_file_executor( From 3ba0dcd207ed32067b4a3dfd8c41833bdd28cbc1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:42:27 +0200 Subject: [PATCH 10/14] Update documentation --- notebooks/4-developer.ipynb | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/notebooks/4-developer.ipynb b/notebooks/4-developer.ipynb index a0abcf05..29653b93 100644 --- a/notebooks/4-developer.ipynb +++ b/notebooks/4-developer.ipynb @@ -71,16 +71,16 @@ "## Modules\n", "While it is not recommended to link to specific internal components of executorlib in external Python packages but rather only the `Executor` classes should be used as central interfaces to executorlib, the internal architecture is briefly outlined below. \n", "* `backend` - the backend module contains the functionality for the Python processes created by executorlib to execute the submitted Python functions.\n", - "* `base` - the base module contains the definition of the executorlib `ExecutorBase` class which is internally used to create the different interfaces. To compare if an given `Executor` class is based on executorlib compare with the `ExecutorBase` class which can be imported as `from executorlib.base.executor import ExecutorBase`.\n", - "* `cache` - the cache module defines the file based communication for the [HPC Cluster Executor](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html).\n", - "* `interactive` - the interactive modules defines the [zero message queue](https://zeromq.org) based communication for the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) and the [HPC Job Executor](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html).\n", - "* `interfaces` - the different `Executor` classes are defined here, namely `SingleNodeExecutor`, `SlurmClusterExecutor`, `SlurmJobExecutor`, `FluxClusterExecutor` and `FluxJobExecutor`.\n", + "* `executor` - the executor module defines the different `Executor` classes, namely `SingleNodeExecutor`, `SlurmClusterExecutor`, `SlurmJobExecutor`, `FluxClusterExecutor` and `FluxJobExecutor`. These are the interfaces the user interacts with.\n", "* `standalone` - the standalone module contains a number of utility functions which only depend on external libraries and do not have any internal dependency to other parts of `executorlib`. This includes the functionality to generate executable commands, the [h5py](https://www.h5py.org) based interface for caching, a number of input checks, routines to plot the dependencies of a number of future objects, functionality to interact with the [queues defined in the Python standard library](https://docs.python.org/3/library/queue.html), the interface for serialization based on [cloudpickle](https://github.com/cloudpipe/cloudpickle) and finally an extension to the [threading](https://docs.python.org/3/library/threading.html) of the Python standard library.\n", + "* `task_scheduler` - the internal task scheduler module defines the task schedulers, namely `BlockAllocationTaskScheduler`, `DependencyTaskScheduler`, `FileTaskScheduler` and `OneProcessTaskScheduler`. They are divided into two sub modules:\n", + " * `file` - the file based task scheduler module defines the file based communication for the [HPC Cluster Executor](https://executorlib.readthedocs.io/en/latest/2-hpc-cluster.html).\n", + " * `interactive` - the interactive task scheduler module defines the [zero message queue](https://zeromq.org) based communication for the [Single Node Executor](https://executorlib.readthedocs.io/en/latest/1-single-node.html) and the [HPC Job Executor](https://executorlib.readthedocs.io/en/latest/3-hpc-job.html).\n", "\n", "Given the level of separation the integration of submodules from the standalone module in external software packages should be the easiest way to benefit from the developments in executorlib beyond just using the `Executor` class. \n", "\n", "## Interface Class Hierarchy\n", - "executorlib provides five different interfaces, namely `SingleNodeExecutor`, `SlurmClusterExecutor`, `SlurmJobExecutor`, `FluxClusterExecutor` and `FluxJobExecutor`, internally these are mapped to four types of task schedulers `Executor._task_scheduler`, namely `BlockAllocationTaskScheduler`, `DependencyTaskScheduler`, `FileTaskScheduler` and `OneProcessTaskScheduler` depending on which options are selected. The dependence is illustrated in the following table:\n", + "executorlib provides five different interfaces, namely `SingleNodeExecutor`, `SlurmClusterExecutor`, `SlurmJobExecutor`, `FluxClusterExecutor` and `FluxJobExecutor`, internally these are mapped to four types of task schedulers, namely `BlockAllocationTaskScheduler`, `DependencyTaskScheduler`, `FileTaskScheduler` and `OneProcessTaskScheduler` depending on which options are selected. Finally, the task schedulers are connected to spawners to start new processes, namely the `MpiExecSpawner`, `SrunSpawner` and `FluxPythonSpawner`. The dependence is illustrated in the following table:\n", "\n", "| | `BlockAllocationTaskScheduler` | `DependencyTaskScheduler` | `FileTaskScheduler` | `OneProcessTaskScheduler` |\n", "|-------------------------------------------------------------------------|--------------------------------|---------------------------|---------------------|---------------------------|\n", From e1a8a8cee5c8d75c45581426b720d20c34083728 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:54:52 +0200 Subject: [PATCH 11/14] revert file to cache --- executorlib/executor/flux.py | 10 +++++----- executorlib/executor/single.py | 6 +++--- executorlib/executor/slurm.py | 10 +++++----- executorlib/task_scheduler/file/task_scheduler.py | 4 ++-- executorlib/task_scheduler/interactive/shared.py | 4 ++-- tests/test_cache_backend_execute.py | 12 ++++++------ tests/test_cache_fileexecutor_mpi.py | 4 ++-- tests/test_cache_fileexecutor_serial.py | 10 +++++----- tests/test_fluxclusterexecutor.py | 8 ++++---- tests/test_singlenodeexecutor_cache.py | 4 ++-- tests/test_standalone_hdf.py | 14 +++++++------- 11 files changed, 43 insertions(+), 43 deletions(-) diff --git a/executorlib/executor/flux.py b/executorlib/executor/flux.py index 9147778e..c7568eb9 100644 --- a/executorlib/executor/flux.py +++ b/executorlib/executor/flux.py @@ -36,7 +36,7 @@ class FluxJobExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -120,7 +120,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -221,7 +221,7 @@ class FluxClusterExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -298,7 +298,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -410,7 +410,7 @@ def create_flux_executor( number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. max_cores (int): defines the number cores which can be used in parallel - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - threads_per_core (int): number of OpenMP threads to be used for each function call diff --git a/executorlib/executor/single.py b/executorlib/executor/single.py index 3435df06..f4af810e 100644 --- a/executorlib/executor/single.py +++ b/executorlib/executor/single.py @@ -29,7 +29,7 @@ class SingleNodeExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -104,7 +104,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -198,7 +198,7 @@ def create_single_node_executor( number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. max_cores (int): defines the number cores which can be used in parallel - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - threads_per_core (int): number of OpenMP threads to be used for each function call diff --git a/executorlib/executor/slurm.py b/executorlib/executor/slurm.py index 1a557fc0..fe604386 100644 --- a/executorlib/executor/slurm.py +++ b/executorlib/executor/slurm.py @@ -30,7 +30,7 @@ class SlurmClusterExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -107,7 +107,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -206,7 +206,7 @@ class SlurmJobExecutor(ExecutorBase): max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -285,7 +285,7 @@ def __init__( max_workers (int): for backwards compatibility with the standard library, max_workers also defines the number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". max_cores (int): defines the number cores which can be used in parallel resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call @@ -383,7 +383,7 @@ def create_slurm_executor( number of cores which can be used in parallel - just like the max_cores parameter. Using max_cores is recommended, as computers have a limited number of compute cores. max_cores (int): defines the number cores which can be used in parallel - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - threads_per_core (int): number of OpenMP threads to be used for each function call diff --git a/executorlib/task_scheduler/file/task_scheduler.py b/executorlib/task_scheduler/file/task_scheduler.py index 3a220019..2a2ca099 100644 --- a/executorlib/task_scheduler/file/task_scheduler.py +++ b/executorlib/task_scheduler/file/task_scheduler.py @@ -27,7 +27,7 @@ class FileTaskScheduler(TaskSchedulerBase): def __init__( self, - cache_directory: str = "file", + cache_directory: str = "cache", resource_dict: Optional[dict] = None, execute_function: Callable = execute_with_pysqa, terminate_function: Optional[Callable] = None, @@ -39,7 +39,7 @@ def __init__( Initialize the FileExecutor. Args: - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". resource_dict (dict): A dictionary of resources required by the task. With the following keys: - cores (int): number of MPI cores to be used for each function call - cwd (str/None): current working directory where the parallel python task is executed diff --git a/executorlib/task_scheduler/interactive/shared.py b/executorlib/task_scheduler/interactive/shared.py index 6e82b1a0..5216b9ce 100644 --- a/executorlib/task_scheduler/interactive/shared.py +++ b/executorlib/task_scheduler/interactive/shared.py @@ -40,7 +40,7 @@ def execute_tasks( this look up for security reasons. So on MacOS it is required to set this option to true init_function (Callable): optional function to preset arguments for functions which are submitted later - cache_directory (str, optional): The directory to store file files. Defaults to "file". + cache_directory (str, optional): The directory to store cache files. Defaults to "cache". queue_join_on_shutdown (bool): Join communication queue when thread is closed. Defaults to True. """ interface = interface_bootup( @@ -138,7 +138,7 @@ def _execute_task_with_cache( task_dict (dict): task submitted to the executor as dictionary. This dictionary has the following keys {"fn": Callable, "args": (), "kwargs": {}, "resource_dict": {}} future_queue (Queue): Queue for receiving new tasks. - cache_directory (str): The directory to store file files. + cache_directory (str): The directory to store cache files. """ from executorlib.standalone.hdf import dump, get_output diff --git a/tests/test_cache_backend_execute.py b/tests/test_cache_backend_execute.py index 5e934404..fd8e9bad 100644 --- a/tests/test_cache_backend_execute.py +++ b/tests/test_cache_backend_execute.py @@ -28,7 +28,7 @@ def get_error(a): ) class TestSharedFunctions(unittest.TestCase): def test_execute_function_mixed(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=my_funct, @@ -56,7 +56,7 @@ def test_execute_function_mixed(self): self.assertEqual(future_file_obj.result(), 3) def test_execute_function_args(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=my_funct, @@ -84,7 +84,7 @@ def test_execute_function_args(self): self.assertEqual(future_file_obj.result(), 3) def test_execute_function_kwargs(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=my_funct, @@ -112,7 +112,7 @@ def test_execute_function_kwargs(self): self.assertEqual(future_file_obj.result(), 3) def test_execute_function_error(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) task_key, data_dict = serialize_funct_h5( fn=get_error, @@ -142,5 +142,5 @@ def test_execute_function_error(self): future_file_obj.result() def tearDown(self): - if os.path.exists("file"): - shutil.rmtree("file") + if os.path.exists("cache"): + shutil.rmtree("cache") diff --git a/tests/test_cache_fileexecutor_mpi.py b/tests/test_cache_fileexecutor_mpi.py index 71efbf12..d4a4f3a7 100644 --- a/tests/test_cache_fileexecutor_mpi.py +++ b/tests/test_cache_fileexecutor_mpi.py @@ -40,5 +40,5 @@ def test_executor(self): self.assertTrue(fs1.done()) def tearDown(self): - if os.path.exists("file"): - shutil.rmtree("file") + if os.path.exists("cache"): + shutil.rmtree("cache") diff --git a/tests/test_cache_fileexecutor_serial.py b/tests/test_cache_fileexecutor_serial.py index 8479dccf..c28c858b 100644 --- a/tests/test_cache_fileexecutor_serial.py +++ b/tests/test_cache_fileexecutor_serial.py @@ -93,7 +93,7 @@ def test_executor_function(self): "resource_dict": {}, } ) - cache_dir = os.path.abspath("file") + cache_dir = os.path.abspath("cache") os.makedirs(cache_dir, exist_ok=True) process = Thread( target=execute_tasks_h5, @@ -134,7 +134,7 @@ def test_executor_function_dependence_kwargs(self): "resource_dict": {}, } ) - cache_dir = os.path.abspath("file") + cache_dir = os.path.abspath("cache") os.makedirs(cache_dir, exist_ok=True) process = Thread( target=execute_tasks_h5, @@ -175,7 +175,7 @@ def test_executor_function_dependence_args(self): "resource_dict": {}, } ) - cache_dir = os.path.abspath("file") + cache_dir = os.path.abspath("cache") os.makedirs(cache_dir, exist_ok=True) process = Thread( target=execute_tasks_h5, @@ -203,5 +203,5 @@ def test_execute_in_subprocess_errors(self): execute_in_subprocess(file_name=__file__, command=[], backend="flux") def tearDown(self): - if os.path.exists("file"): - shutil.rmtree("file") + if os.path.exists("cache"): + shutil.rmtree("cache") diff --git a/tests/test_fluxclusterexecutor.py b/tests/test_fluxclusterexecutor.py index 28262726..73deb661 100644 --- a/tests/test_fluxclusterexecutor.py +++ b/tests/test_fluxclusterexecutor.py @@ -33,9 +33,9 @@ def mpi_funct(i): class TestCacheExecutorPysqa(unittest.TestCase): def test_executor(self): with FluxClusterExecutor( - resource_dict={"cores": 2, "cwd": "file"}, + resource_dict={"cores": 2, "cwd": "cache"}, block_allocation=False, - cache_directory="file", + cache_directory="cache", ) as exe: cloudpickle_register(ind=1) fs1 = exe.submit(mpi_funct, 1) @@ -44,5 +44,5 @@ def test_executor(self): self.assertTrue(fs1.done()) def tearDown(self): - if os.path.exists("file"): - shutil.rmtree("file") + if os.path.exists("cache"): + shutil.rmtree("cache") diff --git a/tests/test_singlenodeexecutor_cache.py b/tests/test_singlenodeexecutor_cache.py index 6a81caaf..c3a327a4 100644 --- a/tests/test_singlenodeexecutor_cache.py +++ b/tests/test_singlenodeexecutor_cache.py @@ -42,7 +42,7 @@ def test_cache_error(self): print(f.result()) def tearDown(self): - if os.path.exists("file"): - shutil.rmtree("file") + if os.path.exists("cache"): + shutil.rmtree("cache") if os.path.exists("cache_error"): shutil.rmtree("cache_error") diff --git a/tests/test_standalone_hdf.py b/tests/test_standalone_hdf.py index a130d112..95d1aa6e 100644 --- a/tests/test_standalone_hdf.py +++ b/tests/test_standalone_hdf.py @@ -26,7 +26,7 @@ def my_funct(a, b): ) class TestSharedFunctions(unittest.TestCase): def test_hdf_mixed(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_mixed.h5") a = 1 @@ -46,7 +46,7 @@ def test_hdf_mixed(self): self.assertIsNone(output) def test_hdf_args(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_args.h5") a = 1 @@ -63,7 +63,7 @@ def test_hdf_args(self): self.assertIsNone(output) def test_hdf_kwargs(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_kwargs.h5") a = 1 @@ -89,7 +89,7 @@ def test_hdf_kwargs(self): self.assertIsNone(output) def test_hdf_queue_id(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_queue.h5") queue_id = 123 @@ -105,7 +105,7 @@ def test_hdf_queue_id(self): self.assertIsNone(output) def test_hdf_error(self): - cache_directory = os.path.abspath("file") + cache_directory = os.path.abspath("cache") os.makedirs(cache_directory, exist_ok=True) file_name = os.path.join(cache_directory, "test_error.h5") error = ValueError() @@ -120,5 +120,5 @@ def test_hdf_error(self): self.assertTrue(isinstance(output, error.__class__)) def tearDown(self): - if os.path.exists("file"): - shutil.rmtree("file") + if os.path.exists("cache"): + shutil.rmtree("cache") From 124f9253e66c208c3cb76cad9848b06cedf5c578 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 09:58:03 +0200 Subject: [PATCH 12/14] rename cache file --- executorlib/standalone/hdf.py | 2 +- executorlib/task_scheduler/file/shared.py | 8 +++---- .../task_scheduler/interactive/shared.py | 4 ++-- tests/test_cache_backend_execute.py | 24 +++++++++---------- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/executorlib/standalone/hdf.py b/executorlib/standalone/hdf.py index b782b5c0..8fb26f72 100644 --- a/executorlib/standalone/hdf.py +++ b/executorlib/standalone/hdf.py @@ -108,7 +108,7 @@ def get_queue_id(file_name: Optional[str]) -> Optional[int]: def get_cache_data(cache_directory: str) -> list[dict]: file_lst = [] for task_key in os.listdir(cache_directory): - file_name = os.path.join(cache_directory, task_key, "file.h5out") + file_name = os.path.join(cache_directory, task_key, "cache.h5out") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) if os.path.exists(file_name): with h5py.File(file_name, "r") as hdf: diff --git a/executorlib/task_scheduler/file/shared.py b/executorlib/task_scheduler/file/shared.py index deb59374..36c3f693 100644 --- a/executorlib/task_scheduler/file/shared.py +++ b/executorlib/task_scheduler/file/shared.py @@ -110,11 +110,11 @@ def execute_tasks_h5( if task_key not in memory_dict: if not ( task_key in os.listdir(cache_directory) - and "file.h5out" + and "cache.h5out" in os.listdir(os.path.join(cache_directory, task_key)) ): os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) - file_name = os.path.join(cache_directory, task_key, "file.h5in") + file_name = os.path.join(cache_directory, task_key, "cache.h5in") dump(file_name=file_name, data_dict=data_dict) if not disable_dependencies: task_dependent_lst = [ @@ -141,7 +141,7 @@ def execute_tasks_h5( cache_directory=os.path.join(cache_directory, task_key), ) file_name_dict[task_key] = os.path.join( - cache_directory, task_key, "file.h5out" + cache_directory, task_key, "cache.h5out" ) memory_dict[task_key] = task_dict["future"] future_queue.task_done() @@ -197,7 +197,7 @@ def _check_task_output( Future: The updated future object. """ - file_name = os.path.join(cache_directory, task_key, "file.h5out") + file_name = os.path.join(cache_directory, task_key, "cache.h5out") if not os.path.exists(file_name): return future_obj exec_flag, no_error_flag, result = get_output(file_name=file_name) diff --git a/executorlib/task_scheduler/interactive/shared.py b/executorlib/task_scheduler/interactive/shared.py index 5216b9ce..40ef84d9 100644 --- a/executorlib/task_scheduler/interactive/shared.py +++ b/executorlib/task_scheduler/interactive/shared.py @@ -149,10 +149,10 @@ def _execute_task_with_cache( resource_dict=task_dict.get("resource_dict", {}), ) os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) - file_name = os.path.join(cache_directory, task_key, "file.h5out") + file_name = os.path.join(cache_directory, task_key, "cache.h5out") if not ( task_key in os.listdir(cache_directory) - and "file.h5out" in os.listdir(os.path.join(cache_directory, task_key)) + and "cache.h5out" in os.listdir(os.path.join(cache_directory, task_key)) ): f = task_dict.pop("future") if f.set_running_or_notify_cancel(): diff --git a/tests/test_cache_backend_execute.py b/tests/test_cache_backend_execute.py index fd8e9bad..2e38f2cf 100644 --- a/tests/test_cache_backend_execute.py +++ b/tests/test_cache_backend_execute.py @@ -35,7 +35,7 @@ def test_execute_function_mixed(self): fn_args=[1], fn_kwargs={"b": 2}, ) - file_name = os.path.join(cache_directory, task_key, "file.h5in") + file_name = os.path.join(cache_directory, task_key, "cache.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -46,11 +46,11 @@ def test_execute_function_mixed(self): self.assertTrue(future_obj.done()) self.assertEqual(future_obj.result(), 3) self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "file.h5out") + file_name=os.path.join(cache_directory, task_key, "cache.h5out") ) self.assertTrue(future_file_obj.done()) self.assertEqual(future_file_obj.result(), 3) @@ -63,7 +63,7 @@ def test_execute_function_args(self): fn_args=[1, 2], fn_kwargs={}, ) - file_name = os.path.join(cache_directory, task_key, "file.h5in") + file_name = os.path.join(cache_directory, task_key, "cache.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -74,11 +74,11 @@ def test_execute_function_args(self): self.assertTrue(future_obj.done()) self.assertEqual(future_obj.result(), 3) self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "file.h5out") + file_name=os.path.join(cache_directory, task_key, "cache.h5out") ) self.assertTrue(future_file_obj.done()) self.assertEqual(future_file_obj.result(), 3) @@ -91,7 +91,7 @@ def test_execute_function_kwargs(self): fn_args=[], fn_kwargs={"a": 1, "b": 2}, ) - file_name = os.path.join(cache_directory, task_key, "file.h5in") + file_name = os.path.join(cache_directory, task_key, "cache.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -102,11 +102,11 @@ def test_execute_function_kwargs(self): self.assertTrue(future_obj.done()) self.assertEqual(future_obj.result(), 3) self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "file.h5out") + file_name=os.path.join(cache_directory, task_key, "cache.h5out") ) self.assertTrue(future_file_obj.done()) self.assertEqual(future_file_obj.result(), 3) @@ -119,7 +119,7 @@ def test_execute_function_error(self): fn_args=[], fn_kwargs={"a": 1}, ) - file_name = os.path.join(cache_directory, task_key, "file.h5in") + file_name = os.path.join(cache_directory, task_key, "cache.h5in") os.makedirs(os.path.join(cache_directory, task_key), exist_ok=True) dump(file_name=file_name, data_dict=data_dict) backend_execute_task_in_file(file_name=file_name) @@ -131,11 +131,11 @@ def test_execute_function_error(self): with self.assertRaises(ValueError): future_obj.result() self.assertTrue( - get_runtime(file_name=os.path.join(cache_directory, task_key, "file.h5out")) + get_runtime(file_name=os.path.join(cache_directory, task_key, "cache.h5out")) > 0.0 ) future_file_obj = FutureItem( - file_name=os.path.join(cache_directory, task_key, "file.h5out") + file_name=os.path.join(cache_directory, task_key, "cache.h5out") ) self.assertTrue(future_file_obj.done()) with self.assertRaises(ValueError): From b2962e2befcc5196dfbf6ebdbbff9bef2d297ae3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 10:00:14 +0200 Subject: [PATCH 13/14] another fix --- executorlib/task_scheduler/interactive/shared.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executorlib/task_scheduler/interactive/shared.py b/executorlib/task_scheduler/interactive/shared.py index 40ef84d9..cc582347 100644 --- a/executorlib/task_scheduler/interactive/shared.py +++ b/executorlib/task_scheduler/interactive/shared.py @@ -131,7 +131,7 @@ def _execute_task_with_cache( cache_directory: str, ): """ - Execute the task in the task_dict by communicating it via the interface using the file in the file directory. + Execute the task in the task_dict by communicating it via the interface using the cache in the cache directory. Args: interface (SocketInterface): socket interface for zmq communication From 35c05327a8614cf744653e60f08dc53c67fa2838 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Jan=C3=9Fen?= Date: Sat, 26 Apr 2025 10:03:12 +0200 Subject: [PATCH 14/14] last fixes --- tests/test_mpiexecspawner.py | 6 +++--- tests/test_singlenodeexecutor_cache.py | 2 +- tests/test_singlenodeexecutor_mpi.py | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/test_mpiexecspawner.py b/tests/test_mpiexecspawner.py index 79f14b9f..a1d08cdc 100644 --- a/tests/test_mpiexecspawner.py +++ b/tests/test_mpiexecspawner.py @@ -503,7 +503,7 @@ def test_execute_task_parallel(self): class TestFuturePoolCache(unittest.TestCase): def tearDown(self): - shutil.rmtree("./file") + shutil.rmtree("./cache") @unittest.skipIf( skip_h5py_test, "h5py is not installed, so the h5py tests are skipped." @@ -519,7 +519,7 @@ def test_execute_task_cache(self): cores=1, openmpi_oversubscribe=False, spawner=MpiExecSpawner, - cache_directory="./file", + cache_directory="./cache", ) self.assertEqual(f.result(), 1) q.join() @@ -538,6 +538,6 @@ def test_execute_task_cache_failed_no_argument(self): cores=1, openmpi_oversubscribe=False, spawner=MpiExecSpawner, - cache_directory="./file", + cache_directory="./cache", ) q.join() diff --git a/tests/test_singlenodeexecutor_cache.py b/tests/test_singlenodeexecutor_cache.py index c3a327a4..bded9cab 100644 --- a/tests/test_singlenodeexecutor_cache.py +++ b/tests/test_singlenodeexecutor_cache.py @@ -22,7 +22,7 @@ def get_error(a): ) class TestCacheFunctions(unittest.TestCase): def test_cache_data(self): - cache_directory = "./file" + cache_directory = "./cache" with SingleNodeExecutor(cache_directory=cache_directory) as exe: future_lst = [exe.submit(sum, [i, i]) for i in range(1, 4)] result_lst = [f.result() for f in future_lst] diff --git a/tests/test_singlenodeexecutor_mpi.py b/tests/test_singlenodeexecutor_mpi.py index 7b238959..ca7abe39 100644 --- a/tests/test_singlenodeexecutor_mpi.py +++ b/tests/test_singlenodeexecutor_mpi.py @@ -83,7 +83,7 @@ def test_errors(self): class TestExecutorBackendCache(unittest.TestCase): def tearDown(self): - shutil.rmtree("./file") + shutil.rmtree("./cache") @unittest.skipIf( skip_mpi4py_test, "mpi4py is not installed, so the mpi4py tests are skipped." @@ -93,7 +93,7 @@ def test_meta_executor_parallel_cache(self): max_workers=2, resource_dict={"cores": 2}, block_allocation=True, - cache_directory="./file", + cache_directory="./cache", ) as exe: cloudpickle_register(ind=1) time_1 = time.time()