From 3e89c0e3c4c2a9dfe53ceb4cb771f955011f694a Mon Sep 17 00:00:00 2001 From: hategan Date: Fri, 2 Feb 2024 11:00:01 -0800 Subject: [PATCH 01/39] Initial staging commit --- src/psij/exceptions.py | 18 + .../batch/batch_scheduler_executor.py | 118 +++- .../executors/batch/common/batch_lib.mustache | 80 +++ src/psij/executors/batch/script_generator.py | 3 +- src/psij/executors/local.py | 546 +++++++++++------- src/psij/job_spec.py | 27 +- src/psij/job_state.py | 48 +- src/psij/staging.py | 94 +++ tests/plugins1/_batch_test/test/test.mustache | 22 +- tests/test_staging.py | 25 + 10 files changed, 745 insertions(+), 236 deletions(-) create mode 100644 src/psij/executors/batch/common/batch_lib.mustache create mode 100644 src/psij/staging.py create mode 100644 tests/test_staging.py diff --git a/src/psij/exceptions.py b/src/psij/exceptions.py index 8e543d9d..171ed252 100644 --- a/src/psij/exceptions.py +++ b/src/psij/exceptions.py @@ -62,3 +62,21 @@ def __init__(self, message: str, exception: Optional[Exception] = None, conditions such an error would persist across subsequent re-tries until correct credentials are used. """ + +class CompositeException(Exception): + def __init__(self, ex: Exception) -> None: + self.exceptions = [ex] + + def add_exception(self, ex: Exception) -> None: + self.exceptions.append(ex) + + +class LauncherException(Exception): + def __init__(self, message: str) -> None: + super().__init__('Launcher failure: %s' % message) + + +class JobException(Exception): + def __init__(self, exit_code: int) -> None: + super().__init__('Job exited with exit code %s' % exit_code) + self.exit_code = exit_code diff --git a/src/psij/executors/batch/batch_scheduler_executor.py b/src/psij/executors/batch/batch_scheduler_executor.py index 2c6b3dbd..b3bf5ca2 100644 --- a/src/psij/executors/batch/batch_scheduler_executor.py +++ b/src/psij/executors/batch/batch_scheduler_executor.py @@ -1,5 +1,12 @@ +import atexit +import fcntl +import io import logging import os +import tempfile + +import psutil +import socket import subprocess import time import traceback @@ -14,6 +21,7 @@ from psij import JobExecutor, JobExecutorConfig, Launcher, Job, SubmitException, \ JobStatus, JobState from psij.executors.batch.template_function_library import ALL as FUNCTION_LIBRARY +from psij.utils import SingletonThread UNKNOWN_ERROR = 'PSIJ: Unknown error' @@ -265,6 +273,8 @@ def cancel(self, job: Job) -> None: except SubmitException: # re-raise raise + finally: + self._status_update_thread.unregister_job(job) def attach(self, job: Job, native_id: str) -> None: """Attaches a job to a native job. @@ -531,9 +541,11 @@ def _set_job_status(self, job: Job, status: JobStatus) -> None: # is_greater_than returns T/F if the states are comparable and None if not, so # we have to check explicitly for the boolean value rather than truthiness return - if status.state.final and job.native_id: - self._clean_submit_script(job) - self._read_aux_files(job, status) + if status.state.final: + self._status_update_thread.unregister_job(job) + if job.native_id: + self._clean_submit_script(job) + self._read_aux_files(job, status) super()._set_job_status(job, status) def _clean_submit_script(self, job: Job) -> None: @@ -638,13 +650,19 @@ def __init__(self, name: str, config: BatchSchedulerExecutorConfig, # counts consecutive errors while invoking qstat or equivalent self._poll_error_count = 0 self._jobs_lock = RLock() + self._status_updater = _StatusUpdater(config, executor) def run(self) -> None: logger.debug('Executor %s: queue poll thread started', self.executor) time.sleep(self.config.initial_queue_polling_delay) while True: self._poll() - time.sleep(self.config.queue_polling_interval) + start = time.time() + now = start + while now - start < self.config.queue_polling_interval: + self._status_updater.step() + time.sleep(1) + now = time.time() def _poll(self) -> None: with self._jobs_lock: @@ -686,6 +704,8 @@ def _poll(self) -> None: if status.state.final: with self._jobs_lock: del self._jobs[native_id] + for job in job_list: + self._status_updater.unregister_job(job) except Exception as ex: msg = traceback.format_exc() self._handle_poll_error(True, ex, 'Error updating job statuses {}'.format(msg)) @@ -713,9 +733,11 @@ def _handle_poll_error(self, immediate: bool, ex: Exception, msg: str) -> None: self._jobs.clear() for job_list in jobs_copy.values(): for job in job_list: + self._status_updater.unregister_job(job) self.executor._set_job_status(job, JobStatus(JobState.FAILED, message=msg)) def register_job(self, job: Job) -> None: + self._status_updater.register_job(job) assert job.native_id logger.info('Job %s: registering', job.id) with self._jobs_lock: @@ -724,3 +746,91 @@ def register_job(self, job: Job) -> None: self._jobs[native_id] = [job] else: self._jobs[job.native_id].append(job) + +class _StatusUpdater: + # we are expecting short messages in the form + RECV_BUFSZ = 2048 + + def __init__(self, config: BatchSchedulerExecutorConfig, + executor: BatchSchedulerExecutor) -> None: + self.config = config + self.executor = executor + self.socket = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) + self.socket.setblocking(False) + self.socket.bind(('', 0)) + self.port = self.socket.getsockname()[1] + self.ips = self._get_ips() + print('IPS: %s' % self.ips) + print('Port: %s' % self.port) + self._create_update_file() + print('Update file: %s' % self.update_file.name) + self.partial_file_data = '' + self.partial_net_data = '' + self._jobs = {} + self._jobs_lock = RLock() + + def _get_ips(self) -> List[str]: + addrs = psutil.net_if_addrs() + r = [] + for name, l in addrs.items(): + if name == 'lo': + continue + for a in l: + if a.family == socket.AddressFamily.AF_INET: + r.append(a.address) + return r + + def _create_update_file(self) -> None: + f = tempfile.NamedTemporaryFile(dir=self.config.work_directory, prefix='supd_', + delete=False) + name = f.name + atexit.register(os.remove, name) + f.close() + self.update_file = open(name, 'r+b') + self.update_file.seek(0, io.SEEK_END) + self.update_file_pos = self.update_file.tell() + + def register_job(self, job: Job) -> None: + with self._jobs_lock: + self._jobs[job.id] = job + + def unregister_job(self, job: Job) -> None: + with self._jobs_lock: + del self._jobs[job.id] + + def step(self) -> None: + self.update_file.seek(0, io.SEEK_END) + pos = self.update_file.tell() + if pos > self.update_file_pos: + self.update_file.seek(self.update_file_pos, io.SEEK_SET) + n = pos - self.update_file_pos + self._process_update_data(self.update_file.read(n)) + self.update_file_pos = pos + else: + try: + data = self.socket.recv(_StatusUpdater.RECV_BUFSZ) + self._process_update_data(data) + except socket.error as e: + pass + + def _process_update_data(self, data: bytes) -> None: + sdata = data.decode('utf-8') + lines = sdata.splitlines() + for line in lines: + print('Status update line: %s' % line) + els = line.split() + if len(els) != 2: + logger.warning('Invalid status update message received: %s' % line) + continue + job_id = els[0] + state = JobState.from_name(els[1]) + job = None + with self._jobs_lock: + try: + job = self._jobs[job_id] + except KeyError: + logger.warning('Received status updated for inexistent job with id %s' % job_id) + if job: + self.executor._set_job_status(job, JobStatus(state)) + + diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache new file mode 100644 index 00000000..ef282cf9 --- /dev/null +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -0,0 +1,80 @@ +_UPDATE_MODE="none" + +update_status() { + STATUS="$1" + if [ "$_UPDATE_MODE" == "none" ]; then + if which nc >/dev/null; then + $_UPDATE_MODE="nc" + else + $_UPDATE_MODE="file" + fi + fi + + if [ "$_UPDATE_MODE" == "nc" ]; then + for ADDR in "{{psij.us_addrs}}"; do + echo "{{psij.job_id}} $STATUS" | nc -q0 -w0 -4 -u $ADDR {{psij.us_port}} + done + else + echo "{{psij.job_id}} $STATUS" >> {{psij.us_file}} + fi +} + +do_stagein() { + do_stage "$@" 0 +} + +do_stage() { + SOURCE="$1" + TARGET="$2" + MODE="$3" + MISSING_OK="$4" + + if [ ! -a "$SOURCE" ] && [ "$MISSING_OK" == "0" ]; then + echo "Missing source file: $SOURCE" + exit 1 + fi + + if [ "$MODE" == "1" ]; then + # copy + cp -r -T "$SOURCE" "$TARGET" + elif [ "$MODE" == "2" ]; tben + # link + ln -s "$SOURCE" "$TARGET" + elif [ "$MODE" == "3" ]; then + # move + mv -T -f "$SOURCE" "$TARGET" + fi +} + +_FLAG_IF_PRESENT=1 +_FLAG_ON_SUCCESS=2 +_FLAG_ON_ERROR=4 +_FLAG_ON_CANCEL=8 + +do_stageout() { + SOURCE="$1" + TARGET="$2" + MODE="$3" + FLAGS="$4" + FAILED="$5" + + if [ "$FAILED" == "0" ] && $((FLAGS & _FLAG_ON_SUCCESS)) ; then + do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) + elif [ "$FAILED" != "0" ] && $((FLAGS & _FLAG_ON_ERROR)) ; then + do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) + fi +} + +do_cleanup() { + TARGET="$1" + + case "$TARGET" in + "{{job.spec.directory}}"*) + echo "rm -rf $TARGET" >>~/cleanup.log + ;; + *) + echo "Cannot clean $TARGET outside of job directory {{job.spec.directory}}" + exit 1 + ;; + esac +} \ No newline at end of file diff --git a/src/psij/executors/batch/script_generator.py b/src/psij/executors/batch/script_generator.py index c33003ea..62049dcb 100644 --- a/src/psij/executors/batch/script_generator.py +++ b/src/psij/executors/batch/script_generator.py @@ -72,7 +72,8 @@ def __init__(self, config: JobExecutorConfig, template_path: pathlib.Path, super().__init__(config) with template_path.open('r') as template_file: self.template = pystache.parse(template_file.read()) - self.renderer = pystache.Renderer(escape=escape) + common_dir = pathlib.Path(__file__).parent / 'common' + self.renderer = pystache.Renderer(escape=escape, search_dirs=[str(common_dir)]) def generate_submit_script(self, job: Job, context: Dict[str, object], out: IO[str]) -> None: """See :func:`~SubmitScriptGenerator.generate_submit_script`. diff --git a/src/psij/executors/local.py b/src/psij/executors/local.py index 8393b384..bb4fc0bb 100644 --- a/src/psij/executors/local.py +++ b/src/psij/executors/local.py @@ -1,22 +1,26 @@ """This module contains the local :class:`~psij.JobExecutor`.""" import logging import os +import pathlib +import platform import shlex +import shutil import signal import subprocess import threading import time -from abc import ABC, abstractmethod +from abc import abstractmethod from tempfile import mkstemp -from types import FrameType -from typing import Optional, Dict, List, Tuple, Type, cast +from typing import Optional, Dict, List, Tuple, TypeVar, Set, Callable import psutil +from psutil import NoSuchProcess -from psij import InvalidJobException, SubmitException, Launcher, ResourceSpecV1 +from psij import InvalidJobException, SubmitException, ResourceSpecV1 from psij import Job, JobSpec, JobExecutorConfig, JobState, JobStatus from psij import JobExecutor -from psij.utils import SingletonThread +from psij.exceptions import CompositeException, LauncherException, JobException +from psij.staging import StageIn, StagingMode, StageOut, StageOutFlags logger = logging.getLogger(__name__) @@ -30,91 +34,317 @@ def _format_shell_cmd(args: List[str]) -> str: return cmd -def _handle_sigchld(signum: int, frame: Optional[FrameType]) -> None: - _ProcessReaper.get_instance()._handle_sigchld() +class _JobThread(threading.Thread): + def __init__(self, job: Job, executor: JobExecutor) -> None: + super().__init__(name = 'LocalJobThread-' + job.id) + self.executor = executor + self.cancel_flag = False + @abstractmethod + def cancel(self): + pass -if threading.current_thread() != threading.main_thread(): - logger.warning('The psij module is being imported from a non-main thread. This prevents the' - 'use of signals in the local executor, which will slow things down a bit.') -else: - signal.signal(signal.SIGCHLD, _handle_sigchld) + def _get_state_from_ec(self, ec: int) -> Tuple[JobState, Optional[Exception]]: + if ec is None or ec == 0: + return JobState.COMPLETED, None + elif ec < 0: + if self.cancel_flag: + return JobState.CANCELED, None + else: + return JobState.FAILED, JobException(ec) + else: + # ec > 0 + # It is not quite clear what happens in Windows. Windows allows the user + # to specify an exit code when killing a process, exit code which will become + # the exit code of the terminated process. However, psutil does not specify what + # is being done for that on Windows. The psutil sources suggest that signal.SIGTERM + # is used, so we check for that. + if platform.system() == 'Windows' and ec == signal.SIGTERM and self.cancel_flag: + return JobState.CANCELED, None + else: + return JobState.FAILED, JobException(ec) -_REAPER_SLEEP_TIME = 0.1 +# The addition of file staging makes fully asynchronous job management difficult, since we don't +# really have much in the way of something reasonably supporting true async file copying. So since +# we have to use threads anyway, and since the local executor is not really meant to scale, we use +# them for attached processes also. +class _AttachedJobThread(_JobThread): + def __init__(self, job: Job, pid: int, executor: JobExecutor) -> None: + super().__init__(job, executor) + self.job = job + self.pid = pid + self._attach() + def _attach(self): + with self.job._status_cv: + try: + self.process = psutil.Process(self.pid) + except NoSuchProcess: + # will check in run() and set status + self.process = None + except Exception as ex: + raise SubmitException('Cannot attach to pid %s' % self.pid, exception=ex) -class _ProcessEntry(ABC): - def __init__(self, job: Job, executor: 'LocalJobExecutor', launcher: Optional[Launcher]): - self.job = job - self.executor = executor - self.exit_code: Optional[int] = None - self.done_time: Optional[float] = None - self.out: Optional[str] = None - self.kill_flag = False - self.process: Optional[subprocess.Popen[bytes]] = None - self.launcher = launcher + def run(self) -> None: + # We assume that the native_id above is a PID that was obtained at some point using + # list(). If so, the process is either still running or has completed. Either way, we must + # bring it up to ACTIVE state + self.executor._set_job_status(self.job, JobStatus(JobState.QUEUED, time=time.time())) + self.executor._set_job_status(self.job, JobStatus(JobState.ACTIVE, time=time.time())) + try: + self._wait_for_job() + except Exception: + pass - @abstractmethod - def kill(self) -> None: - assert self.process is not None - root = psutil.Process(self.process.pid) - for proc in root.children(recursive=True): - proc.kill() - self.process.kill() + def _wait_for_job(self): + message = None + if self.process is None: + state = JobState.COMPLETED + else: + ec = self.process.wait() + state = self._get_state_from_ec(ec) - @abstractmethod - def poll(self) -> Tuple[Optional[int], Optional[str]]: - pass + if state == JobState.FAILED: + message = 'Job failed with exit code %s' % ec - def __repr__(self) -> str: - pid = '-' - if self.process: - pid = str(self.process.pid) - return '{}[jobid: {}, pid: {}]'.format(self.__class__.__name__, self.job.id, pid) + self.executor._set_job_status(self.job, JobStatus(state, message=message, time=time.time())) + def cancel(self): + with self.job._status_cv: + self.cancel_flag = True + if self.process: + self.process.kill() -class _ChildProcessEntry(_ProcessEntry): - def __init__(self, job: Job, executor: 'LocalJobExecutor', - launcher: Optional[Launcher]) -> None: - super().__init__(job, executor, launcher) - self.nodefile: Optional[str] = None - def kill(self) -> None: - super().kill() +class _JobCanceled(Exception): + pass - def poll(self) -> Tuple[Optional[int], Optional[str]]: - assert self.process is not None - exit_code = self.process.poll() - if exit_code is not None: - if self.nodefile: - os.unlink(self.nodefile) - if self.process.stdout: - return exit_code, self.process.stdout.read().decode('utf-8') - else: - return exit_code, None - else: - return None, None + +T = TypeVar('T') -class _AttachedProcessEntry(_ProcessEntry): - def __init__(self, job: Job, process: psutil.Process, executor: 'LocalJobExecutor'): - super().__init__(job, executor, None) - self.process = process +class _ChildJobThread(_JobThread): - def kill(self) -> None: - super().kill() + FLAG_MAP = {JobState.COMPLETED: StageOutFlags.ON_SUCCESS, + JobState.FAILED: StageOutFlags.ON_ERROR, + JobState.CANCELED: StageOutFlags.ON_CANCEL} + + def __init__(self, job: Job, spec: JobSpec, executor: JobExecutor) -> None: + super().__init__(job, executor) + self.job = job + self.spec = spec + if spec.directory is None: + self.jobdir = pathlib.Path('/tmp') + else: + self.jobdir = spec.directory + self.state = None + # set for any error; the overall job is automatically considered failed if set + self.exception = None + self.exit_code = None + self.process = None + + def run(self): + # The following workflow is based on the idea that no error should go unreported. The + # flow is as follows: + # - if there is an error in staging, fail immediately (i.e., do not perform cleanup or + # any other steps). + # - if there is an internal error (i.e., not an executable failure), treat as above and + # fail immediately + # - if a job is canceled during stage in, clean up. If there is an error in cleanup, + # the job will fail instead. + # - if a job is canceled while running, stage out and clean up. If there is an error in + # stage out and/or cleanup, the job will instead fail. + # - if the job fails and there is a subsequent error in staging or cleanup, a compound + # error is created + # - cancellation is ignored during and after stageout - def poll(self) -> Tuple[Optional[int], Optional[str]]: try: - assert self.process - ec: Optional[int] = self.process.wait(timeout=0) - if ec is None: - return 0, None + try: + self.stage_in() + self.run_job() + self.stage_out() + except _JobCanceled: + # only stage_in and run_job (but before the job is actually started) + # are allowed to raise _JobCanceled + self.state = JobState.CANCELED + self.cleanup() + except Exception as ex: + self.fail_job(ex) + + self.update_job_status() + + def update_job_status(self): + if self.exception: + self.executor._set_job_status(self.job, + JobStatus(JobState.FAILED, time=time.time(), + message=str(self.exception), + metadata={'exception': self.exception}, + exit_code=self.exit_code)) + else: + # failed without an exception set is not allowed + assert self.state != JobState.FAILED + self.executor._set_job_status(self.job, JobStatus(self.state, time=time.time())) + + def fail_job(self, ex: Exception) -> None: + if self.state == JobState.FAILED: + if self.exception is None: + self.exception = ex + else: + if not isinstance(self.exception, CompositeException): + self.exception = CompositeException(self.exception) + self.exception.add_exception(ex) + else: + self.state = JobState.FAILED + self.exception = ex + + def stage_in(self) -> None: + self.executor._set_job_status(self.job, JobStatus(JobState.STAGE_IN, time=time.time())) + self._map(self._stage_in_one, self.spec.stage_in) + + def stage_out(self): + self.executor._set_job_status(self.job, JobStatus(JobState.STAGE_OUT, time=time.time())) + self._map(self._stage_out_one, self.spec.stage_out) + + def cleanup(self): + self.executor._set_job_status(self.job, JobStatus(JobState.CLEANUP, time=time.time())) + self._map(self._cleanup_one, self.spec.cleanup) + + @staticmethod + def _map(fn: Callable[[T], None], s: Optional[Set[T]], ) -> None: + if s is None: + return + for o in s: + fn(o) + + def _stage_in_one(self, stage_in: StageIn) -> None: + if self.cancel_flag: + raise _JobCanceled() + src = stage_in.source + scheme = src.scheme + if scheme == '': + scheme = 'file' + if scheme == 'file': + self._local_copy(pathlib.Path(src.path), self._job_rel(stage_in.target), + stage_in.mode, False) + else: + self.fail_job(ValueError('Unsupported scheme "%s" for %s' % (scheme, src))) + + def _stage_out_one(self, stage_out: StageOut) -> None: + dst = stage_out.target + scheme = dst.scheme + if scheme == '': + scheme = 'file' + if scheme == 'file': + flags = stage_out.flags + state = _ChildJobThread.FLAG_MAP[self.state] + if state in flags: + self._local_copy(self._job_rel(stage_out.source), pathlib.Path(dst.path), + stage_out.mode, StageOutFlags.IF_PRESENT in stage_out.flags) + else: + self.fail_job(ValueError('Unsupported scheme "%s" for %s' % (scheme, dst))) + + def _cleanup_one(self, cleanup: pathlib.Path) -> None: + # do some sanity checks + cleanup = self._job_rel(cleanup) + if cleanup.samefile(pathlib.Path('/')): + raise ValueError('Refusing to clean root directory.') + if cleanup.samefile(pathlib.Path.home()): + raise ValueError('Refusing to clean user home directory.') + if cleanup.is_dir(): + shutil.rmtree(str(cleanup)) + else: + cleanup.unlink(missing_ok=True) + + def _job_rel(self, path: pathlib.Path) -> pathlib.Path: + path = path.expanduser() + if not path.is_absolute(): + path = self.jobdir / path + return path.absolute() + + def _local_copy(self, source: pathlib.Path, target: pathlib.Path, mode: StagingMode, + if_present=False): + if if_present and not os.path.exists(source): + return + if mode == StagingMode.COPY: + if source.is_dir(): + shutil.copytree(source, target) else: - return ec, None - except psutil.TimeoutExpired: - return None, None + shutil.copy(source, target) + elif mode == StagingMode.MOVE: + shutil.move(source, target) + elif mode == StagingMode.LINK: + os.symlink(source, target) + + def run_job(self): + launcher = self.executor._get_launcher(self._get_launcher_name(self.spec)) + args = launcher.get_launch_command(self.job) + + if logger.isEnabledFor(logging.DEBUG): + logger.debug('Running %s', _format_shell_cmd(args)) + nodefile = self._generate_nodefile(self.job) + try: + env = _get_env(self.spec, nodefile) + with self.job._status_cv: + if self.cancel_flag: + raise _JobCanceled() + self.process = subprocess.Popen(args, stdout=subprocess.PIPE, + stderr=subprocess.STDOUT, close_fds=True, + cwd=self.spec.directory, env=env) + self.job._native_id = self.process.pid + self.executor._set_job_status(self.job, + JobStatus(JobState.ACTIVE, time=time.time(), + metadata={'nativeId': self.job._native_id})) + self.exit_code = self.process.wait() + + # We want to capture errors in the launcher scripts. Since, under normal circumstances, + # the exit code of the launcher is the exit code of the job, we must use a different + # mechanism to distinguish between job errors and launcher errors. So we delegate to + # the launcher implementation to figure out if the error belongs to the job or not + if self.process.stdout: + out = self.process.stdout.read().decode('utf-8') + else: + out = None + if out and launcher.is_launcher_failure(out): + message = self.process.launcher.get_launcher_failure_message(out) + self.fail_job(LauncherException(message)) + else: + self.state, self.exception = self._get_state_from_ec(self.exit_code) + finally: + if nodefile: + os.remove(nodefile) + + def cancel(self): + with self.job._status_cv: + self.cancel_flag = True + if self.process is not None: + self.process.kill() + + def _generate_nodefile(self, job: Job) -> Optional[str]: + assert job.spec is not None + if job.spec.resources is None: + return None + if job.spec.resources.version == 1: + assert isinstance(job.spec.resources, ResourceSpecV1) + n = job.spec.resources.computed_process_count + if n == 1: + # as a bit of an optimization, we don't generate a nodefile when doing "single + # node" jobs on local. + return None + (file, nodefile) = mkstemp(suffix='.nodelist') + for i in range(n): + os.write(file, 'localhost\n'.encode()) + os.close(file) + return nodefile + else: + raise SubmitException('Cannot handle resource specification with version %s' + % job.spec.resources.version) + + def _get_launcher_name(self, spec: JobSpec) -> str: + if spec.launcher is None: + return 'single' + else: + return spec.launcher def _get_env(spec: JobSpec, nodefile: Optional[str]) -> Optional[Dict[str, str]]: @@ -143,80 +373,6 @@ def _get_env(spec: JobSpec, nodefile: Optional[str]) -> Optional[Dict[str, str]] return env -class _ProcessReaper(SingletonThread): - - @classmethod - def get_instance(cls: Type['_ProcessReaper']) -> '_ProcessReaper': - return cast('_ProcessReaper', super().get_instance()) - - def __init__(self) -> None: - super().__init__(name='Local Executor Process Reaper', daemon=True) - self._jobs: Dict[Job, _ProcessEntry] = {} - self._lock = threading.RLock() - self._cvar = threading.Condition() - - def register(self, entry: _ProcessEntry) -> None: - logger.debug('Registering process %s', entry) - with self._lock: - self._jobs[entry.job] = entry - - def run(self) -> None: - logger.debug('Started {}'.format(self)) - done: List[_ProcessEntry] = [] - while True: - with self._lock: - for entry in done: - del self._jobs[entry.job] - jobs = dict(self._jobs) - try: - done = self._check_processes(jobs) - except Exception as ex: - logger.error('Error polling for process status', ex) - with self._cvar: - self._cvar.wait(_REAPER_SLEEP_TIME) - - def _handle_sigchld(self) -> None: - with self._cvar: - try: - self._cvar.notify_all() - except RuntimeError: - # In what looks like rare cases, notify_all(), seemingly when combined with - # signal handling, raises `RuntimeError: release unlocked lock`. - # There appears to be an unresolved Python bug about this: - # https://bugs.python.org/issue34486 - # We catch the exception here and log it. It is hard to tell if that will not lead - # to further issues. It would seem like it shouldn't: after all, all we're doing is - # making sure we don't sleep too much, but, even if we do, the consequence is a - # small delay in processing a completed job. However, since this exception seems - # to be a logical impossibility when looking at the code in threading.Condition, - # there is really no telling what else could go wrong. - logger.debug('Exception in Condition.notify_all()') - - def _check_processes(self, jobs: Dict[Job, _ProcessEntry]) -> List[_ProcessEntry]: - done: List[_ProcessEntry] = [] - - for entry in jobs.values(): - if entry.kill_flag: - entry.kill() - - exit_code, out = entry.poll() - if exit_code is not None: - entry.exit_code = exit_code - entry.done_time = time.time() - entry.out = out - done.append(entry) - - for entry in done: - entry.executor._process_done(entry) - - return done - - def cancel(self, job: Job) -> None: - with self._lock: - p = self._jobs[job] - p.kill_flag = True - - class LocalJobExecutor(JobExecutor): """ A job executor that runs jobs locally using :class:`subprocess.Popen`. @@ -245,27 +401,8 @@ def __init__(self, url: Optional[str] = None, :type config: psij.JobExecutorConfig """ super().__init__(url=url, config=config if config else JobExecutorConfig()) - self._reaper = _ProcessReaper.get_instance() - - def _generate_nodefile(self, job: Job, p: _ChildProcessEntry) -> Optional[str]: - assert job.spec is not None - if job.spec.resources is None: - return None - if job.spec.resources.version == 1: - assert isinstance(job.spec.resources, ResourceSpecV1) - n = job.spec.resources.computed_process_count - if n == 1: - # as a bit of an optimization, we don't generate a nodefile when doing "single - # node" jobs on local. - return None - (file, p.nodefile) = mkstemp(suffix='.nodelist') - for i in range(n): - os.write(file, 'localhost\n'.encode()) - os.close(file) - return p.nodefile - else: - raise SubmitException('Cannot handle resource specification with version %s' - % job.spec.resources.version) + self._threads_lock = threading.RLock() + self.threads: Dict[str, _JobThread] = {} def submit(self, job: Job) -> None: """ @@ -281,27 +418,16 @@ def submit(self, job: Job) -> None: """ spec = self._check_job(job) - p = _ChildProcessEntry(job, self, self._get_launcher(self._get_launcher_name(spec))) - assert p.launcher - args = p.launcher.get_launch_command(job) + self._set_job_status(job, JobStatus(JobState.QUEUED, time=time.time())) - try: - with job._status_cv: - if job.status.state == JobState.CANCELED: - raise SubmitException('Job canceled') - if logger.isEnabledFor(logging.DEBUG): - logger.debug('Running %s', _format_shell_cmd(args)) - nodefile = self._generate_nodefile(job, p) - env = _get_env(spec, nodefile) - p.process = subprocess.Popen(args, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, - close_fds=True, cwd=spec.directory, env=env) - self._reaper.register(p) - job._native_id = p.process.pid - self._set_job_status(job, JobStatus(JobState.QUEUED, time=time.time(), - metadata={'nativeId': job._native_id})) - self._set_job_status(job, JobStatus(JobState.ACTIVE, time=time.time())) - except Exception as ex: - raise SubmitException('Failed to submit job', exception=ex) + with job._status_cv: + if job.status.state == JobState.CANCELED: + raise SubmitException('Job canceled') + job_thread = _ChildJobThread(job, spec, self) + + with self._threads_lock: + self.threads[job.id] = job_thread + job_thread.start() def cancel(self, job: Job) -> None: """ @@ -309,27 +435,17 @@ def cancel(self, job: Job) -> None: :param job: The job to cancel. """ - self._set_job_status(job, JobStatus(JobState.CANCELED)) - self._reaper.cancel(job) - - def _process_done(self, p: _ProcessEntry) -> None: - assert p.exit_code is not None - message = None - if p.exit_code == 0: - state = JobState.COMPLETED - elif p.exit_code < 0 and p.kill_flag: - state = JobState.CANCELED - else: - # We want to capture errors in the launcher scripts. Since, under normal circumstances, - # the exit code of the launcher is the exit code of the job, we must use a different - # mechanism to distinguish between job errors and launcher errors. So we delegate to - # the launcher implementation to figure out if the error belongs to the job or not - if p.launcher and p.out and p.launcher.is_launcher_failure(p.out): - message = p.launcher.get_launcher_failure_message(p.out) - state = JobState.FAILED - self._set_job_status(p.job, JobStatus(state, time=p.done_time, exit_code=p.exit_code, - message=message)) + with self._threads_lock: + try: + job_thread = self.threads[job.id] + except KeyError: + raise ValueError('The job %s is not managed by this executor.' % job.id) + with job._status_cv: + if job_thread is not None: + job_thread.cancel() + else: + self._set_job_status(job, JobStatus(JobState.CANCELED)) def list(self) -> List[str]: """ @@ -364,15 +480,11 @@ def attach(self, job: Job, native_id: str) -> None: job.executor = self pid = int(native_id) - self._reaper.register(_AttachedProcessEntry(job, psutil.Process(pid), self)) - # We assume that the native_id above is a PID that was obtained at some point using - # list(). If so, the process is either still running or has completed. Either way, we must - # bring it up to ACTIVE state - self._set_job_status(job, JobStatus(JobState.QUEUED, time=time.time())) - self._set_job_status(job, JobStatus(JobState.ACTIVE, time=time.time())) + with job._status_cv: + if job.status.state == JobState.CANCELED: + raise SubmitException('Job canceled') + job_thread = _AttachedJobThread(job, pid, self) - def _get_launcher_name(self, spec: JobSpec) -> str: - if spec.launcher is None: - return 'single' - else: - return spec.launcher + with self._threads_lock: + self.threads[job.id] = job_thread + job_thread.start() diff --git a/src/psij/job_spec.py b/src/psij/job_spec.py index 7eec03ec..6d746c04 100644 --- a/src/psij/job_spec.py +++ b/src/psij/job_spec.py @@ -3,12 +3,13 @@ # for some reason, Sphinx cannot find Path if imported directly # from pathlib import Path import pathlib -from typing import Dict, List, Optional, Union +from typing import Dict, List, Optional, Union, Set from typeguard import check_argument_types import psij.resource_spec import psij.job_attributes +from psij.staging import StageIn, StageOut def _to_path(arg: Union[str, pathlib.Path, None]) -> Optional[pathlib.Path]: @@ -32,6 +33,9 @@ def _to_env_dict(arg: Union[Dict[str, Union[str, int]], None]) -> Optional[Dict[ ret[k] = v return ret +def _all_to_path(s: Set[Union[str, pathlib.Path]]) -> Set[pathlib.Path]: + return set(map(_to_path, s)) + class JobSpec(object): """A class that describes the details of a job.""" @@ -54,7 +58,11 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st attributes: Optional[psij.job_attributes.JobAttributes] = None, pre_launch: Union[str, pathlib.Path, None] = None, post_launch: Union[str, pathlib.Path, None] = None, - launcher: Optional[str] = None): + launcher: Optional[str] = None, + stage_in: Optional[Set[StageIn]] = None, + stage_out: Optional[Set[StageOut]] = None, + cleanup: Optional[Set[Union[str, pathlib.Path]]] = None, + cleanup_on_failure: bool = True): """ :param executable: An executable, such as "/bin/date". :param arguments: The argument list to be passed to the executable. Unlike with execve(), @@ -88,6 +96,10 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st node as the pre-launch script. :param launcher: The name of a launcher to use, such as "mpirun", "srun", "single", etc. For a list of available launchers, see :ref:`Available Launchers `. + :param stage_in: Specifies a set of files to be staged in before the job is launched. + :param stage_out: Specifies a set of files to be staged out after the job terminates. + :param cleanup: Specifies a set of files to remove after the stage out process. + All constructor parameters are accessible as properties. @@ -148,6 +160,9 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st self._pre_launch = _to_path(pre_launch) self._post_launch = _to_path(post_launch) self.launcher = launcher + self.stage_in = stage_in + self.stage_out = stage_out + self._cleanup = _all_to_path(cleanup) # TODO: `resources` is of type `ResourceSpec`, not `ResourceSpecV1`. An # connector trying to access `job.spec.resources.process_count` @@ -239,6 +254,14 @@ def post_launch(self) -> Optional[pathlib.Path]: def post_launch(self, post_launch: Union[str, pathlib.Path, None]) -> None: self._post_launch = _to_path(post_launch) + @property + def cleanup(self) -> Set[pathlib.Path]: + return self._cleanup + + @cleanup.setter + def cleanup(self, cleanup: Set[Union[str, pathlib.Path]]) -> None: + self._cleanup = _all_to_path(cleanup) + def __eq__(self, o: object) -> bool: """ Tests if this JobSpec is equal to another. diff --git a/src/psij/job_state.py b/src/psij/job_state.py index b028f456..01c85ef0 100644 --- a/src/psij/job_state.py +++ b/src/psij/job_state.py @@ -2,6 +2,8 @@ from typing import Optional +_NAME_MAP = {} + class JobState(bytes, Enum): """ An enumeration holding the possible job states. @@ -16,6 +18,7 @@ def __new__(cls, index: int, order: int, name: str, final: bool) -> 'JobState': obj._order = order obj._name = name obj._final = final + _NAME_MAP[name] = obj return obj def __init__(self, *args: object) -> None: # noqa: D107 @@ -34,19 +37,32 @@ def __init__(self, *args: object) -> None: # noqa: D107 This is the state of the job after being accepted by a backend for execution, but before the execution of the job begins. """ - ACTIVE = (2, 2, 'ACTIVE', False) + STAGE_IN = (2, 2, 'STAGE_IN', False) + """ + This state indicates that the job is staging files in, in preparation for execution. + """ + ACTIVE = (3, 3, 'ACTIVE', False) """This state represents an actively running job.""" - COMPLETED = (3, 3, 'COMPLETED', True) + STAGE_OUT = (4, 4, 'STAGE_OUT', False) + """ + This state indicates that the executable has finished running and that files are being staged + out. + """ + CLEANUP = (5, 5, 'CLEANUP', False) + """ + This state indicates that cleanup is actively being done for this job. + """ + COMPLETED = (6, 6, 'COMPLETED', True) """ This state represents a job that has completed *successfully* (i.e., with a zero exit code). In other words, a job with the executable set to `/bin/false` cannot enter this state. """ - FAILED = (4, 3, 'FAILED', True) + FAILED = (7, 6, 'FAILED', True) """ Represents a job that has either completed unsuccessfully (with a non-zero exit code) or a job whose handling and/or execution by the backend has failed in some way. """ - CANCELED = (5, 3, 'CANCELED', True) + CANCELED = (8, 6, 'CANCELED', True) """Represents a job that was canceled by a call to :func:`~psij.Job.cancel()`.""" def is_greater_than(self, other: 'JobState') -> Optional[bool]: @@ -112,6 +128,22 @@ def __hash__(self) -> int: """Returns a hash for this object.""" return self._value_ # type: ignore + @staticmethod + def from_name(name: str) -> 'JobState': + return _NAME_MAP[name] + + +_PREV_STATE = { + JobState.NEW: None, + JobState.QUEUED: JobState.NEW, + JobState.STAGE_IN: JobState.QUEUED, + JobState.ACTIVE: JobState.STAGE_IN, + JobState.STAGE_OUT: JobState.ACTIVE, + JobState.CLEANUP: JobState.STAGE_OUT, + JobState.COMPLETED: JobState.CLEANUP, + JobState.FAILED: None, + JobState.CANCELED: None +} class JobStateOrder: """A class that can be used to reconstruct missing states.""" @@ -125,10 +157,4 @@ def prev(state: JobState) -> Optional[JobState]: previous state. For example, the FAILED state does not have a previous state, since it can be reached from multiple states. """ - if state == JobState.COMPLETED: - return JobState.ACTIVE - if state == JobState.ACTIVE: - return JobState.QUEUED - if state == JobState.QUEUED: - return JobState.NEW - return None + return _PREV_STATE[state] diff --git a/src/psij/staging.py b/src/psij/staging.py new file mode 100644 index 00000000..cc0329fc --- /dev/null +++ b/src/psij/staging.py @@ -0,0 +1,94 @@ +import urllib +from enum import Enum, Flag +from pathlib import Path +from typing import Optional, Union + + +class URI: + def __init__(self, urlstring: str) -> None: + self.parts = urllib.parse.urlparse(urlstring) + + # a __getattr__ solution may be simpler, but doesn't play well with IDEs and + # is not quite self-documenting + @property + def hostname(self) -> Optional[str]: + return self.hostname + + @property + def port(self) -> int: + return self.port + + @property + def scheme(self) -> str: + return self.parts.scheme + + @property + def netloc(self) -> str: + return self.parts.netloc + + @property + def path(self) -> str: + return self.parts.path + + @property + def params(self) -> str: + return self.parts.params + + @property + def query(self) -> str: + return self.parts.query + + @property + def fragment(self) -> str: + return self.parts.fragment + + @property + def username(self) -> str: + return self.parts.username + + @property + def password(self) -> str: + return self.parts.password + + def __str__(self) -> str: + return self.parts.geturl() + + +class StagingMode(Enum): + COPY = 1 + LINK = 2 + MOVE = 3 + + +class StageOutFlags(Flag): + IF_PRESENT = 1 + ON_SUCCESS = 2 + ON_ERROR = 4 + ON_CANCEL = 8 + ALWAYS = ON_SUCCESS | ON_ERROR | ON_CANCEL + + +class StageIn: + def __init__(self, source: Union[URI, str], target: Union[str, Path], + mode: StagingMode = StagingMode.COPY) -> None: + if isinstance(source, str): + source = URI(source) + if isinstance(target, str): + target = Path(target) + self.source = source + self.target = target + self.mode = mode + +class StageOut: + def __init__(self, source: Union[str, Path], target: Union[str, URI], + flags: StageOutFlags = StageOutFlags.ALWAYS, + mode: StagingMode = StagingMode.COPY): + if isinstance(source, str): + source = Path(source) + if isinstance(target, str): + target = URI(target) + + self.source = source + self.target = target + self.flags = flags + self.mode = mode diff --git a/tests/plugins1/_batch_test/test/test.mustache b/tests/plugins1/_batch_test/test/test.mustache index ae51ae68..134b0443 100644 --- a/tests/plugins1/_batch_test/test/test.mustache +++ b/tests/plugins1/_batch_test/test/test.mustache @@ -2,6 +2,8 @@ exec &> "{{psij.script_dir}}/$PSIJ_BATCH_TEST_JOB_ID.out" +{{> batch_lib}} + {{#job.spec.directory}} cd "{{.}}" {{/job.spec.directory}} @@ -42,8 +44,26 @@ done export PSIJ_NODEFILE +update_status STAGEIN +{{#stagein_set}} + do_stagein "{{source}}" "{{target}}" {{mode}} +{{/stagein_set}} + +update_status ACTIVE {{#job.spec.inherit_environment}}env \{{/job.spec.inherit_environment}}{{^job.spec.inherit_environment}}env --ignore-environment \{{/job.spec.inherit_environment}}{{#env}} {{name}}="{{value}}" \{{/env}} {{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_EC="$?" + +update_status STAGEOUT +{{#stageout_set}} + do_stageout "{{source}}" "{{target}}" {{mode}} {{flags}} $_EC +{{/stageout_set}} + +update_status CLEANUP +{{#cleanup_set}} + do_cleanup {{.}} +{{/cleanup_set}} -echo "$?" > "{{psij.script_dir}}/$PSIJ_BATCH_TEST_JOB_ID.ec" +echo "$_EC" > "{{psij.script_dir}}/$PSIJ_BATCH_TEST_JOB_ID.ec" +>>>>>>> 554b43e (Initial staging commit) diff --git a/tests/test_staging.py b/tests/test_staging.py new file mode 100644 index 00000000..2d6db631 --- /dev/null +++ b/tests/test_staging.py @@ -0,0 +1,25 @@ +from tempfile import NamedTemporaryFile + +from executor_test_params import ExecutorTestParams +from _test_tools import _get_executor_instance, _get_timeout, assert_completed, _make_test_dir +from psij import Job, JobSpec + + +def test_stagein(execparams: ExecutorTestParams) -> None: + with NamedTemporaryFile(delete=False) as outf: + outf.close() + with NamedTemporaryFile(mode='w', delete=False) as f: + + f.write('ABCD') + f.close() + + job = Job(JobSpec(executable='/bin/cat', stdout_path=outf.name, + launcher=execparams.launcher)) + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + + with open(outf, 'r') as out: + result = out.read() + assert result.strip() == 'ABCD' From 57ea7e78374f32943339f24dcfe9a33ff35e7832 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 09:50:22 -0800 Subject: [PATCH 02/39] Staging updates --- .gitignore | 1 + src/psij-descriptors/cobalt_descriptor.py | 2 +- src/psij-descriptors/core_descriptors.py | 8 +- src/psij-descriptors/lsf_descriptor.py | 2 +- src/psij-descriptors/pbs_descriptor.py | 4 +- src/psij-descriptors/slurm_descriptor.py | 2 +- src/psij/exceptions.py | 18 - .../batch/batch_scheduler_executor.py | 202 +++--- .../executors/batch/cobalt/cobalt.mustache | 11 +- .../executors/batch/common/batch_lib.mustache | 122 +++- .../executors/batch/common/cleanup.mustache | 5 + .../executors/batch/common/stagein.mustache | 5 + .../executors/batch/common/stageout.mustache | 5 + src/psij/executors/batch/lsf/lsf.mustache | 11 +- .../executors/batch/pbs/pbs_classic.mustache | 12 +- src/psij/executors/batch/pbs/pbspro.mustache | 10 +- src/psij/executors/batch/script_generator.py | 19 +- src/psij/executors/batch/slurm/slurm.mustache | 11 +- src/psij/executors/local.py | 603 ++++++++---------- src/psij/executors/local/local.mustache | 25 + src/psij/job_spec.py | 20 +- src/psij/job_state.py | 7 + src/psij/launchers/script_based_launcher.py | 8 +- src/psij/staging.py | 211 +++++- src/psij/utils.py | 158 ++++- tests/_test_tools.py | 68 +- tests/plugins1/_batch_test/test/test.mustache | 22 +- tests/test_callbacks.py | 15 +- tests/test_staging.py | 228 ++++++- 29 files changed, 1216 insertions(+), 599 deletions(-) create mode 100644 src/psij/executors/batch/common/cleanup.mustache create mode 100644 src/psij/executors/batch/common/stagein.mustache create mode 100644 src/psij/executors/batch/common/stageout.mustache create mode 100644 src/psij/executors/local/local.mustache diff --git a/.gitignore b/.gitignore index 8f0c3dc8..bcc9a4bf 100644 --- a/.gitignore +++ b/.gitignore @@ -17,3 +17,4 @@ psi_j_python.egg-info/ venv* .venv* build/ +.packages/ \ No newline at end of file diff --git a/src/psij-descriptors/cobalt_descriptor.py b/src/psij-descriptors/cobalt_descriptor.py index 7ea1ad27..eb73a3a1 100644 --- a/src/psij-descriptors/cobalt_descriptor.py +++ b/src/psij-descriptors/cobalt_descriptor.py @@ -3,5 +3,5 @@ from psij.descriptor import Descriptor -__PSI_J_EXECUTORS__ = [Descriptor(name="cobalt", nice_name='Cobalt', version=StrictVersion("0.0.1"), +__PSI_J_EXECUTORS__ = [Descriptor(name="cobalt", nice_name='Cobalt', version=StrictVersion("0.2.0"), cls='psij.executors.batch.cobalt.CobaltJobExecutor')] diff --git a/src/psij-descriptors/core_descriptors.py b/src/psij-descriptors/core_descriptors.py index e5a1a741..5212f204 100644 --- a/src/psij-descriptors/core_descriptors.py +++ b/src/psij-descriptors/core_descriptors.py @@ -2,15 +2,15 @@ from psij.descriptor import Descriptor __PSI_J_EXECUTORS__ = [ - Descriptor(name='local', nice_name='Local', version=StrictVersion('0.0.1'), + Descriptor(name='local', nice_name='Local', version=StrictVersion('0.2.0'), cls='psij.executors.local.LocalJobExecutor') ] __PSI_J_LAUNCHERS__ = [ - Descriptor(name='single', version=StrictVersion('0.0.1'), + Descriptor(name='single', version=StrictVersion('0.2.0'), cls='psij.launchers.single.SingleLauncher'), - Descriptor(name='multiple', version=StrictVersion('0.0.1'), + Descriptor(name='multiple', version=StrictVersion('0.2.0'), cls='psij.launchers.multiple.MultipleLauncher'), - Descriptor(name='mpirun', version=StrictVersion('0.0.1'), + Descriptor(name='mpirun', version=StrictVersion('0.2.0'), cls='psij.launchers.mpirun.MPILauncher'), ] diff --git a/src/psij-descriptors/lsf_descriptor.py b/src/psij-descriptors/lsf_descriptor.py index 8c002b30..c0677ec9 100644 --- a/src/psij-descriptors/lsf_descriptor.py +++ b/src/psij-descriptors/lsf_descriptor.py @@ -3,5 +3,5 @@ from psij.descriptor import Descriptor -__PSI_J_EXECUTORS__ = [Descriptor(name='lsf', nice_name='LSF', version=StrictVersion('0.0.1'), +__PSI_J_EXECUTORS__ = [Descriptor(name='lsf', nice_name='LSF', version=StrictVersion('0.2.0'), cls='psij.executors.batch.lsf.LsfJobExecutor')] diff --git a/src/psij-descriptors/pbs_descriptor.py b/src/psij-descriptors/pbs_descriptor.py index d603f83c..29f22d9a 100644 --- a/src/psij-descriptors/pbs_descriptor.py +++ b/src/psij-descriptors/pbs_descriptor.py @@ -4,8 +4,8 @@ __PSI_J_EXECUTORS__ = [Descriptor(name='pbs', nice_name='PBS Pro', aliases=['pbspro'], - version=StrictVersion('0.0.2'), + version=StrictVersion('0.2.0'), cls='psij.executors.batch.pbs.PBSJobExecutor'), Descriptor(name='pbs_classic', nice_name='PBS Classic', aliases=['torque'], - version=StrictVersion('0.0.2'), + version=StrictVersion('0.2.0'), cls='psij.executors.batch.pbs_classic.PBSClassicJobExecutor')] diff --git a/src/psij-descriptors/slurm_descriptor.py b/src/psij-descriptors/slurm_descriptor.py index 5ec5465b..732f4af6 100644 --- a/src/psij-descriptors/slurm_descriptor.py +++ b/src/psij-descriptors/slurm_descriptor.py @@ -3,5 +3,5 @@ from psij.descriptor import Descriptor -__PSI_J_EXECUTORS__ = [Descriptor(name='slurm', nice_name='Slurm', version=StrictVersion('0.0.1'), +__PSI_J_EXECUTORS__ = [Descriptor(name='slurm', nice_name='Slurm', version=StrictVersion('0.2.0'), cls='psij.executors.batch.slurm.SlurmJobExecutor')] diff --git a/src/psij/exceptions.py b/src/psij/exceptions.py index 171ed252..8e543d9d 100644 --- a/src/psij/exceptions.py +++ b/src/psij/exceptions.py @@ -62,21 +62,3 @@ def __init__(self, message: str, exception: Optional[Exception] = None, conditions such an error would persist across subsequent re-tries until correct credentials are used. """ - -class CompositeException(Exception): - def __init__(self, ex: Exception) -> None: - self.exceptions = [ex] - - def add_exception(self, ex: Exception) -> None: - self.exceptions.append(ex) - - -class LauncherException(Exception): - def __init__(self, message: str) -> None: - super().__init__('Launcher failure: %s' % message) - - -class JobException(Exception): - def __init__(self, exit_code: int) -> None: - super().__init__('Job exited with exit code %s' % exit_code) - self.exit_code = exit_code diff --git a/src/psij/executors/batch/batch_scheduler_executor.py b/src/psij/executors/batch/batch_scheduler_executor.py index b3bf5ca2..090138b0 100644 --- a/src/psij/executors/batch/batch_scheduler_executor.py +++ b/src/psij/executors/batch/batch_scheduler_executor.py @@ -1,12 +1,7 @@ -import atexit -import fcntl -import io import logging import os -import tempfile +import weakref -import psutil -import socket import subprocess import time import traceback @@ -14,14 +9,14 @@ from datetime import timedelta from pathlib import Path from threading import Thread, RLock -from typing import Optional, List, Dict, Collection, cast, Union, IO +from typing import Optional, List, Dict, Collection, cast, Union, IO, Set from psij.launchers.script_based_launcher import ScriptBasedLauncher from psij import JobExecutor, JobExecutorConfig, Launcher, Job, SubmitException, \ JobStatus, JobState from psij.executors.batch.template_function_library import ALL as FUNCTION_LIBRARY -from psij.utils import SingletonThread +from psij.utils import _StatusUpdater UNKNOWN_ERROR = 'PSIJ: Unknown error' @@ -207,18 +202,14 @@ def __init__(self, url: Optional[str] = None, configuration is used. """ super().__init__(url=url, config=config if config else BatchSchedulerExecutorConfig()) + self._queue_poll_thread = self._start_queue_poll_thread() assert config self.work_directory = config.work_directory / self.name - self._queue_poll_thread = self._start_queue_poll_thread() - - def _ensure_work_dir(self) -> None: self.work_directory.mkdir(parents=True, exist_ok=True) def submit(self, job: Job) -> None: """See :func:`~psij.JobExecutor.submit`.""" logger.info('Job %s: submitting', job.id) - self._ensure_work_dir() - self._check_job(job) context = self._create_script_context(job) @@ -273,8 +264,6 @@ def cancel(self, job: Job) -> None: except SubmitException: # re-raise raise - finally: - self._status_update_thread.unregister_job(job) def attach(self, job: Job, native_id: str) -> None: """Attaches a job to a native job. @@ -499,7 +488,10 @@ def _create_script_context(self, job: Job) -> Dict[str, object]: 'psij': { 'lib': FUNCTION_LIBRARY, 'launch_command': launch_command, - 'script_dir': str(self.work_directory) + 'script_dir': str(self.work_directory), + 'us_file': self._queue_poll_thread.status_updater.update_file_name, + 'us_port': self._queue_poll_thread.status_updater.update_port, + 'us_addrs': ', '.join(self._queue_poll_thread.status_updater.ips) } } assert job.spec is not None @@ -542,7 +534,7 @@ def _set_job_status(self, job: Job, status: JobStatus) -> None: # we have to check explicitly for the boolean value rather than truthiness return if status.state.final: - self._status_update_thread.unregister_job(job) + self._queue_poll_thread.unregister_job(job) if job.native_id: self._clean_submit_script(job) self._read_aux_files(job, status) @@ -643,35 +635,46 @@ def __init__(self, name: str, config: BatchSchedulerExecutorConfig, super().__init__() self.name = name self.daemon = True - self.config = config - self.executor = executor + # We don't at this time cache executor instances. Even if we did, it may be wise + # to shut down queue polling threads when their executors (the only entities that + # use them) are garbage collected. So we wrap the references to the executor and + # config in a weak ref and exit when the ref becomes invalid. + self.config = weakref.ref(config) + self.executor = weakref.ref(executor) # native_id -> job - self._jobs: Dict[str, List[Job]] = {} + self._jobs: Dict[str, Set[Job]] = {} # counts consecutive errors while invoking qstat or equivalent self._poll_error_count = 0 self._jobs_lock = RLock() - self._status_updater = _StatusUpdater(config, executor) + self.status_updater = cast(_StatusUpdater, _StatusUpdater.get_instance()) + self.active = True def run(self) -> None: logger.debug('Executor %s: queue poll thread started', self.executor) - time.sleep(self.config.initial_queue_polling_delay) - while True: - self._poll() - start = time.time() - now = start - while now - start < self.config.queue_polling_interval: - self._status_updater.step() - time.sleep(1) - now = time.time() + try: + time.sleep(self.get_config().initial_queue_polling_delay) + while self.active: + self._poll() + start = time.time() + now = start + while now - start < self.get_config().queue_polling_interval: + time.sleep(1) + now = time.time() + except StopIteration: + logger.info('Thread %s exiting due to executor collection' % self) + + def stop(self) -> None: + self.active = False def _poll(self) -> None: + executor = self.get_executor() with self._jobs_lock: if len(self._jobs) == 0: return jobs_copy = dict(self._jobs) logger.info('Polling for %s jobs', len(jobs_copy)) try: - out = self.executor._run_command(self.executor.get_status_command(jobs_copy.keys())) + out = executor._run_command(executor.get_status_command(jobs_copy.keys())) except subprocess.CalledProcessError as ex: out = ex.output exit_code = ex.returncode @@ -685,27 +688,23 @@ def _poll(self) -> None: self._poll_error_count = 0 logger.debug('Output from status command: %s', out) try: - status_map = self.executor.parse_status_output(exit_code, out) + status_map = executor.parse_status_output(exit_code, out) except Exception as ex: self._handle_poll_error(False, ex, f'Failed to poll for job status: {traceback.format_exc()}') return try: - for native_id, job_list in jobs_copy.items(): + for native_id, job_set in jobs_copy.items(): try: status = self._get_job_status(native_id, status_map) except Exception: status = JobStatus(JobState.FAILED, message='Failed to update job status: %s' % traceback.format_exc()) - for job in job_list: - self.executor._set_job_status(job, status) - if status.state.final: - with self._jobs_lock: - del self._jobs[native_id] - for job in job_list: - self._status_updater.unregister_job(job) + + for job in job_set: + executor._set_job_status(job, status) except Exception as ex: msg = traceback.format_exc() self._handle_poll_error(True, ex, 'Error updating job statuses {}'.format(msg)) @@ -719,7 +718,7 @@ def _get_job_status(self, native_id: str, status_map: Dict[str, JobStatus]) -> J def _handle_poll_error(self, immediate: bool, ex: Exception, msg: str) -> None: logger.warning('Polling error: %s', msg) self._poll_error_count += 1 - if immediate or (self._poll_error_count > self.config.queue_polling_error_threshold): + if immediate or (self._poll_error_count > self.get_config().queue_polling_error_threshold): self._poll_error_count = 0 # fail all jobs with self._jobs_lock: @@ -731,106 +730,47 @@ def _handle_poll_error(self, immediate: bool, ex: Exception, msg: str) -> None: assert len(self._jobs) > 0 jobs_copy = dict(self._jobs) self._jobs.clear() - for job_list in jobs_copy.values(): - for job in job_list: - self._status_updater.unregister_job(job) - self.executor._set_job_status(job, JobStatus(JobState.FAILED, message=msg)) + for job_set in jobs_copy.values(): + for job in job_set: + self.unregister_job(job) + self.get_executor()._set_job_status(job, JobStatus(JobState.FAILED, + message=msg)) def register_job(self, job: Job) -> None: - self._status_updater.register_job(job) + self.status_updater.register_job(job, self.get_executor()) assert job.native_id logger.info('Job %s: registering', job.id) with self._jobs_lock: native_id = job.native_id - if native_id not in self._jobs: - self._jobs[native_id] = [job] - else: - self._jobs[job.native_id].append(job) - -class _StatusUpdater: - # we are expecting short messages in the form - RECV_BUFSZ = 2048 - - def __init__(self, config: BatchSchedulerExecutorConfig, - executor: BatchSchedulerExecutor) -> None: - self.config = config - self.executor = executor - self.socket = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) - self.socket.setblocking(False) - self.socket.bind(('', 0)) - self.port = self.socket.getsockname()[1] - self.ips = self._get_ips() - print('IPS: %s' % self.ips) - print('Port: %s' % self.port) - self._create_update_file() - print('Update file: %s' % self.update_file.name) - self.partial_file_data = '' - self.partial_net_data = '' - self._jobs = {} - self._jobs_lock = RLock() - - def _get_ips(self) -> List[str]: - addrs = psutil.net_if_addrs() - r = [] - for name, l in addrs.items(): - if name == 'lo': - continue - for a in l: - if a.family == socket.AddressFamily.AF_INET: - r.append(a.address) - return r - - def _create_update_file(self) -> None: - f = tempfile.NamedTemporaryFile(dir=self.config.work_directory, prefix='supd_', - delete=False) - name = f.name - atexit.register(os.remove, name) - f.close() - self.update_file = open(name, 'r+b') - self.update_file.seek(0, io.SEEK_END) - self.update_file_pos = self.update_file.tell() - - def register_job(self, job: Job) -> None: - with self._jobs_lock: - self._jobs[job.id] = job + try: + self._jobs[native_id].add(job) + except KeyError: + self._jobs[native_id] = {job} def unregister_job(self, job: Job) -> None: + self.status_updater.unregister_job(job) + assert job.native_id + logger.info('Job %s: unregistering', job.id) with self._jobs_lock: - del self._jobs[job.id] - - def step(self) -> None: - self.update_file.seek(0, io.SEEK_END) - pos = self.update_file.tell() - if pos > self.update_file_pos: - self.update_file.seek(self.update_file_pos, io.SEEK_SET) - n = pos - self.update_file_pos - self._process_update_data(self.update_file.read(n)) - self.update_file_pos = pos - else: + native_id = job.native_id try: - data = self.socket.recv(_StatusUpdater.RECV_BUFSZ) - self._process_update_data(data) - except socket.error as e: + del self._jobs[native_id] + except KeyError: + # If two or more jobs are attached to the same native ID, the + # first one being unregistered would already have removed + # the dict entry pass - def _process_update_data(self, data: bytes) -> None: - sdata = data.decode('utf-8') - lines = sdata.splitlines() - for line in lines: - print('Status update line: %s' % line) - els = line.split() - if len(els) != 2: - logger.warning('Invalid status update message received: %s' % line) - continue - job_id = els[0] - state = JobState.from_name(els[1]) - job = None - with self._jobs_lock: - try: - job = self._jobs[job_id] - except KeyError: - logger.warning('Received status updated for inexistent job with id %s' % job_id) - if job: - self.executor._set_job_status(job, JobStatus(state)) - + def get_config(self) -> BatchSchedulerExecutorConfig: + config = self.config() + if config: + return config + else: + raise StopIteration() + def get_executor(self) -> BatchSchedulerExecutor: + ex = self.executor() + if ex: + return ex + else: + raise StopIteration() diff --git a/src/psij/executors/batch/cobalt/cobalt.mustache b/src/psij/executors/batch/cobalt/cobalt.mustache index 922e9eea..b41cdb75 100644 --- a/src/psij/executors/batch/cobalt/cobalt.mustache +++ b/src/psij/executors/batch/cobalt/cobalt.mustache @@ -46,16 +46,25 @@ only results in empty files that are not cleaned up}} #COBALT -e /dev/null #COBALT -o /dev/null +{{> batch_lib}} + {{!like PBS, this is also cheap and there is not need to check setting}} PSIJ_NODEFILE="$COBALT_NODEFILE" export PSIJ_NODEFILE +{{> stagein}} +update_status ACTIVE + {{!redirect output here instead of through #COBALT directive since COBALT_JOB_ID is not available when the directives are evaluated; the reason for using the job id in the first place being the same as for the exit code file.}} exec &>> "{{psij.script_dir}}/$COBALT_JOBID.out" {{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_PSIJ_JOB_EC=$? + +{{> stageout}} +{{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} -echo "$?" > "{{psij.script_dir}}/$COBALT_JOBID.ec" +echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$COBALT_JOBID.ec" diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index ef282cf9..f0fe162d 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -4,23 +4,50 @@ update_status() { STATUS="$1" if [ "$_UPDATE_MODE" == "none" ]; then if which nc >/dev/null; then - $_UPDATE_MODE="nc" + _UPDATE_MODE="nc" else - $_UPDATE_MODE="file" + _UPDATE_MODE="file" fi fi if [ "$_UPDATE_MODE" == "nc" ]; then - for ADDR in "{{psij.us_addrs}}"; do - echo "{{psij.job_id}} $STATUS" | nc -q0 -w0 -4 -u $ADDR {{psij.us_port}} + ADDRS={{psij.us_addrs}} + for ADDR in ${ADDRS//,/ }; do + echo "{{job.id}} $STATUS" | nc -q0 -w0 -4 -u $ADDR {{psij.us_port}} done else - echo "{{psij.job_id}} $STATUS" >> {{psij.us_file}} + echo "{{job.id}} $STATUS" >> {{psij.us_file}} + fi +} + +fail() { + [ "{{psij.debug}}" != "0" ] && update_status "LOG Failing: $2" + echo $2 + exit $1 +} + +check_remote() { + SCHEME="$1" + HOSTPORT="$2" + + if [ "$SCHEME" != "" ] && [ "$SCHEME" != "file" ]; then + fail 121 "$SCHEME staging is not supported" + fi + if [ "$HOSTPORT" != "" ] && [ "$HOSTPORT" != "localhost" ]; then + fail 121 "The host, if specified, must be \"localhost\". Got \"$HOSTPORT\"." fi } do_stagein() { - do_stage "$@" 0 + SOURCE="$1" + TARGET="$2" + MODE="$3" + SCHEME="$6" + HOSTPORT="$7" + + check_remote "$SCHEME" "$HOSTPORT" || exit $? + + do_stage "$SOURCE" "$TARGET" "$MODE" 0 } do_stage() { @@ -29,20 +56,43 @@ do_stage() { MODE="$3" MISSING_OK="$4" - if [ ! -a "$SOURCE" ] && [ "$MISSING_OK" == "0" ]; then - echo "Missing source file: $SOURCE" - exit 1 + [ "{{psij.debug}}" != "0" ] && update_status "LOG Stage $SOURCE -> $TARGET, mode: $MODE, missingok: $MISSING_OK" + + if [ ! -e "$SOURCE" ]; then + if [ "$MISSING_OK" == "0" ]; then + [ "{{psij.debug}}" != "0" ] && update_status "LOG Missing source file: $SOURCE" + fail 121 "Missing source file: $SOURCE" + else + [ "{{psij.debug}}" != "0" ] && update_status "LOG Skipping staging of missing file $SOURCE" + return 0 + fi + fi + + [ "{{psij.debug}}" != "0" ] && update_status "LOG Staging $SOURCE to $TARGET" + + TARGET_DIR=`dirname "$TARGET"` + + if [ "$TARGET_DIR" != "" ]; then + mkdir -p "$TARGET_DIR" + fi + + if [ -d "$TARGET" ] && [ ! -d "$SOURCE" ]; then + fail 121 "Target is a directory: $TARGET" fi if [ "$MODE" == "1" ]; then # copy - cp -r -T "$SOURCE" "$TARGET" - elif [ "$MODE" == "2" ]; tben + cp -r -T "$SOURCE" "$TARGET" || fail 121 "Failed to copy \"$SOURCE\" to \"$TARGET\"" + elif [ "$MODE" == "2" ]; then # link - ln -s "$SOURCE" "$TARGET" + {{!we want the same semantics as cp and mv, which is "overwrite if exists"}} + {{!we resolve the source since it may be a path relative to the job dir}} + rm -f "$TARGET" + SOURCE=`readlink -m $SOURCE` + ln -s "$SOURCE" "$TARGET" || fail 121 "Failed to link \"$SOURCE\" to \"$TARGET\"" elif [ "$MODE" == "3" ]; then # move - mv -T -f "$SOURCE" "$TARGET" + mv -T -f "$SOURCE" "$TARGET" || fail 121 "Failed to move \"$SOURCE\" to \"$TARGET\"" fi } @@ -57,24 +107,46 @@ do_stageout() { MODE="$3" FLAGS="$4" FAILED="$5" + SCHEME="$6" + HOSTPORT="$7" + + check_remote "$SCHEME" "$HOSTPORT" - if [ "$FAILED" == "0" ] && $((FLAGS & _FLAG_ON_SUCCESS)) ; then + [ "{{psij.debug}}" != "0" ] && update_status "LOG do_stageout $SOURCE -> $TARGET, mode: $MODE, flags: $FLAGS, failed: $FAILED" + + if [ "$FAILED" == "0" ] && [ "$((FLAGS & _FLAG_ON_SUCCESS))" != "0" ]; then do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) - elif [ "$FAILED" != "0" ] && $((FLAGS & _FLAG_ON_ERROR)) ; then + elif [ "$FAILED" != "0" ] && [ "$((FLAGS & _FLAG_ON_ERROR))" != "0" ]; then do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) fi } do_cleanup() { TARGET="$1" + FAILED="$2" + + if [ "$FAILED" == "0" ] || [ "{{job.spec.cleanup_on_failure}}" != "0" ]; then - case "$TARGET" in - "{{job.spec.directory}}"*) - echo "rm -rf $TARGET" >>~/cleanup.log - ;; - *) - echo "Cannot clean $TARGET outside of job directory {{job.spec.directory}}" - exit 1 - ;; - esac -} \ No newline at end of file + TARGET=`readlink -m "$TARGET"` + + [ "{{psij.debug}}" != "0" ] && update_status "LOG Cleaning up $TARGET" + + case "$TARGET" in + "{{job.spec.directory}}"*) + rm -rf "$TARGET" + ;; + *) + fail 121 "Cannot clean $TARGET outside of job directory {{job.spec.directory}}" + ;; + esac + fi +} + +stagein() { + update_status STAGE_IN + +{{#job.spec.stage_in}} + do_stagein "{{source.path}}" "{{target}}" {{mode}} \ + "{{{source.scheme}}}" "{{#source.hostname}}{{{.}}}{{#source.port}}:{{{.}}}{{/source.port}}{{/source.hostname}}" +{{/job.spec.stage_in}} +} diff --git a/src/psij/executors/batch/common/cleanup.mustache b/src/psij/executors/batch/common/cleanup.mustache new file mode 100644 index 00000000..0f161c4a --- /dev/null +++ b/src/psij/executors/batch/common/cleanup.mustache @@ -0,0 +1,5 @@ +update_status CLEANUP + +{{#job.spec.cleanup}} +do_cleanup {{.}} $_PSIJ_JOB_EC +{{/job.spec.cleanup}} diff --git a/src/psij/executors/batch/common/stagein.mustache b/src/psij/executors/batch/common/stagein.mustache new file mode 100644 index 00000000..7cad872b --- /dev/null +++ b/src/psij/executors/batch/common/stagein.mustache @@ -0,0 +1,5 @@ +update_status STAGE_IN +{{#job.spec.stage_in}} +do_stagein "{{source.path}}" "{{target}}" {{mode}} \ + "{{{source.scheme}}}" "{{#source.hostname}}{{{.}}}{{#source.port}}:{{{.}}}{{/source.port}}{{/source.hostname}}" +{{/job.spec.stage_in}} diff --git a/src/psij/executors/batch/common/stageout.mustache b/src/psij/executors/batch/common/stageout.mustache new file mode 100644 index 00000000..8d63dd64 --- /dev/null +++ b/src/psij/executors/batch/common/stageout.mustache @@ -0,0 +1,5 @@ +update_status STAGE_OUT +{{#job.spec.stage_out}} +do_stageout "{{source}}" "{{target.path}}" {{mode}} {{flags}} $_PSIJ_JOB_EC \ + "{{{target.scheme}}}" "{{#target.hostname}}{{{.}}}{{#target.port}}:{{{.}}}{{/target.port}}{{/target.hostname}}" +{{/job.spec.stage_out}} diff --git a/src/psij/executors/batch/lsf/lsf.mustache b/src/psij/executors/batch/lsf/lsf.mustache index 781e9f82..0ca54ecb 100644 --- a/src/psij/executors/batch/lsf/lsf.mustache +++ b/src/psij/executors/batch/lsf/lsf.mustache @@ -71,15 +71,24 @@ only results in empty files that are not cleaned up}} #BSUB -e /dev/null #BSUB -o /dev/null +{{> batch_lib}} + PSIJ_NODEFILE="$LSB_HOSTS" export PSIJ_NODEFILE +{{> stagein}} +update_status ACTIVE + {{!redirect output here instead of through #BSUB directive since LSB_JOBID is not available when the directives are evaluated; the reason for using the job id in the first place being the same as for the exit code file.}} exec &>> "{{psij.script_dir}}/$LSB_JOBID.out" {{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_PSIJ_JOB_EC=$? + +{{> stageout}} +{{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} -echo "$?" > "{{psij.script_dir}}/$LSB_JOBID.ec" +echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$LSB_JOBID.ec" diff --git a/src/psij/executors/batch/pbs/pbs_classic.mustache b/src/psij/executors/batch/pbs/pbs_classic.mustache index a531cfa6..a7859437 100644 --- a/src/psij/executors/batch/pbs/pbs_classic.mustache +++ b/src/psij/executors/batch/pbs/pbs_classic.mustache @@ -55,17 +55,25 @@ only results in empty files that are not cleaned up}} #PBS -v {{name}}={{value}} {{/env}} +{{> batch_lib}} + PSIJ_NODEFILE="$PBS_NODEFILE" export PSIJ_NODEFILE - {{#job.spec.directory}} cd "{{.}}" {{/job.spec.directory}} +{{> stagein}} +update_status ACTIVE + exec &>> "{{psij.script_dir}}/$PBS_JOBID.out" {{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_PSIJ_JOB_EC=$? + +{{> stageout}} +{{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} -echo "$?" > "{{psij.script_dir}}/$PBS_JOBID.ec" +echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$PBS_JOBID.ec" diff --git a/src/psij/executors/batch/pbs/pbspro.mustache b/src/psij/executors/batch/pbs/pbspro.mustache index 001384cf..78a3d2ae 100644 --- a/src/psij/executors/batch/pbs/pbspro.mustache +++ b/src/psij/executors/batch/pbs/pbspro.mustache @@ -58,17 +58,25 @@ only results in empty files that are not cleaned up}} #PBS -v {{name}}={{value}} {{/env}} +{{> batch_lib}} + PSIJ_NODEFILE="$PBS_NODEFILE" export PSIJ_NODEFILE - {{#job.spec.directory}} cd "{{.}}" {{/job.spec.directory}} +{{> stagein}} +update_status ACTIVE + exec &>> "{{psij.script_dir}}/$PBS_JOBID.out" {{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_PSIJ_JOB_EC=$? + +{{> stageout}} +{{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} echo "$?" > "{{psij.script_dir}}/$PBS_JOBID.ec" diff --git a/src/psij/executors/batch/script_generator.py b/src/psij/executors/batch/script_generator.py index 62049dcb..527bb12b 100644 --- a/src/psij/executors/batch/script_generator.py +++ b/src/psij/executors/batch/script_generator.py @@ -1,6 +1,7 @@ import pathlib from abc import ABC -from typing import Dict, Callable, IO +from enum import Enum +from typing import Dict, Callable, IO, Optional import pystache @@ -8,6 +9,16 @@ from .escape_functions import bash_escape +class _Renderer(pystache.Renderer): # type: ignore + def str_coerce(self, val: object) -> str: + if isinstance(val, Enum): + return str(val.value) + elif isinstance(val, bool): + return str(int(val)) + else: + return super().str_coerce(val) # type: ignore + + class SubmitScriptGenerator(ABC): """A base class representing a submit script generator. @@ -16,7 +27,7 @@ class SubmitScriptGenerator(ABC): script specific to a certain batch scheduler. """ - def __init__(self, config: JobExecutorConfig) -> None: + def __init__(self, config: Optional[JobExecutorConfig]) -> None: """ Parameters ---------- @@ -56,7 +67,7 @@ class TemplatedScriptGenerator(SubmitScriptGenerator): implementation of the Mustache templating language (https://mustache.github.io/). """ - def __init__(self, config: JobExecutorConfig, template_path: pathlib.Path, + def __init__(self, config: Optional[JobExecutorConfig], template_path: pathlib.Path, escape: Callable[[object], str] = bash_escape) -> None: """ Parameters @@ -73,7 +84,7 @@ def __init__(self, config: JobExecutorConfig, template_path: pathlib.Path, with template_path.open('r') as template_file: self.template = pystache.parse(template_file.read()) common_dir = pathlib.Path(__file__).parent / 'common' - self.renderer = pystache.Renderer(escape=escape, search_dirs=[str(common_dir)]) + self.renderer = _Renderer(escape=escape, search_dirs=[str(common_dir)]) def generate_submit_script(self, job: Job, context: Dict[str, object], out: IO[str]) -> None: """See :func:`~SubmitScriptGenerator.generate_submit_script`. diff --git a/src/psij/executors/batch/slurm/slurm.mustache b/src/psij/executors/batch/slurm/slurm.mustache index d6f850fc..9daa060b 100644 --- a/src/psij/executors/batch/slurm/slurm.mustache +++ b/src/psij/executors/batch/slurm/slurm.mustache @@ -88,6 +88,8 @@ _PSIJ_PPN={{.}} {{/processes_per_node}} {{/job.spec.resources}} +{{> batch_lib}} + _PSIJ_NC=`scontrol show hostnames | wc -l` {{!Unlike PBS, Slurm only lists the nodes once in the nodelist, so, to bring it to uniform PBS @@ -106,7 +108,8 @@ else fi export PSIJ_NODEFILE - +{{> stagein}} +update_status ACTIVE {{!redirect output here instead of through #SBATCH directive since SLURM_JOB_ID is not available when the directives are evaluated; the reason for using the job id in the first place being the @@ -114,6 +117,10 @@ same as for the exit code file.}} exec &>> "{{psij.script_dir}}/$SLURM_JOB_ID.out" {{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_PSIJ_JOB_EC=$? + +{{> stageout}} +{{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} -echo "$?" > "{{psij.script_dir}}/$SLURM_JOB_ID.ec" +echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$SLURM_JOB_ID.ec" diff --git a/src/psij/executors/local.py b/src/psij/executors/local.py index bb4fc0bb..ab3d4bc5 100644 --- a/src/psij/executors/local.py +++ b/src/psij/executors/local.py @@ -1,26 +1,28 @@ """This module contains the local :class:`~psij.JobExecutor`.""" import logging import os -import pathlib -import platform import shlex -import shutil import signal import subprocess import threading import time -from abc import abstractmethod +from abc import ABC, abstractmethod +from pathlib import Path from tempfile import mkstemp -from typing import Optional, Dict, List, Tuple, TypeVar, Set, Callable +from types import FrameType +from typing import Optional, Dict, List, Tuple, Type, cast import psutil -from psutil import NoSuchProcess -from psij import InvalidJobException, SubmitException, ResourceSpecV1 +from psij import InvalidJobException, SubmitException, Launcher, ResourceSpecV1 from psij import Job, JobSpec, JobExecutorConfig, JobState, JobStatus from psij import JobExecutor -from psij.exceptions import CompositeException, LauncherException, JobException -from psij.staging import StageIn, StagingMode, StageOut, StageOutFlags +from psij.executors.batch.batch_scheduler_executor import _env_to_mustache +from psij.executors.batch.script_generator import TemplatedScriptGenerator +from psij.utils import SingletonThread, _StatusUpdater + +from psij.executors.batch.template_function_library import ALL as FUNCTION_LIBRARY + logger = logging.getLogger(__name__) @@ -34,343 +36,169 @@ def _format_shell_cmd(args: List[str]) -> str: return cmd -class _JobThread(threading.Thread): - def __init__(self, job: Job, executor: JobExecutor) -> None: - super().__init__(name = 'LocalJobThread-' + job.id) - self.executor = executor - self.cancel_flag = False +def _handle_sigchld(signum: int, frame: Optional[FrameType]) -> None: + _ProcessReaper.get_instance()._handle_sigchld() - @abstractmethod - def cancel(self): - pass - def _get_state_from_ec(self, ec: int) -> Tuple[JobState, Optional[Exception]]: - if ec is None or ec == 0: - return JobState.COMPLETED, None - elif ec < 0: - if self.cancel_flag: - return JobState.CANCELED, None - else: - return JobState.FAILED, JobException(ec) - else: - # ec > 0 - # It is not quite clear what happens in Windows. Windows allows the user - # to specify an exit code when killing a process, exit code which will become - # the exit code of the terminated process. However, psutil does not specify what - # is being done for that on Windows. The psutil sources suggest that signal.SIGTERM - # is used, so we check for that. - if platform.system() == 'Windows' and ec == signal.SIGTERM and self.cancel_flag: - return JobState.CANCELED, None - else: - return JobState.FAILED, JobException(ec) +if threading.current_thread() != threading.main_thread(): + logger.warning('The psij module is being imported from a non-main thread. This prevents the' + 'use of signals in the local executor, which will slow things down a bit.') +else: + signal.signal(signal.SIGCHLD, _handle_sigchld) -# The addition of file staging makes fully asynchronous job management difficult, since we don't -# really have much in the way of something reasonably supporting true async file copying. So since -# we have to use threads anyway, and since the local executor is not really meant to scale, we use -# them for attached processes also. -class _AttachedJobThread(_JobThread): - def __init__(self, job: Job, pid: int, executor: JobExecutor) -> None: - super().__init__(job, executor) - self.job = job - self.pid = pid - self._attach() +_REAPER_SLEEP_TIME = 0.1 - def _attach(self): - with self.job._status_cv: - try: - self.process = psutil.Process(self.pid) - except NoSuchProcess: - # will check in run() and set status - self.process = None - except Exception as ex: - raise SubmitException('Cannot attach to pid %s' % self.pid, exception=ex) - def run(self) -> None: - # We assume that the native_id above is a PID that was obtained at some point using - # list(). If so, the process is either still running or has completed. Either way, we must - # bring it up to ACTIVE state - self.executor._set_job_status(self.job, JobStatus(JobState.QUEUED, time=time.time())) - self.executor._set_job_status(self.job, JobStatus(JobState.ACTIVE, time=time.time())) +class _ProcessEntry(ABC): + def __init__(self, job: Job, executor: 'LocalJobExecutor', launcher: Optional[Launcher]): + self.job = job + self.executor = executor + self.exit_code: Optional[int] = None + self.done_time: Optional[float] = None + self.out: Optional[str] = None + self.kill_flag = False + self.process: Optional[subprocess.Popen[bytes]] = None + self.launcher = launcher + + @abstractmethod + def kill(self) -> None: + assert self.process is not None try: - self._wait_for_job() - except Exception: + root = psutil.Process(self.process.pid) + for proc in root.children(recursive=True): + proc.kill() + self.process.kill() + except psutil.NoSuchProcess: pass - def _wait_for_job(self): - message = None - if self.process is None: - state = JobState.COMPLETED - else: - ec = self.process.wait() - state = self._get_state_from_ec(ec) - - if state == JobState.FAILED: - message = 'Job failed with exit code %s' % ec - - self.executor._set_job_status(self.job, JobStatus(state, message=message, time=time.time())) + @abstractmethod + def poll(self) -> Tuple[Optional[int], Optional[str]]: + pass - def cancel(self): - with self.job._status_cv: - self.cancel_flag = True - if self.process: - self.process.kill() + def __repr__(self) -> str: + pid = '-' + if self.process: + pid = str(self.process.pid) + return '{}[jobid: {}, pid: {}]'.format(self.__class__.__name__, self.job.id, pid) -class _JobCanceled(Exception): - pass +class _ChildProcessEntry(_ProcessEntry): + def __init__(self, job: Job, executor: 'LocalJobExecutor', + launcher: Optional[Launcher]) -> None: + super().__init__(job, executor, launcher) + self.nodefile: Optional[str] = None + def kill(self) -> None: + super().kill() -T = TypeVar('T') + def poll(self) -> Tuple[Optional[int], Optional[str]]: + assert self.process is not None + exit_code = self.process.poll() + if exit_code is not None: + if self.nodefile: + os.unlink(self.nodefile) + if self.process.stdout: + return exit_code, self.process.stdout.read().decode('utf-8') + else: + return exit_code, None + else: + return None, None -class _ChildJobThread(_JobThread): +class _AttachedProcessEntry(_ProcessEntry): + def __init__(self, job: Job, process: psutil.Process, executor: 'LocalJobExecutor'): + super().__init__(job, executor, None) + self.process = process - FLAG_MAP = {JobState.COMPLETED: StageOutFlags.ON_SUCCESS, - JobState.FAILED: StageOutFlags.ON_ERROR, - JobState.CANCELED: StageOutFlags.ON_CANCEL} - - def __init__(self, job: Job, spec: JobSpec, executor: JobExecutor) -> None: - super().__init__(job, executor) - self.job = job - self.spec = spec - if spec.directory is None: - self.jobdir = pathlib.Path('/tmp') - else: - self.jobdir = spec.directory - self.state = None - # set for any error; the overall job is automatically considered failed if set - self.exception = None - self.exit_code = None - self.process = None - - def run(self): - # The following workflow is based on the idea that no error should go unreported. The - # flow is as follows: - # - if there is an error in staging, fail immediately (i.e., do not perform cleanup or - # any other steps). - # - if there is an internal error (i.e., not an executable failure), treat as above and - # fail immediately - # - if a job is canceled during stage in, clean up. If there is an error in cleanup, - # the job will fail instead. - # - if a job is canceled while running, stage out and clean up. If there is an error in - # stage out and/or cleanup, the job will instead fail. - # - if the job fails and there is a subsequent error in staging or cleanup, a compound - # error is created - # - cancellation is ignored during and after stageout + def kill(self) -> None: + super().kill() + def poll(self) -> Tuple[Optional[int], Optional[str]]: try: - try: - self.stage_in() - self.run_job() - self.stage_out() - except _JobCanceled: - # only stage_in and run_job (but before the job is actually started) - # are allowed to raise _JobCanceled - self.state = JobState.CANCELED - self.cleanup() - except Exception as ex: - self.fail_job(ex) + assert self.process + ec: Optional[int] = self.process.wait(timeout=0) + if ec is None: + return 0, None + else: + return ec, None + except psutil.TimeoutExpired: + return None, None - self.update_job_status() - def update_job_status(self): - if self.exception: - self.executor._set_job_status(self.job, - JobStatus(JobState.FAILED, time=time.time(), - message=str(self.exception), - metadata={'exception': self.exception}, - exit_code=self.exit_code)) - else: - # failed without an exception set is not allowed - assert self.state != JobState.FAILED - self.executor._set_job_status(self.job, JobStatus(self.state, time=time.time())) - - def fail_job(self, ex: Exception) -> None: - if self.state == JobState.FAILED: - if self.exception is None: - self.exception = ex - else: - if not isinstance(self.exception, CompositeException): - self.exception = CompositeException(self.exception) - self.exception.add_exception(ex) - else: - self.state = JobState.FAILED - self.exception = ex - - def stage_in(self) -> None: - self.executor._set_job_status(self.job, JobStatus(JobState.STAGE_IN, time=time.time())) - self._map(self._stage_in_one, self.spec.stage_in) - - def stage_out(self): - self.executor._set_job_status(self.job, JobStatus(JobState.STAGE_OUT, time=time.time())) - self._map(self._stage_out_one, self.spec.stage_out) - - def cleanup(self): - self.executor._set_job_status(self.job, JobStatus(JobState.CLEANUP, time=time.time())) - self._map(self._cleanup_one, self.spec.cleanup) - - @staticmethod - def _map(fn: Callable[[T], None], s: Optional[Set[T]], ) -> None: - if s is None: - return - for o in s: - fn(o) - - def _stage_in_one(self, stage_in: StageIn) -> None: - if self.cancel_flag: - raise _JobCanceled() - src = stage_in.source - scheme = src.scheme - if scheme == '': - scheme = 'file' - if scheme == 'file': - self._local_copy(pathlib.Path(src.path), self._job_rel(stage_in.target), - stage_in.mode, False) - else: - self.fail_job(ValueError('Unsupported scheme "%s" for %s' % (scheme, src))) - - def _stage_out_one(self, stage_out: StageOut) -> None: - dst = stage_out.target - scheme = dst.scheme - if scheme == '': - scheme = 'file' - if scheme == 'file': - flags = stage_out.flags - state = _ChildJobThread.FLAG_MAP[self.state] - if state in flags: - self._local_copy(self._job_rel(stage_out.source), pathlib.Path(dst.path), - stage_out.mode, StageOutFlags.IF_PRESENT in stage_out.flags) - else: - self.fail_job(ValueError('Unsupported scheme "%s" for %s' % (scheme, dst))) - - def _cleanup_one(self, cleanup: pathlib.Path) -> None: - # do some sanity checks - cleanup = self._job_rel(cleanup) - if cleanup.samefile(pathlib.Path('/')): - raise ValueError('Refusing to clean root directory.') - if cleanup.samefile(pathlib.Path.home()): - raise ValueError('Refusing to clean user home directory.') - if cleanup.is_dir(): - shutil.rmtree(str(cleanup)) - else: - cleanup.unlink(missing_ok=True) - - def _job_rel(self, path: pathlib.Path) -> pathlib.Path: - path = path.expanduser() - if not path.is_absolute(): - path = self.jobdir / path - return path.absolute() - - def _local_copy(self, source: pathlib.Path, target: pathlib.Path, mode: StagingMode, - if_present=False): - if if_present and not os.path.exists(source): - return - if mode == StagingMode.COPY: - if source.is_dir(): - shutil.copytree(source, target) - else: - shutil.copy(source, target) - elif mode == StagingMode.MOVE: - shutil.move(source, target) - elif mode == StagingMode.LINK: - os.symlink(source, target) - - def run_job(self): - launcher = self.executor._get_launcher(self._get_launcher_name(self.spec)) - args = launcher.get_launch_command(self.job) - - if logger.isEnabledFor(logging.DEBUG): - logger.debug('Running %s', _format_shell_cmd(args)) - nodefile = self._generate_nodefile(self.job) - try: - env = _get_env(self.spec, nodefile) - with self.job._status_cv: - if self.cancel_flag: - raise _JobCanceled() - self.process = subprocess.Popen(args, stdout=subprocess.PIPE, - stderr=subprocess.STDOUT, close_fds=True, - cwd=self.spec.directory, env=env) - self.job._native_id = self.process.pid - self.executor._set_job_status(self.job, - JobStatus(JobState.ACTIVE, time=time.time(), - metadata={'nativeId': self.job._native_id})) - self.exit_code = self.process.wait() +class _ProcessReaper(SingletonThread): - # We want to capture errors in the launcher scripts. Since, under normal circumstances, - # the exit code of the launcher is the exit code of the job, we must use a different - # mechanism to distinguish between job errors and launcher errors. So we delegate to - # the launcher implementation to figure out if the error belongs to the job or not - if self.process.stdout: - out = self.process.stdout.read().decode('utf-8') - else: - out = None - if out and launcher.is_launcher_failure(out): - message = self.process.launcher.get_launcher_failure_message(out) - self.fail_job(LauncherException(message)) - else: - self.state, self.exception = self._get_state_from_ec(self.exit_code) - finally: - if nodefile: - os.remove(nodefile) - - def cancel(self): - with self.job._status_cv: - self.cancel_flag = True - if self.process is not None: - self.process.kill() - - def _generate_nodefile(self, job: Job) -> Optional[str]: - assert job.spec is not None - if job.spec.resources is None: - return None - if job.spec.resources.version == 1: - assert isinstance(job.spec.resources, ResourceSpecV1) - n = job.spec.resources.computed_process_count - if n == 1: - # as a bit of an optimization, we don't generate a nodefile when doing "single - # node" jobs on local. - return None - (file, nodefile) = mkstemp(suffix='.nodelist') - for i in range(n): - os.write(file, 'localhost\n'.encode()) - os.close(file) - return nodefile - else: - raise SubmitException('Cannot handle resource specification with version %s' - % job.spec.resources.version) + @classmethod + def get_instance(cls: Type['_ProcessReaper']) -> '_ProcessReaper': + return cast('_ProcessReaper', super().get_instance()) - def _get_launcher_name(self, spec: JobSpec) -> str: - if spec.launcher is None: - return 'single' - else: - return spec.launcher + def __init__(self) -> None: + super().__init__(name='Local Executor Process Reaper', daemon=True) + self._jobs: Dict[Job, _ProcessEntry] = {} + self._lock = threading.RLock() + self._cvar = threading.Condition() + def register(self, entry: _ProcessEntry) -> None: + logger.debug('Registering process %s', entry) + with self._lock: + self._jobs[entry.job] = entry -def _get_env(spec: JobSpec, nodefile: Optional[str]) -> Optional[Dict[str, str]]: - env: Optional[Dict[str, str]] = None - if spec.inherit_environment: - if spec.environment is None and nodefile is None: - # if env is none in Popen, it inherits env from parent - return None - else: - # merge current env with spec env - env = os.environ.copy() - if spec.environment: - env.update(spec.environment) - if nodefile is not None: - env['PSIJ_NODEFILE'] = nodefile - return env - else: - # only spec env - if nodefile is None: - env = spec.environment - else: - env = {'PSIJ_NODEFILE': nodefile} - if spec.environment: - env.update(spec.environment) + def run(self) -> None: + logger.debug('Started {}'.format(self)) + done: List[_ProcessEntry] = [] + while True: + with self._lock: + for entry in done: + del self._jobs[entry.job] + jobs = dict(self._jobs) + try: + done = self._check_processes(jobs) + except Exception: + logger.exception('Error polling for process status.') + with self._cvar: + self._cvar.wait(_REAPER_SLEEP_TIME) + + def _handle_sigchld(self) -> None: + with self._cvar: + try: + self._cvar.notify_all() + except RuntimeError: + # In what looks like rare cases, notify_all(), seemingly when combined with + # signal handling, raises `RuntimeError: release unlocked lock`. + # There appears to be an unresolved Python bug about this: + # https://bugs.python.org/issue34486 + # We catch the exception here and log it. It is hard to tell if that will not lead + # to further issues. It would seem like it shouldn't: after all, all we're doing is + # making sure we don't sleep too much, but, even if we do, the consequence is a + # small delay in processing a completed job. However, since this exception seems + # to be a logical impossibility when looking at the code in threading.Condition, + # there is really no telling what else could go wrong. + logger.debug('Exception in Condition.notify_all()') + + def _check_processes(self, jobs: Dict[Job, _ProcessEntry]) -> List[_ProcessEntry]: + done: List[_ProcessEntry] = [] + + for entry in jobs.values(): + if entry.kill_flag: + entry.kill() + + exit_code, out = entry.poll() + if exit_code is not None: + entry.exit_code = exit_code + entry.done_time = time.time() + entry.out = out + logger.debug('Output from job: %s' % out) + done.append(entry) + + for entry in done: + entry.executor._process_done(entry) + + return done - return env + def cancel(self, job: Job) -> None: + with self._lock: + p = self._jobs[job] + p.kill_flag = True class LocalJobExecutor(JobExecutor): @@ -401,8 +229,32 @@ def __init__(self, url: Optional[str] = None, :type config: psij.JobExecutorConfig """ super().__init__(url=url, config=config if config else JobExecutorConfig()) - self._threads_lock = threading.RLock() - self.threads: Dict[str, _JobThread] = {} + self._reaper = _ProcessReaper.get_instance() + self._work_dir = Path.home() / '.psij' / 'work' / 'local' + self._work_dir.mkdir(parents=True, exist_ok=True) + self._status_updater = cast(_StatusUpdater, _StatusUpdater.get_instance()) + self.generator = TemplatedScriptGenerator(config, Path(__file__).parent / 'local' + / 'local.mustache') + + def _generate_nodefile(self, job: Job, p: _ChildProcessEntry) -> Optional[str]: + assert job.spec is not None + if job.spec.resources is None: + return None + if job.spec.resources.version == 1: + assert isinstance(job.spec.resources, ResourceSpecV1) + n = job.spec.resources.computed_process_count + if n == 1: + # as a bit of an optimization, we don't generate a nodefile when doing "single + # node" jobs on local. + return None + (file, p.nodefile) = mkstemp(suffix='.nodelist') + for i in range(n): + os.write(file, 'localhost\n'.encode()) + os.close(file) + return p.nodefile + else: + raise SubmitException('Cannot handle resource specification with version %s' + % job.spec.resources.version) def submit(self, job: Job) -> None: """ @@ -418,16 +270,47 @@ def submit(self, job: Job) -> None: """ spec = self._check_job(job) - self._set_job_status(job, JobStatus(JobState.QUEUED, time=time.time())) - - with job._status_cv: - if job.status.state == JobState.CANCELED: - raise SubmitException('Job canceled') - job_thread = _ChildJobThread(job, spec, self) - - with self._threads_lock: - self.threads[job.id] = job_thread - job_thread.start() + p = _ChildProcessEntry(job, self, self._get_launcher(self._get_launcher_name(spec))) + assert p.launcher + launch_command = p.launcher.get_launch_command(job) + + nodefile = self._generate_nodefile(job, p) + ctx = { + 'job': job, + 'env': _env_to_mustache(job), + 'psij': { + 'lib': FUNCTION_LIBRARY, + 'launch_command': launch_command, + 'script_dir': str(self._work_dir), + 'us_file': self._status_updater.update_file_name, + 'us_port': self._status_updater.update_port, + 'us_addrs': ', '.join(self._status_updater.ips), + 'debug': logger.isEnabledFor(logging.DEBUG), + 'nodefile': nodefile + } + } + + submit_file_path = self._work_dir / (job.id + '.job') + with submit_file_path.open('w') as submit_file: + self.generator.generate_submit_script(job, ctx, submit_file) + + args = ['/bin/bash', str(submit_file_path.absolute())] + self._status_updater.register_job(job, self) + try: + with job._status_cv: + if job.status.state == JobState.CANCELED: + raise SubmitException('Job canceled') + if logger.isEnabledFor(logging.DEBUG): + logger.debug('Running %s', _format_shell_cmd(args)) + p.process = subprocess.Popen(args, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, + close_fds=True, cwd=spec.directory) + self._reaper.register(p) + job._native_id = p.process.pid + self._set_job_status(job, JobStatus(JobState.QUEUED, time=time.time(), + metadata={'nativeId': job._native_id})) + except Exception as ex: + self._status_updater.unregister_job(job) + raise SubmitException('Failed to submit job', exception=ex) def cancel(self, job: Job) -> None: """ @@ -435,17 +318,33 @@ def cancel(self, job: Job) -> None: :param job: The job to cancel. """ + self._status_updater.unregister_job(job) + self._set_job_status(job, JobStatus(JobState.CANCELED)) + self._reaper.cancel(job) - with self._threads_lock: - try: - job_thread = self.threads[job.id] - except KeyError: - raise ValueError('The job %s is not managed by this executor.' % job.id) - with job._status_cv: - if job_thread is not None: - job_thread.cancel() - else: - self._set_job_status(job, JobStatus(JobState.CANCELED)) + def _process_done(self, p: _ProcessEntry) -> None: + assert p.exit_code is not None + logger.debug('%s Process done. EC: %s', p.job.id, p.exit_code) + message = None + if p.exit_code == 0: + state = JobState.COMPLETED + elif p.exit_code < 0 and p.kill_flag: + state = JobState.CANCELED + else: + # We want to capture errors in the launcher scripts. Since, under normal circumstances, + # the exit code of the launcher is the exit code of the job, we must use a different + # mechanism to distinguish between job errors and launcher errors. So we delegate to + # the launcher implementation to figure out if the error belongs to the job or not + if p.launcher and p.out and p.launcher.is_launcher_failure(p.out): + message = p.launcher.get_launcher_failure_message(p.out) + state = JobState.FAILED + + # We need to ensure that the status updater has processed all updates that + # have been sent up to this point + self._status_updater.flush() + self._status_updater.unregister_job(p.job) + self._set_job_status(p.job, JobStatus(state, time=p.done_time, exit_code=p.exit_code, + message=message)) def list(self) -> List[str]: """ @@ -480,11 +379,17 @@ def attach(self, job: Job, native_id: str) -> None: job.executor = self pid = int(native_id) - with job._status_cv: - if job.status.state == JobState.CANCELED: - raise SubmitException('Job canceled') - job_thread = _AttachedJobThread(job, pid, self) + job._native_id = pid + self._status_updater.register_job(job, self) + self._reaper.register(_AttachedProcessEntry(job, psutil.Process(pid), self)) + # We assume that the native_id above is a PID that was obtained at some point using + # list(). If so, the process is either still running or has completed. Either way, we must + # bring it up to ACTIVE state + self._set_job_status(job, JobStatus(JobState.QUEUED, time=time.time())) + self._set_job_status(job, JobStatus(JobState.ACTIVE, time=time.time())) - with self._threads_lock: - self.threads[job.id] = job_thread - job_thread.start() + def _get_launcher_name(self, spec: JobSpec) -> str: + if spec.launcher is None: + return 'single' + else: + return spec.launcher diff --git a/src/psij/executors/local/local.mustache b/src/psij/executors/local/local.mustache new file mode 100644 index 00000000..84941895 --- /dev/null +++ b/src/psij/executors/local/local.mustache @@ -0,0 +1,25 @@ +#!/bin/bash + +set -e + +{{> batch_lib}} + +{{#job.spec.directory}} +cd "{{.}}" +{{/job.spec.directory}} + +{{> stagein}} +update_status ACTIVE + +set +e +{{#job.spec.inherit_environment}}env \{{/job.spec.inherit_environment}}{{^job.spec.inherit_environment}}env --ignore-environment \{{/job.spec.inherit_environment}}{{#env}} +{{name}}="{{value}}" \{{/env}} +PSIJ_NODEFILE={{psij.nodefile}} \ +{{#psij.launch_command}}{{.}} {{/psij.launch_command}} +_PSIJ_JOB_EC="$?" +set -e + +{{> stageout}} +{{> cleanup}} + +exit "$_PSIJ_JOB_EC" \ No newline at end of file diff --git a/src/psij/job_spec.py b/src/psij/job_spec.py index 6d746c04..67ea4980 100644 --- a/src/psij/job_spec.py +++ b/src/psij/job_spec.py @@ -13,10 +13,15 @@ def _to_path(arg: Union[str, pathlib.Path, None]) -> Optional[pathlib.Path]: + if arg is None: + return None + else: + return _to_path_strict(arg) + + +def _to_path_strict(arg: Union[str, pathlib.Path]) -> pathlib.Path: if isinstance(arg, pathlib.Path): return arg - elif arg is None: - return None else: assert isinstance(arg, str) return pathlib.Path(arg) @@ -33,8 +38,11 @@ def _to_env_dict(arg: Union[Dict[str, Union[str, int]], None]) -> Optional[Dict[ ret[k] = v return ret -def _all_to_path(s: Set[Union[str, pathlib.Path]]) -> Set[pathlib.Path]: - return set(map(_to_path, s)) + +def _all_to_path(s: Optional[Set[Union[str, pathlib.Path]]]) -> Optional[Set[pathlib.Path]]: + if s is None: + return None + return {_to_path_strict(x) for x in s if x is not None} class JobSpec(object): @@ -163,6 +171,7 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st self.stage_in = stage_in self.stage_out = stage_out self._cleanup = _all_to_path(cleanup) + self.cleanup_on_failure = cleanup_on_failure # TODO: `resources` is of type `ResourceSpec`, not `ResourceSpecV1`. An # connector trying to access `job.spec.resources.process_count` @@ -255,7 +264,8 @@ def post_launch(self, post_launch: Union[str, pathlib.Path, None]) -> None: self._post_launch = _to_path(post_launch) @property - def cleanup(self) -> Set[pathlib.Path]: + def cleanup(self) -> Optional[Set[pathlib.Path]]: + """An optional set of cleanup directives.""" return self._cleanup @cleanup.setter diff --git a/src/psij/job_state.py b/src/psij/job_state.py index 01c85ef0..701035e8 100644 --- a/src/psij/job_state.py +++ b/src/psij/job_state.py @@ -4,6 +4,7 @@ _NAME_MAP = {} + class JobState(bytes, Enum): """ An enumeration holding the possible job states. @@ -130,6 +131,11 @@ def __hash__(self) -> int: @staticmethod def from_name(name: str) -> 'JobState': + """ + Returns a `JobState` object corresponding to its string representation. + + This method is such that `state == JobState.from_name(str(state))`. + """ return _NAME_MAP[name] @@ -145,6 +151,7 @@ def from_name(name: str) -> 'JobState': JobState.CANCELED: None } + class JobStateOrder: """A class that can be used to reconstruct missing states.""" diff --git a/src/psij/launchers/script_based_launcher.py b/src/psij/launchers/script_based_launcher.py index 9d11f932..4a263106 100644 --- a/src/psij/launchers/script_based_launcher.py +++ b/src/psij/launchers/script_based_launcher.py @@ -200,8 +200,12 @@ def get_additional_args(self, job: Job) -> List[str]: def is_launcher_failure(self, output: str) -> bool: """See :func:`~psij.Launcher.is_launcher_failure`.""" - return output.split('\n')[-2] != '_PSI_J_LAUNCHER_DONE' + lines = output.split('\n') + return lines[-2] != '_PSI_J_LAUNCHER_DONE' or lines[-1] != '' def get_launcher_failure_message(self, output: str) -> str: """See :func:`~psij.Launcher.get_launcher_failure_message`.""" - return '\n'.join(output.split('\n')[:-2]) + # Errors can occur in the job script after the launcher is done (e.g., stageout), + # so we need to filter the launcher tag + lines = output.split('\n') + return '\n'.join(filter(lambda x: x != '_PSI_J_LAUNCHER_DONE', lines)) diff --git a/src/psij/staging.py b/src/psij/staging.py index cc0329fc..af2fec29 100644 --- a/src/psij/staging.py +++ b/src/psij/staging.py @@ -1,94 +1,287 @@ -import urllib +from urllib.parse import urlparse from enum import Enum, Flag from pathlib import Path from typing import Optional, Union class URI: + """A class representing a local or remote file.""" + def __init__(self, urlstring: str) -> None: - self.parts = urllib.parse.urlparse(urlstring) + """ + Parameters + ---------- + urlstring + A string representation of a URI, such as "http://example.com/file.txt" or "file.txt". + The precise format of an URI string is defined in + `RFC3986 `_. + """ + self.parts = urlparse(urlstring) # a __getattr__ solution may be simpler, but doesn't play well with IDEs and # is not quite self-documenting @property def hostname(self) -> Optional[str]: - return self.hostname + """ + Returns + ------- + Represents the hostname in this URI or `None` if no hostname was specified. + """ + return self.parts.hostname @property - def port(self) -> int: - return self.port + def port(self) -> Optional[int]: + """ + Returns + ------- + Returns the TCP port of this URI or None if a port was not specified. + """ + return self.parts.port @property def scheme(self) -> str: + """ + Returns + ------- + Returns the URI scheme in this URI or the empty string if no scheme was specified. + """ return self.parts.scheme @property def netloc(self) -> str: + """ + Returns + ------- + Returns the network location, which may the host name, the port, and possibly login + information. If none of these are specified, the empty string is returned. + """ return self.parts.netloc @property def path(self) -> str: + """ + Returns + ------- + Returns the path in this URI or an empty string if no path was specified. + """ return self.parts.path @property def params(self) -> str: + """ + Returns + ------- + Returns the URI parameters or an empty string if there are no parameters. + """ return self.parts.params @property def query(self) -> str: + """ + Returns + ------- + Returns the URI query string or an empty string if no query string was specified. + """ return self.parts.query @property def fragment(self) -> str: + """ + Returns + ------- + Returns the fragment in this URI or the empty string if no fragment is specified. + """ return self.parts.fragment @property - def username(self) -> str: + def username(self) -> Optional[str]: + """ + Returns + ------- + Returns the username in this URI if any, or None if there is no username specified. + """ return self.parts.username @property - def password(self) -> str: + def password(self) -> Optional[str]: + """ + Returns + ------- + Returns the password specified in this URI or None if there is no password. + """ return self.parts.password def __str__(self) -> str: + """Returns a string representation of this URL.""" return self.parts.geturl() class StagingMode(Enum): + """ + Defines the possible modes in which the staging of a file can be done. + + JobExecutor implementations are not required to support all staging modes, but must default + to `COPY` if other modes are not implemented. Furthermore, modes different from `COPY` may only + make sense when staging is done locally. + """ + COPY = 1 + """ + Copies the file to be staged by performing an operation that is equivalent to the familiar + `cp` command. + """ LINK = 2 + """ + Creates a symbolic link instead of copying the contents of files. + """ MOVE = 3 + """ + Moves a file instead of copying it. Moving a file can be nearly instantaneous if both the + source and the destination are on the same filesystem. However, the OS will likely have to + resort to copying the contents of the file and the removing the source file if the source and + destination are on different filesystems, so it is unlikely for this mode to be beneficial over + a `COPY`. + """ class StageOutFlags(Flag): + """ + Specifies a set of flags that can be used to alter stage out behavior. + + The flags can be combined using the bitwise or operator (`|`). For example, + `IF_PRESENT | ON_ERROR`. If none of the state conditions + (`ON_SUCCESS`, `ON_ERROR`, `ON_CANCEL`) are specified, it is assumed that the file should be + transferred in all cases, subject to the presence of the `IF_PRESENT` flag. That is, + `NONE` is equivalent to `ALWAYS` or `ON_SUCCESS | ON_ERROR | ON_CANCEL`, while + `IF_PRESENT` is equivalent to `IF_PRESENT | ALWAYS`. + """ + + NONE = 0 + """ + Indicates that no flags are set. This is equivalent to `ALWAYS`. + """ IF_PRESENT = 1 + """ + Indicates that a file should only be transferred if it exists. If the file does not exist, + the stageout operation continues with the next file. If this flag is not set for a given file, + its absence will result in a stageout error which will cause the job to fail. + """ ON_SUCCESS = 2 + """ + Indicates that a file should be transferred when the job succeeds (i.e., its exit code is zero). + If a job fails or is cancelled, and no other flags are set, the executor will not attempt to + stage out the file. + """ ON_ERROR = 4 + """ + Indicates that a stageout should be performed if the job has failed (i.e., its exit code is + non-zero). + """ ON_CANCEL = 8 + """ + Indicates that a file should be staged out if the job has been canceled. + """ ALWAYS = ON_SUCCESS | ON_ERROR | ON_CANCEL + """ + Indicates that a file should be staged out irrespective of the status of the job. + """ class StageIn: - def __init__(self, source: Union[URI, str], target: Union[str, Path], + """A class representing a stagein directive.""" + + def __init__(self, source: Union[URI, Path, str], target: Union[str, Path], mode: StagingMode = StagingMode.COPY) -> None: + """ + Parameters + ---------- + source + The source location of the stagein. If the source is a string or a : + class:`~pathlib.Path`, the location refers to a file on a filesystem accessible by the + process in which PSI/J is running. If the path is relative, it is interpreted to be + relative to the current working directory of the process in which PSI/J is running and + normalized to an absolute path. If the source is a :class:`.URI`, it may refer to a + remote location. Support for remote staging is not guaranteed and depends on the + implementation of the :class:`~psij.JobExecutor` that the job to which this stagein + directive belongs is submitted to. + target + The target location for the stagein, which can be either a string or a + :class:`~pathlib.Path`. If the path is relative, it is considered to be relative to the + job directory. That is, a job can access this file at the location specified by + `target` if it does not change its working directory from the one it starts in. + mode + A staging mode, which indicates how the staging is done. For details, see + :class:`.StagingMode`. + """ if isinstance(source, str): source = URI(source) + if isinstance(source, Path): + source = URI(str(source)) if isinstance(target, str): target = Path(target) self.source = source self.target = target self.mode = mode + +def _normalize_flags(flags: StageOutFlags) -> StageOutFlags: + if (flags & StageOutFlags.ALWAYS).value == 0: + return flags | StageOutFlags.ALWAYS + else: + return flags + + class StageOut: - def __init__(self, source: Union[str, Path], target: Union[str, URI], + """A class encapsulating a stageout directive.""" + + def __init__(self, source: Union[str, Path], target: Union[str, Path, URI], flags: StageOutFlags = StageOutFlags.ALWAYS, mode: StagingMode = StagingMode.COPY): + """ + Parameters + ---------- + source + The source location for the stagein, which can be either a string or a + :class:`~pathlib.Path`. If the path is relative, it is considered to be relative to the + job directory. + target + The target location of the stageout. If the target is a string or a + :class:`~pathlib.Path`, the location refers to a file on a filesystem accessible by the + process in which PSI/J is running. If the path is relative, it is interpreted to be + relative to the current working directory of the process in which PSI/J is running and + normalized to an absolute path. If the target is a :class:`.URI`, it may refer to a + remote location. Support for remote staging is not guaranteed and depends on the + implementation of the :class:`~psij.JobExecutor` that the job to which this stageout + directive belongs is submitted to. + flags + A set of flags specifying the conditions under which the stageout should occur. For + details, see :class:`.StageOutFlags`. + mode + A staging mode, which indicates how the staging is done. For details, see + :class:`.StagingMode`. + """ if isinstance(source, str): source = Path(source) if isinstance(target, str): target = URI(target) + if isinstance(target, Path): + target = URI(str(target)) + print(target.parts) self.source = source self.target = target self.flags = flags self.mode = mode + + @property + def flags(self) -> StageOutFlags: + """ + A set of flags specifying the conditions under which the stageout should occur. + + For details, see :class:`.StageOutFlags`. + """ + return self._flags + + @flags.setter + def flags(self, flags: StageOutFlags) -> None: + self._flags = _normalize_flags(flags) diff --git a/src/psij/utils.py b/src/psij/utils.py index 0db4cde7..8f763a9e 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -1,6 +1,20 @@ +import atexit +import io +import logging import os +import random +import socket +import tempfile import threading -from typing import Type, Dict, Optional +import time +from pathlib import Path +from typing import Type, Dict, Optional, Tuple, Set, List + +import psutil + +from psij import JobExecutor, Job, JobState, JobStatus + +logger = logging.getLogger(__name__) class SingletonThread(threading.Thread): @@ -16,7 +30,7 @@ class SingletonThread(threading.Thread): the `run` method. """ - _instances: Dict[int, 'SingletonThread'] = {} + _instances: Dict[int, Dict[type, 'SingletonThread']] = {} _lock = threading.RLock() def __init__(self, name: Optional[str] = None, daemon: bool = False) -> None: @@ -42,8 +56,142 @@ def get_instance(cls: Type['SingletonThread']) -> 'SingletonThread': """ with cls._lock: my_pid = os.getpid() - if my_pid not in cls._instances: + if my_pid in cls._instances: + classes = cls._instances[my_pid] + else: + classes = {} + cls._instances[my_pid] = classes + if cls in classes: + return classes[cls] + else: instance = cls() - cls._instances[my_pid] = instance + classes[cls] = instance instance.start() - return cls._instances[my_pid] + return instance + + +class _StatusUpdater(SingletonThread): + # we are expecting short messages in the form + RECV_BUFSZ = 2048 + + def __init__(self) -> None: + super().__init__() + self.name = 'Status Update Thread' + self.daemon = True + self.work_directory = Path.home() / '.psij' + self.socket = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) + self.socket.setblocking(True) + self.socket.settimeout(0.5) + self.socket.bind(('', 0)) + self.update_port = self.socket.getsockname()[1] + self.ips = self._get_ips() + logger.debug('Local IPs: %s' % self.ips) + logger.debug('Status updater port: %s' % self.update_port) + self._create_update_file() + logger.debug('Update file: %s' % self.update_file.name) + self.partial_file_data = '' + self.partial_net_data = '' + self._jobs: Dict[str, Tuple[Job, JobExecutor]] = {} + self._jobs_lock = threading.RLock() + self._sync_ids: Set[str] = set() + self._last_received = '' + + def _get_ips(self) -> List[str]: + addrs = psutil.net_if_addrs() + r = [] + for name, l in addrs.items(): + if name == 'lo': + continue + for a in l: + if a.family == socket.AddressFamily.AF_INET: + r.append(a.address) + return r + + def _create_update_file(self) -> None: + f = tempfile.NamedTemporaryFile(dir=self.work_directory, prefix='supd_', delete=False) + name = f.name + self.update_file_name = name + atexit.register(os.remove, name) + f.close() + self.update_file = open(name, 'r+b') + self.update_file.seek(0, io.SEEK_END) + self.update_file_pos = self.update_file.tell() + + def register_job(self, job: Job, ex: JobExecutor) -> None: + with self._jobs_lock: + self._jobs[job.id] = (job, ex) + + def unregister_job(self, job: Job) -> None: + with self._jobs_lock: + try: + del self._jobs[job.id] + except KeyError: + # There are cases when it's difficult to esnure that this method is only called + # once for each job. Instead, ignore errors here, since the ultimate goal is to + # remove the job from the _jobs dictionary. + pass + + def step(self) -> None: + self.update_file.seek(0, io.SEEK_END) + pos = self.update_file.tell() + if pos > self.update_file_pos: + self.update_file.seek(self.update_file_pos, io.SEEK_SET) + n = pos - self.update_file_pos + self._process_update_data(self.update_file.read(n)) + self.update_file_pos = pos + else: + try: + data = self.socket.recv(_StatusUpdater.RECV_BUFSZ) + self._process_update_data(data) + except TimeoutError: + pass + except BlockingIOError: + pass + + def run(self) -> None: + while True: + self.step() + + def flush(self) -> None: + # Ensures that, upon return from this call, all updates available before this call have + # been processed. To do so, we send a UDP packet to the socket to wake it up and wait until + # it is received. This does not guarantee that file-based updates are necessarily + # processes, since that depends on many factors. + token = '_SYNC ' + str(random.getrandbits(128)) + sock = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) + sock.sendto(bytes(token, 'utf-8'), ('127.0.0.1', self.update_port)) + delay = 0.0001 + while token not in self._sync_ids: + time.sleep(delay) + delay *= 2 + + def _process_update_data(self, data: bytes) -> None: + sdata = data.decode('utf-8') + if sdata == self._last_received: + # we send UDP packets to all IP addresses of the submit host, which may + # result in duplicates, so we drop consecutive messages that are identical + return + else: + self._last_received = sdata + lines = sdata.splitlines() + for line in lines: + if sdata.startswith('_SYNC '): + self._sync_ids.add(sdata) + continue + els = line.split() + if len(els) > 2 and els[1] == 'LOG': + logger.info('%s %s' % (els[0], ' '.join(els[2:]))) + continue + if len(els) != 2: + logger.warning('Invalid status update message received: %s' % line) + continue + job_id = els[0] + state = JobState.from_name(els[1]) + job = None + with self._jobs_lock: + try: + (job, executor) = self._jobs[job_id] + except KeyError: + logger.debug('Received status updated for inexistent job with id %s' % job_id) + if job: + executor._set_job_status(job, JobStatus(state)) diff --git a/tests/_test_tools.py b/tests/_test_tools.py index b8c00f0a..cedf1391 100644 --- a/tests/_test_tools.py +++ b/tests/_test_tools.py @@ -4,7 +4,7 @@ from contextlib import contextmanager from datetime import timedelta from pathlib import Path -from typing import Optional, Union, Iterator +from typing import Optional, Union, Iterator, IO from executor_test_params import ExecutorTestParams @@ -37,8 +37,14 @@ def assert_completed(job: Job, status: Optional[JobStatus], attached: bool = Fal if status.state != JobState.COMPLETED: if not attached: assert job.spec is not None - stdout = _read_file(job.spec.stdout_path) - stderr = _read_file(job.spec.stderr_path) + try: + stdout = _read_file(job.spec.stdout_path) + except Exception: + stdout = '' + try: + stderr = _read_file(job.spec.stderr_path) + except Exception: + stderr = '' raise AssertionError('Job not completed. Exit code: %s, Status message: %s, ' 'stdout: %s, stderr: %s' % (status.exit_code, status.message, stdout, stderr)) @@ -72,3 +78,59 @@ def _deploy(path: Union[Path, str]) -> Iterator[Path]: yield Path(df.name) finally: os.unlink(df.name) + + +@contextmanager +def _tempfile() -> Iterator[IO[str]]: + # we have type: ignore above because mypy complains that _TemporaryFileWrapper is generic, + # but adding [str] to it results in a runtime error stating that _TemporaryFileWrapper is + # not subscriptable + _make_test_dir() + test_dir = Path.home() / '.psij' / 'test' + with tempfile.NamedTemporaryFile(mode='w', dir=test_dir, delete=False) as f: + try: + yield f + finally: + try: + os.unlink(f.name) + except FileNotFoundError: + # some tests may remove the file themselves + pass + + +@contextmanager +def _temppath() -> Iterator[Path]: + _make_test_dir() + test_dir = Path.home() / '.psij' / 'test' + with tempfile.NamedTemporaryFile(mode='w', dir=test_dir, delete=False) as f: + try: + f.close() + yield Path(f.name) + finally: + try: + os.unlink(f.name) + except FileNotFoundError: + # some tests may remove the file themselves + pass + + +@contextmanager +def _tempdir(keep: bool = False) -> Iterator[Path]: + _make_test_dir() + d = tempfile.mkdtemp(dir=Path.cwd()) + try: + yield Path(d) + shutil.rmtree(d) + except Exception: + if not keep: + shutil.rmtree(d) + raise + + +def _write_file(f: Union[Path, IO[str]], contents: str) -> None: + if isinstance(f, Path): + f = f.open('w') + try: + f.write(contents) + finally: + f.close() diff --git a/tests/plugins1/_batch_test/test/test.mustache b/tests/plugins1/_batch_test/test/test.mustache index 134b0443..8fec2597 100644 --- a/tests/plugins1/_batch_test/test/test.mustache +++ b/tests/plugins1/_batch_test/test/test.mustache @@ -44,26 +44,16 @@ done export PSIJ_NODEFILE -update_status STAGEIN -{{#stagein_set}} - do_stagein "{{source}}" "{{target}}" {{mode}} -{{/stagein_set}} - +{{> stagein}} update_status ACTIVE + {{#job.spec.inherit_environment}}env \{{/job.spec.inherit_environment}}{{^job.spec.inherit_environment}}env --ignore-environment \{{/job.spec.inherit_environment}}{{#env}} {{name}}="{{value}}" \{{/env}} {{#psij.launch_command}}{{.}} {{/psij.launch_command}} -_EC="$?" +_PSIJ_JOB_EC="$?" -update_status STAGEOUT -{{#stageout_set}} - do_stageout "{{source}}" "{{target}}" {{mode}} {{flags}} $_EC -{{/stageout_set}} +{{> stageout}} +{{> cleanup}} -update_status CLEANUP -{{#cleanup_set}} - do_cleanup {{.}} -{{/cleanup_set}} -echo "$_EC" > "{{psij.script_dir}}/$PSIJ_BATCH_TEST_JOB_ID.ec" ->>>>>>> 554b43e (Initial staging commit) +echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$PSIJ_BATCH_TEST_JOB_ID.ec" diff --git a/tests/test_callbacks.py b/tests/test_callbacks.py index c0b53813..af51d941 100644 --- a/tests/test_callbacks.py +++ b/tests/test_callbacks.py @@ -29,9 +29,12 @@ def test_job_callbacks(self) -> None: jex.submit(job) job.wait() - self.assertEqual(len(self._cb_states), 3) + self.assertEqual(len(self._cb_states), 6) self.assertIn(psij.JobState.QUEUED, self._cb_states) + self.assertIn(psij.JobState.STAGE_IN, self._cb_states) self.assertIn(psij.JobState.ACTIVE, self._cb_states) + self.assertIn(psij.JobState.STAGE_OUT, self._cb_states) + self.assertIn(psij.JobState.CLEANUP, self._cb_states) self.assertIn(psij.JobState.FAILED, self._cb_states) self._cb_states = list() @@ -41,9 +44,12 @@ def test_job_callbacks(self) -> None: jex.submit(job) job.wait() - self.assertEqual(len(self._cb_states), 3) + self.assertEqual(len(self._cb_states), 6) self.assertIn(psij.JobState.QUEUED, self._cb_states) + self.assertIn(psij.JobState.STAGE_IN, self._cb_states) self.assertIn(psij.JobState.ACTIVE, self._cb_states) + self.assertIn(psij.JobState.STAGE_OUT, self._cb_states) + self.assertIn(psij.JobState.CLEANUP, self._cb_states) self.assertIn(psij.JobState.COMPLETED, self._cb_states) def test_job_executor_callbacks(self) -> None: @@ -55,7 +61,10 @@ def test_job_executor_callbacks(self) -> None: jex.submit(job) job.wait() - self.assertEqual(len(self._cb_states), 3) + self.assertEqual(len(self._cb_states), 6) self.assertIn(psij.JobState.QUEUED, self._cb_states) + self.assertIn(psij.JobState.STAGE_IN, self._cb_states) self.assertIn(psij.JobState.ACTIVE, self._cb_states) + self.assertIn(psij.JobState.STAGE_OUT, self._cb_states) + self.assertIn(psij.JobState.CLEANUP, self._cb_states) self.assertIn(psij.JobState.COMPLETED, self._cb_states) diff --git a/tests/test_staging.py b/tests/test_staging.py index 2d6db631..f5225df2 100644 --- a/tests/test_staging.py +++ b/tests/test_staging.py @@ -1,25 +1,227 @@ -from tempfile import NamedTemporaryFile +from pathlib import Path from executor_test_params import ExecutorTestParams -from _test_tools import _get_executor_instance, _get_timeout, assert_completed, _make_test_dir -from psij import Job, JobSpec +from _test_tools import _get_executor_instance, _get_timeout, assert_completed, _tempfile, \ + _temppath, _tempdir, _write_file, _read_file +from psij import Job, JobSpec, JobState +from psij.staging import StageIn, StageOut, StagingMode, StageOutFlags +import pytest -def test_stagein(execparams: ExecutorTestParams) -> None: - with NamedTemporaryFile(delete=False) as outf: - outf.close() - with NamedTemporaryFile(mode='w', delete=False) as f: +@pytest.mark.parametrize('mode', [StagingMode.COPY, StagingMode.MOVE, StagingMode.LINK]) +def test_stagein(execparams: ExecutorTestParams, mode: StagingMode) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + # The launcher should not affect the staging, so we test all launchers locally, + # but for the other executors, we only test with the single launcher + pytest.skip() + # The executors are not mandated to implement the staging modes, but they are + # meant to default to COPY if MOVE and LINK are not implemented, so we test + # that things function correctly, but not how that is done + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + with _tempfile() as f1, _tempfile() as f2: + _write_file(f1, 'ABCD') + _write_file(f2, 'EFGH') + + job = Job(JobSpec('/bin/cat', ['in1.txt', 'subdir/in2.txt'], + directory=dir, stdout_path=out_path, stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_in = { + StageIn(f1.name, 'in1.txt', mode=mode), + StageIn(f2.name, 'subdir/in2.txt', mode=mode), + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + + assert _read_file(out_path) == 'ABCDEFGH' - f.write('ABCD') - f.close() - job = Job(JobSpec(executable='/bin/cat', stdout_path=outf.name, +@pytest.mark.parametrize('mode', [StagingMode.COPY, StagingMode.MOVE, StagingMode.LINK]) +def test_dir_stagein(execparams: ExecutorTestParams, mode: StagingMode) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + # The executors are not mandated to implement the staging modes, but they are + # meant to default to COPY if MOVE and LINK are not implemented, so we test + # that things function correctly, but not how that is done + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + with _tempdir() as in_dir: + sub_dir = in_dir / 'subdir' + sub_dir.mkdir() + f1 = sub_dir / 'in3.txt' + f2 = sub_dir / 'in4.txt' + _write_file(f1, 'IJKL') + _write_file(f2, 'MNOP') + + job = Job(JobSpec('/bin/cat', ['indir/in3.txt', 'indir/in4.txt'], + directory=dir, stdout_path=out_path, stderr_path=err_path, launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_in = { + StageIn(sub_dir, 'indir', mode=mode), + } ex = _get_executor_instance(execparams, job) ex.submit(job) status = job.wait(timeout=_get_timeout(execparams)) assert_completed(job, status) - with open(outf, 'r') as out: - result = out.read() - assert result.strip() == 'ABCD' + assert _read_file(out_path) == 'IJKLMNOP' + + +@pytest.mark.parametrize('mode', [StagingMode.COPY, StagingMode.MOVE, StagingMode.LINK]) +def test_stageout(execparams: ExecutorTestParams, mode: StagingMode) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + job = Job(JobSpec('/bin/echo', ['-n', 'CDEF'], directory=dir, + stdout_path='out.txt', stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, mode=mode) + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + + assert _read_file(out_path) == 'CDEF' + + +def test_stageout_flags1(execparams: ExecutorTestParams) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + with (_temppath() as out1_path, _temppath() as out2_path, _temppath() as err_path, + _tempdir() as dir): + + out2_path.unlink() + job = Job(JobSpec('/bin/echo', ['-n', 'ABC123'], + directory=dir, stdout_path='out1.txt', stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out1.txt', out1_path, flags=StageOutFlags.IF_PRESENT), + StageOut('out2.txt', out2_path, flags=StageOutFlags.IF_PRESENT) + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + + assert _read_file(out1_path) == 'ABC123' + assert not out2_path.exists() + + +def test_stageout_flags2(execparams: ExecutorTestParams) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + job = Job(JobSpec('/bin/echo', ['-n', 'EFG456'], + directory=dir, stdout_path='out.txt', stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, flags=StageOutFlags.ON_SUCCESS), + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + + assert _read_file(out_path) == 'EFG456' + + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + out_path.unlink() + job = Job(JobSpec('/bin/bash', ['-c', 'echo -n "ABC" > out.txt; exit 1'], + directory=dir, stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, flags=StageOutFlags.ON_SUCCESS), + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert status + assert status.state == JobState.FAILED + assert not out_path.exists() + + +def test_stageout_flags3(execparams: ExecutorTestParams) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + out_path.unlink() + job = Job(JobSpec('/bin/echo', ['-n', 'EFG456'], + directory=dir, stdout_path='out.txt', stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, flags=StageOutFlags.ON_ERROR), + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + + assert not out_path.exists() + + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + job = Job(JobSpec('/bin/bash', ['-c', 'echo -n "ABC" > out.txt; exit 1'], + directory=dir, stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, flags=StageOutFlags.ON_ERROR), + } + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert status + assert status.state == JobState.FAILED + assert out_path.exists() + assert _read_file(out_path) == 'ABC' + + +def test_cleanup(execparams: ExecutorTestParams) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + job = Job(JobSpec('/bin/echo', ['-n', 'ABC'], + directory=dir, stdout_path='out.txt', stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, flags=StageOutFlags.IF_PRESENT), + } + job.spec.cleanup = {Path('out.txt')} + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert_completed(job, status) + assert out_path.exists() + assert not (dir / 'out.txt').exists() + assert _read_file(out_path) == 'ABC' + + +def test_cleanup2(execparams: ExecutorTestParams) -> None: + if execparams.executor != 'local' and execparams.launcher != 'single': + pytest.skip() + with _temppath() as out_path, _temppath() as err_path, _tempdir() as dir: + job = Job(JobSpec('/bin/bash', ['-c', 'echo -n "ABC" > out.txt; exit 1'], + directory=dir, stderr_path=err_path, + launcher=execparams.launcher)) + assert job.spec is not None + job.spec.stage_out = { + StageOut('out.txt', out_path, flags=StageOutFlags.IF_PRESENT), + } + job.spec.cleanup = {Path('out.txt')} + job.spec.cleanup_on_failure = False + ex = _get_executor_instance(execparams, job) + ex.submit(job) + status = job.wait(timeout=_get_timeout(execparams)) + assert status is not None + assert status.state == JobState.FAILED + assert (dir / 'out.txt').exists() + assert _read_file(out_path) == 'ABC' From e55aedb8a1563fed5a3f2c1f7edd0383ea082665 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 10:06:11 -0800 Subject: [PATCH 03/39] Spelling --- src/psij/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/psij/utils.py b/src/psij/utils.py index 8f763a9e..72ddeada 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -126,7 +126,7 @@ def unregister_job(self, job: Job) -> None: try: del self._jobs[job.id] except KeyError: - # There are cases when it's difficult to esnure that this method is only called + # There are cases when it's difficult to ensure that this method is only called # once for each job. Instead, ignore errors here, since the ultimate goal is to # remove the job from the _jobs dictionary. pass From 157120011cc6c8affc7b9f4323d6eaca90ed9b1e Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 10:13:26 -0800 Subject: [PATCH 04/39] Hmm... --- tests/test_staging.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/test_staging.py b/tests/test_staging.py index f5225df2..d381612d 100644 --- a/tests/test_staging.py +++ b/tests/test_staging.py @@ -92,8 +92,8 @@ def test_stageout(execparams: ExecutorTestParams, mode: StagingMode) -> None: def test_stageout_flags1(execparams: ExecutorTestParams) -> None: if execparams.executor != 'local' and execparams.launcher != 'single': pytest.skip() - with (_temppath() as out1_path, _temppath() as out2_path, _temppath() as err_path, - _tempdir() as dir): + with _temppath() as out1_path, _temppath() as out2_path, _temppath() as err_path, \ + _tempdir() as dir: out2_path.unlink() job = Job(JobSpec('/bin/echo', ['-n', 'ABC123'], From 4f42c970ae47dd1079efd208295954a22f063279 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 16:38:22 -0800 Subject: [PATCH 05/39] Print IP address of GitHub CI runner. --- .github/workflows/python-package.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 74dab345..bdd13cfb 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -55,6 +55,7 @@ jobs: chmod 600 "$HOME/.ssh/authorized_keys" ls -la /etc/init.d/ sudo /etc/init.d/ssh start + curl https://api.ipify.org pip install -r requirements.txt pip install -r requirements-dev.txt pip install -r requirements-connector-radical.txt From 21374af4884b7c06a75b6a73a6c48ea0a0dbac8e Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 16:41:17 -0800 Subject: [PATCH 06/39] Need to install curl --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index bdd13cfb..6e53db51 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -46,7 +46,7 @@ jobs: cd /workspace echo "Running in ${PWD}" sudo apt update - sudo apt install -y openssh-server openssh-client + sudo apt install -y openssh-server openssh-client curl mkdir -p "$HOME/.ssh" chmod 0755 "$HOME" chmod 0700 "$HOME/.ssh" From 988da9ceb6dfe70913f035e00ffb26fab29af95c Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 17:09:56 -0800 Subject: [PATCH 07/39] Try a different way of connecting to CI VM --- .github/workflows/python-package.yml | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 6e53db51..e2f08856 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -38,15 +38,18 @@ jobs: steps: - uses: actions/checkout@v2 - name: Test with pytest - uses: addnab/docker-run-action@v3 + uses: | + addnab/docker-run-action@v3 + lhotari/action-upterm@v1 with: image: fluxrm/flux-core:focal options: -v ${{ github.workspace }}:/workspace -u root + limit-access-to-actor: true run: | cd /workspace echo "Running in ${PWD}" sudo apt update - sudo apt install -y openssh-server openssh-client curl + sudo apt install -y openssh-server openssh-client mkdir -p "$HOME/.ssh" chmod 0755 "$HOME" chmod 0700 "$HOME/.ssh" @@ -55,7 +58,6 @@ jobs: chmod 600 "$HOME/.ssh/authorized_keys" ls -la /etc/init.d/ sudo /etc/init.d/ssh start - curl https://api.ipify.org pip install -r requirements.txt pip install -r requirements-dev.txt pip install -r requirements-connector-radical.txt From eb19fcff40b48e02afa03eb80e4adb1a8a108348 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 17:14:20 -0800 Subject: [PATCH 08/39] Second try --- .github/workflows/python-package.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index e2f08856..70f4ee90 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -38,13 +38,13 @@ jobs: steps: - uses: actions/checkout@v2 - name: Test with pytest - uses: | - addnab/docker-run-action@v3 - lhotari/action-upterm@v1 + uses: lhotari/action-upterm@v1 + with: + limit-access-to-actor: true + uses: addnab/docker-run-action@v3 with: image: fluxrm/flux-core:focal options: -v ${{ github.workspace }}:/workspace -u root - limit-access-to-actor: true run: | cd /workspace echo "Running in ${PWD}" From 56c4d333416c9ffc1034b72a6cbfb3e0cb5ee081 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 17:18:37 -0800 Subject: [PATCH 09/39] and more --- .github/workflows/python-package.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 70f4ee90..e2f29c7e 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -38,10 +38,10 @@ jobs: steps: - uses: actions/checkout@v2 - name: Test with pytest - uses: lhotari/action-upterm@v1 + - uses: lhotari/action-upterm@v1 with: limit-access-to-actor: true - uses: addnab/docker-run-action@v3 + - uses: addnab/docker-run-action@v3 with: image: fluxrm/flux-core:focal options: -v ${{ github.workspace }}:/workspace -u root From d668832940e641bd04925a9ad0528d67dd267d68 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 17:20:46 -0800 Subject: [PATCH 10/39] Moved name in the proper place --- .github/workflows/python-package.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index e2f29c7e..3f694f19 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -37,11 +37,11 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 - - name: Test with pytest - uses: lhotari/action-upterm@v1 with: limit-access-to-actor: true - - uses: addnab/docker-run-action@v3 + - name: Test with pytest + uses: addnab/docker-run-action@v3 with: image: fluxrm/flux-core:focal options: -v ${{ github.workspace }}:/workspace -u root From 02576d1967e39506758b605d34b1b7cf875c66c8 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 17:27:51 -0800 Subject: [PATCH 11/39] Need something that doesn't block --- .github/workflows/python-package.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 3f694f19..ee26828f 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -37,8 +37,9 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 - - uses: lhotari/action-upterm@v1 + - uses: mxschmitt/action-tmate@v3 with: + detached: true limit-access-to-actor: true - name: Test with pytest uses: addnab/docker-run-action@v3 From 6e5eeb49a3bbda61cda12eb98af9d822d6c321e0 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 17:54:56 -0800 Subject: [PATCH 12/39] Dump thread stacks on SIGUSR1 when running tests to troubleshoot deadlocks and similar. --- tests/conftest.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/tests/conftest.py b/tests/conftest.py index 12af6530..1c7a420f 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -8,11 +8,13 @@ import re import secrets import shutil +import signal import socket import subprocess import sys import threading import time +import traceback from functools import partial from pathlib import Path from typing import Dict, List, Optional @@ -76,6 +78,22 @@ def pytest_addoption(parser): 'is uploaded to the test aggregation server. ') +def debug(sig, frame): + with open('/tmp/python-dump.txt', 'w') as f: + f.write('Test') + try: + for thr in threading.enumerate(): + f.write(str(thr)) + f.write('\n') + traceback.print_stack(sys._current_frames()[thr.ident], file=f) + f.write('\n\n') + except Exception as ex: + f.write(str(ex)) + + +signal.signal(signal.SIGUSR1, debug) # Register handler + + def _get_executors(config: Dict[str, str]) -> List[str]: execs_str = config.getoption('executors') execs = execs_str.split(',') From 3fb10354b472d1839ca9ddcbd459adb69c28a850 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 18:16:47 -0800 Subject: [PATCH 13/39] Disable output capturing temporarily --- .github/workflows/python-package.yml | 2 +- tests/conftest.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index ee26828f..0a380324 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -64,4 +64,4 @@ jobs: pip install -r requirements-connector-radical.txt make launcher-scripts make install - PYTHONPATH=$(flux env | grep PYTHONPATH | sed -E 's/.*PYTHONPATH="(.*)"/\1/') OMPI_ALLOW_RUN_AS_ROOT=1 OMPI_ALLOW_RUN_AS_ROOT_CONFIRM=1 make tests + PYTHONPATH=$(flux env | grep PYTHONPATH | sed -E 's/.*PYTHONPATH="(.*)"/\1/') OMPI_ALLOW_RUN_AS_ROOT=1 OMPI_ALLOW_RUN_AS_ROOT_CONFIRM=1 make tests -- -s diff --git a/tests/conftest.py b/tests/conftest.py index 1c7a420f..f888cc6e 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -80,7 +80,6 @@ def pytest_addoption(parser): def debug(sig, frame): with open('/tmp/python-dump.txt', 'w') as f: - f.write('Test') try: for thr in threading.enumerate(): f.write(str(thr)) @@ -91,7 +90,8 @@ def debug(sig, frame): f.write(str(ex)) -signal.signal(signal.SIGUSR1, debug) # Register handler +signal.signal(signal.SIGUSR1, debug) +print('SIGUSR1 handler installed.') def _get_executors(config: Dict[str, str]) -> List[str]: From 2af91b2ae45c42c6e6059440da76933232f9ee0f Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 20:41:24 -0800 Subject: [PATCH 14/39] More robustness in status updater --- src/psij/utils.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/psij/utils.py b/src/psij/utils.py index 72ddeada..89a2f262 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -145,12 +145,18 @@ def step(self) -> None: self._process_update_data(data) except TimeoutError: pass + except socket.timeout: + # before 3.10, this was a separate exception from TimeoutError + pass except BlockingIOError: pass def run(self) -> None: while True: - self.step() + try: + self.step() + except Exception: + logger.exception('Exception in status updater thread. Ignoring.') def flush(self) -> None: # Ensures that, upon return from this call, all updates available before this call have From 307820f7fc9b89e761e55e35ecdf0a4afe95fcc7 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 20:41:40 -0800 Subject: [PATCH 15/39] Some more debugging info when dumping stacks --- tests/conftest.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/conftest.py b/tests/conftest.py index f888cc6e..65c37d48 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -79,6 +79,7 @@ def pytest_addoption(parser): def debug(sig, frame): + print('Dumping thread info') with open('/tmp/python-dump.txt', 'w') as f: try: for thr in threading.enumerate(): @@ -87,6 +88,7 @@ def debug(sig, frame): traceback.print_stack(sys._current_frames()[thr.ident], file=f) f.write('\n\n') except Exception as ex: + logger.exception('Failed to dump thread info') f.write(str(ex)) From 3e168081a633c89e65c5247c5a39c996c20d75ac Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 21:44:45 -0800 Subject: [PATCH 16/39] A bit of cleanup and a warning for the future --- src/psij/utils.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/psij/utils.py b/src/psij/utils.py index 89a2f262..4ccb6a8c 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -162,7 +162,9 @@ def flush(self) -> None: # Ensures that, upon return from this call, all updates available before this call have # been processed. To do so, we send a UDP packet to the socket to wake it up and wait until # it is received. This does not guarantee that file-based updates are necessarily - # processes, since that depends on many factors. + # processed, since that depends on many factors. + # On the minus side, this method, as implemented, can cause deadlocks if the socket + # reads fail for unexpected reasons. This should probably be accounted for. token = '_SYNC ' + str(random.getrandbits(128)) sock = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) sock.sendto(bytes(token, 'utf-8'), ('127.0.0.1', self.update_port)) @@ -170,6 +172,7 @@ def flush(self) -> None: while token not in self._sync_ids: time.sleep(delay) delay *= 2 + self._sync_ids.remove(token) def _process_update_data(self, data: bytes) -> None: sdata = data.decode('utf-8') From 318c1124f06c9c961598086d0123c6cf9d73c670 Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 24 Feb 2024 21:45:02 -0800 Subject: [PATCH 17/39] Removed debugging stuff --- .github/workflows/python-package.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 0a380324..94eefaa0 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -37,10 +37,10 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 - - uses: mxschmitt/action-tmate@v3 - with: - detached: true - limit-access-to-actor: true + #- uses: mxschmitt/action-tmate@v3 + # with: + # detached: true + # limit-access-to-actor: true - name: Test with pytest uses: addnab/docker-run-action@v3 with: @@ -64,4 +64,4 @@ jobs: pip install -r requirements-connector-radical.txt make launcher-scripts make install - PYTHONPATH=$(flux env | grep PYTHONPATH | sed -E 's/.*PYTHONPATH="(.*)"/\1/') OMPI_ALLOW_RUN_AS_ROOT=1 OMPI_ALLOW_RUN_AS_ROOT_CONFIRM=1 make tests -- -s + PYTHONPATH=$(flux env | grep PYTHONPATH | sed -E 's/.*PYTHONPATH="(.*)"/\1/') OMPI_ALLOW_RUN_AS_ROOT=1 OMPI_ALLOW_RUN_AS_ROOT_CONFIRM=1 make tests From 0faccf510970e135563dc914115bb939dbf0cf8c Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 09:04:28 -0800 Subject: [PATCH 18/39] Don't print spurious notifications, since they mostly pop up for canceled jobs and spam the pytest suite, since pytest doesn't capture the log after the test is done. --- src/psij/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/psij/utils.py b/src/psij/utils.py index 4ccb6a8c..2c89d19e 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -201,6 +201,6 @@ def _process_update_data(self, data: bytes) -> None: try: (job, executor) = self._jobs[job_id] except KeyError: - logger.debug('Received status updated for inexistent job with id %s' % job_id) + pass if job: executor._set_job_status(job, JobStatus(state)) From 9478857167d4c83a8efd99283cd28ec5d0a84698 Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 09:04:51 -0800 Subject: [PATCH 19/39] Fixed race condition in test. --- tests/user_guide/test_job_wait_active.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/user_guide/test_job_wait_active.py b/tests/user_guide/test_job_wait_active.py index 839bfc2c..8b67b8f5 100644 --- a/tests/user_guide/test_job_wait_active.py +++ b/tests/user_guide/test_job_wait_active.py @@ -3,8 +3,9 @@ def test_user_guide_job_wait_active() -> None: ex = JobExecutor.get_instance('local') - job = Job(JobSpec(executable='/bin/date')) + job = Job(JobSpec(executable='/bin/sleep', arguments=['1'])) ex.submit(job) status = job.wait(target_states=[JobState.ACTIVE]) assert status is not None assert status.state == JobState.ACTIVE + job.wait() # prevent logging messages from showing up after test completes From 7656b1a36f04c916ea3364b62c5ba4b0efcb7407 Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 09:12:42 -0800 Subject: [PATCH 20/39] formatting --- tests/user_guide/test_job_wait_active.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/user_guide/test_job_wait_active.py b/tests/user_guide/test_job_wait_active.py index 8b67b8f5..86fbade0 100644 --- a/tests/user_guide/test_job_wait_active.py +++ b/tests/user_guide/test_job_wait_active.py @@ -8,4 +8,4 @@ def test_user_guide_job_wait_active() -> None: status = job.wait(target_states=[JobState.ACTIVE]) assert status is not None assert status.state == JobState.ACTIVE - job.wait() # prevent logging messages from showing up after test completes + job.wait() # prevent logging messages from showing up after test completes From adc7486447ec820fd5728d8cc050a6c186c142f2 Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 10:04:54 -0800 Subject: [PATCH 21/39] Added some debugging statements. --- tests/test_callbacks.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/test_callbacks.py b/tests/test_callbacks.py index af51d941..1d3f03f3 100644 --- a/tests/test_callbacks.py +++ b/tests/test_callbacks.py @@ -18,6 +18,7 @@ def __init__(self, arg: Any) -> None: def state_cb(self, job: psij.Job, status: psij.JobStatus) -> None: """State callback.""" + print('status change: %s' % status) self._cb_states.append(status.state) def test_job_callbacks(self) -> None: @@ -29,6 +30,8 @@ def test_job_callbacks(self) -> None: jex.submit(job) job.wait() + print('States: %s' % self._cb_states) + self.assertEqual(len(self._cb_states), 6) self.assertIn(psij.JobState.QUEUED, self._cb_states) self.assertIn(psij.JobState.STAGE_IN, self._cb_states) @@ -61,6 +64,8 @@ def test_job_executor_callbacks(self) -> None: jex.submit(job) job.wait() + print('States: %s' % self._cb_states) + self.assertEqual(len(self._cb_states), 6) self.assertIn(psij.JobState.QUEUED, self._cb_states) self.assertIn(psij.JobState.STAGE_IN, self._cb_states) From 19698883a08be8a10043220ea80025b2fd58dd7c Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 16:11:36 -0800 Subject: [PATCH 22/39] Trying to remove some seemingly spurious options for better portability --- src/psij/executors/batch/common/batch_lib.mustache | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index f0fe162d..29c07d10 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -13,7 +13,7 @@ update_status() { if [ "$_UPDATE_MODE" == "nc" ]; then ADDRS={{psij.us_addrs}} for ADDR in ${ADDRS//,/ }; do - echo "{{job.id}} $STATUS" | nc -q0 -w0 -4 -u $ADDR {{psij.us_port}} + echo "{{job.id}} $STATUS" | nc -4 -u $ADDR {{psij.us_port}} done else echo "{{job.id}} $STATUS" >> {{psij.us_file}} From 10a1c244170354894c5738b67d71f3b1c3d8d68a Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 16:25:54 -0800 Subject: [PATCH 23/39] Install netcat to test status updates --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 94eefaa0..edab8dad 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -50,7 +50,7 @@ jobs: cd /workspace echo "Running in ${PWD}" sudo apt update - sudo apt install -y openssh-server openssh-client + sudo apt install -y openssh-server openssh-client netcat-traditional mkdir -p "$HOME/.ssh" chmod 0755 "$HOME" chmod 0700 "$HOME/.ssh" From 3afb09478d661618498801286eb91a89356466c1 Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 17:05:50 -0800 Subject: [PATCH 24/39] There is, of course, more than one type of netcat. --- .../executors/batch/common/batch_lib.mustache | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index 29c07d10..271399bf 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -3,17 +3,28 @@ _UPDATE_MODE="none" update_status() { STATUS="$1" if [ "$_UPDATE_MODE" == "none" ]; then - if which nc >/dev/null; then - _UPDATE_MODE="nc" + NCPROBE=`nc -h 2>&1` + EC=$? + if [ "$EC" == "0" ]; then + if echo $NCPROBE | grep "send-only" >/dev/null 2>&1 ; then + _UPDATE_MODE="nc1" + else + _UPDATE_MODE="nc2" + fi else _UPDATE_MODE="file" fi fi - if [ "$_UPDATE_MODE" == "nc" ]; then + if [ "$_UPDATE_MODE" == "nc1" ]; then ADDRS={{psij.us_addrs}} for ADDR in ${ADDRS//,/ }; do - echo "{{job.id}} $STATUS" | nc -4 -u $ADDR {{psij.us_port}} + echo "{{job.id}} $STATUS" | nc --send-only -4 -u $ADDR {{psij.us_port}} + done + elif [ "$_UPDATE_MODE" == "nc2" ]; then + ADDRS={{psij.us_addrs}} + for ADDR in ${ADDRS//,/ }; do + echo "{{job.id}} $STATUS" | nc -w0 -q0 -4 -u $ADDR {{psij.us_port}} done else echo "{{job.id}} $STATUS" >> {{psij.us_file}} From 2b64908b82f80d4e18dfc560ef10c53aef980f95 Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 17:20:56 -0800 Subject: [PATCH 25/39] A version that should work with netcat-classic in the flux container. --- src/psij/executors/batch/common/batch_lib.mustache | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index 271399bf..e575e660 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -24,7 +24,7 @@ update_status() { elif [ "$_UPDATE_MODE" == "nc2" ]; then ADDRS={{psij.us_addrs}} for ADDR in ${ADDRS//,/ }; do - echo "{{job.id}} $STATUS" | nc -w0 -q0 -4 -u $ADDR {{psij.us_port}} + echo "{{job.id}} $STATUS" | nc -q0 -u $ADDR {{psij.us_port}} done else echo "{{job.id}} $STATUS" >> {{psij.us_file}} From c4721b4cf78611a6f5c45a56f59b04dbd505afa9 Mon Sep 17 00:00:00 2001 From: hategan Date: Sun, 25 Feb 2024 17:21:44 -0800 Subject: [PATCH 26/39] Poll the file, too, on flush. --- src/psij/utils.py | 25 ++++++++++++++----------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/src/psij/utils.py b/src/psij/utils.py index 2c89d19e..970ca6c5 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -132,6 +132,19 @@ def unregister_job(self, job: Job) -> None: pass def step(self) -> None: + self._poll_file() + try: + data = self.socket.recv(_StatusUpdater.RECV_BUFSZ) + self._process_update_data(data) + except TimeoutError: + pass + except socket.timeout: + # before 3.10, this was a separate exception from TimeoutError + pass + except BlockingIOError: + pass + + def _poll_file(self) -> None: self.update_file.seek(0, io.SEEK_END) pos = self.update_file.tell() if pos > self.update_file_pos: @@ -139,17 +152,6 @@ def step(self) -> None: n = pos - self.update_file_pos self._process_update_data(self.update_file.read(n)) self.update_file_pos = pos - else: - try: - data = self.socket.recv(_StatusUpdater.RECV_BUFSZ) - self._process_update_data(data) - except TimeoutError: - pass - except socket.timeout: - # before 3.10, this was a separate exception from TimeoutError - pass - except BlockingIOError: - pass def run(self) -> None: while True: @@ -168,6 +170,7 @@ def flush(self) -> None: token = '_SYNC ' + str(random.getrandbits(128)) sock = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) sock.sendto(bytes(token, 'utf-8'), ('127.0.0.1', self.update_port)) + self._poll_file() delay = 0.0001 while token not in self._sync_ids: time.sleep(delay) From 94a2c0ed17d116c8f81a77dbf384fa3c17560637 Mon Sep 17 00:00:00 2001 From: hategan Date: Mon, 26 Feb 2024 16:11:16 -0800 Subject: [PATCH 27/39] Don't bother with netcat, which may not be installed, when we can use special devices which bash will happily emulate if not found. --- .../executors/batch/common/batch_lib.mustache | 32 +++---------------- 1 file changed, 4 insertions(+), 28 deletions(-) diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index e575e660..6b706e63 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -1,34 +1,10 @@ -_UPDATE_MODE="none" - update_status() { STATUS="$1" - if [ "$_UPDATE_MODE" == "none" ]; then - NCPROBE=`nc -h 2>&1` - EC=$? - if [ "$EC" == "0" ]; then - if echo $NCPROBE | grep "send-only" >/dev/null 2>&1 ; then - _UPDATE_MODE="nc1" - else - _UPDATE_MODE="nc2" - fi - else - _UPDATE_MODE="file" - fi - fi - if [ "$_UPDATE_MODE" == "nc1" ]; then - ADDRS={{psij.us_addrs}} - for ADDR in ${ADDRS//,/ }; do - echo "{{job.id}} $STATUS" | nc --send-only -4 -u $ADDR {{psij.us_port}} - done - elif [ "$_UPDATE_MODE" == "nc2" ]; then - ADDRS={{psij.us_addrs}} - for ADDR in ${ADDRS//,/ }; do - echo "{{job.id}} $STATUS" | nc -q0 -u $ADDR {{psij.us_port}} - done - else - echo "{{job.id}} $STATUS" >> {{psij.us_file}} - fi + ADDRS={{psij.us_addrs}} + for ADDR in ${ADDRS//,/ }; do + echo "{{job.id}} $STATUS" >/dev/udp/$ADDR/{{psij.us_port}} + done } fail() { From 4e7678c148cd024972f21f2cdfcc41f3802cbacf Mon Sep 17 00:00:00 2001 From: hategan Date: Wed, 28 Feb 2024 09:23:18 -0800 Subject: [PATCH 28/39] Simplified the error logic in launcher/batch scripts. Since the addition of staging, errors can reasonably occur outside of the launcher as well as inside the launcher and pre- and post- launch scripts. So we treat the output of the whole script (including launcher) as potential indicator of an error. --- .../batch/batch_scheduler_executor.py | 5 +-- .../executors/batch/cobalt/cobalt.mustache | 12 +++--- src/psij/executors/batch/lsf/lsf.mustache | 12 +++--- .../executors/batch/pbs/pbs_classic.mustache | 5 ++- src/psij/executors/batch/pbs/pbspro.mustache | 5 ++- src/psij/executors/batch/slurm/slurm.mustache | 12 +++--- src/psij/executors/local.py | 4 +- src/psij/executors/local/local.mustache | 1 + src/psij/job_launcher.py | 37 ------------------ src/psij/launcher.py | 38 ------------------- src/psij/launchers/script_based_launcher.py | 12 ------ src/psij/launchers/scripts/aprun_launch.sh | 1 - src/psij/launchers/scripts/jsrun_launch.sh | 1 - src/psij/launchers/scripts/mpi_launch.sh | 1 - src/psij/launchers/scripts/multi_launch.sh | 1 - src/psij/launchers/scripts/single_launch.sh | 1 - src/psij/launchers/scripts/srun_launch.sh | 1 - tests/plugins1/_batch_test/test/launcher.sh | 1 - tests/plugins1/_batch_test/test/test.mustache | 1 + 19 files changed, 33 insertions(+), 118 deletions(-) diff --git a/src/psij/executors/batch/batch_scheduler_executor.py b/src/psij/executors/batch/batch_scheduler_executor.py index 090138b0..1895c281 100644 --- a/src/psij/executors/batch/batch_scheduler_executor.py +++ b/src/psij/executors/batch/batch_scheduler_executor.py @@ -573,9 +573,8 @@ def _read_aux_files(self, job: Job, status: JobStatus) -> None: # already present out = self._read_aux_file(job, '.out') if out: - launcher = self._get_launcher_from_job(job) - if launcher.is_launcher_failure(out): - status.message = launcher.get_launcher_failure_message(out) + if '_PSIJ_SCRIPT_DONE' not in out: + status.message = out logger.debug('Output from launcher: %s', status.message) else: self._delete_aux_file(job, '.out') diff --git a/src/psij/executors/batch/cobalt/cobalt.mustache b/src/psij/executors/batch/cobalt/cobalt.mustache index b41cdb75..4f67721b 100644 --- a/src/psij/executors/batch/cobalt/cobalt.mustache +++ b/src/psij/executors/batch/cobalt/cobalt.mustache @@ -46,6 +46,12 @@ only results in empty files that are not cleaned up}} #COBALT -e /dev/null #COBALT -o /dev/null +{{!redirect output here instead of through #COBALT directive since COBALT_JOB_ID is not available +when the directives are evaluated; the reason for using the job id in the first place being the +same as for the exit code file.}} +exec &>> "{{psij.script_dir}}/$COBALT_JOBID.out" + + {{> batch_lib}} {{!like PBS, this is also cheap and there is not need to check setting}} @@ -55,11 +61,6 @@ export PSIJ_NODEFILE {{> stagein}} update_status ACTIVE -{{!redirect output here instead of through #COBALT directive since COBALT_JOB_ID is not available -when the directives are evaluated; the reason for using the job id in the first place being the -same as for the exit code file.}} -exec &>> "{{psij.script_dir}}/$COBALT_JOBID.out" - {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? @@ -68,3 +69,4 @@ _PSIJ_JOB_EC=$? {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$COBALT_JOBID.ec" +echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file diff --git a/src/psij/executors/batch/lsf/lsf.mustache b/src/psij/executors/batch/lsf/lsf.mustache index 0ca54ecb..24641d25 100644 --- a/src/psij/executors/batch/lsf/lsf.mustache +++ b/src/psij/executors/batch/lsf/lsf.mustache @@ -71,6 +71,12 @@ only results in empty files that are not cleaned up}} #BSUB -e /dev/null #BSUB -o /dev/null +{{!redirect output here instead of through #BSUB directive since LSB_JOBID is not available +when the directives are evaluated; the reason for using the job id in the first place being the +same as for the exit code file.}} +exec &>> "{{psij.script_dir}}/$LSB_JOBID.out" + + {{> batch_lib}} PSIJ_NODEFILE="$LSB_HOSTS" @@ -79,11 +85,6 @@ export PSIJ_NODEFILE {{> stagein}} update_status ACTIVE -{{!redirect output here instead of through #BSUB directive since LSB_JOBID is not available -when the directives are evaluated; the reason for using the job id in the first place being the -same as for the exit code file.}} -exec &>> "{{psij.script_dir}}/$LSB_JOBID.out" - {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? @@ -92,3 +93,4 @@ _PSIJ_JOB_EC=$? {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$LSB_JOBID.ec" +echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file diff --git a/src/psij/executors/batch/pbs/pbs_classic.mustache b/src/psij/executors/batch/pbs/pbs_classic.mustache index a7859437..c48a6769 100644 --- a/src/psij/executors/batch/pbs/pbs_classic.mustache +++ b/src/psij/executors/batch/pbs/pbs_classic.mustache @@ -55,6 +55,8 @@ only results in empty files that are not cleaned up}} #PBS -v {{name}}={{value}} {{/env}} +exec &>> "{{psij.script_dir}}/$PBS_JOBID.out" + {{> batch_lib}} PSIJ_NODEFILE="$PBS_NODEFILE" @@ -67,8 +69,6 @@ cd "{{.}}" {{> stagein}} update_status ACTIVE -exec &>> "{{psij.script_dir}}/$PBS_JOBID.out" - {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? @@ -77,3 +77,4 @@ _PSIJ_JOB_EC=$? {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$PBS_JOBID.ec" +echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file diff --git a/src/psij/executors/batch/pbs/pbspro.mustache b/src/psij/executors/batch/pbs/pbspro.mustache index 78a3d2ae..cdadd657 100644 --- a/src/psij/executors/batch/pbs/pbspro.mustache +++ b/src/psij/executors/batch/pbs/pbspro.mustache @@ -51,6 +51,8 @@ only results in empty files that are not cleaned up}} #PBS -e /dev/null #PBS -o /dev/null +exec &>> "{{psij.script_dir}}/$PBS_JOBID.out" + {{#job.spec.inherit_environment}} #PBS -V {{/job.spec.inherit_environment}} @@ -70,8 +72,6 @@ cd "{{.}}" {{> stagein}} update_status ACTIVE -exec &>> "{{psij.script_dir}}/$PBS_JOBID.out" - {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? @@ -80,3 +80,4 @@ _PSIJ_JOB_EC=$? {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} echo "$?" > "{{psij.script_dir}}/$PBS_JOBID.ec" +echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file diff --git a/src/psij/executors/batch/slurm/slurm.mustache b/src/psij/executors/batch/slurm/slurm.mustache index 9daa060b..d2bad977 100644 --- a/src/psij/executors/batch/slurm/slurm.mustache +++ b/src/psij/executors/batch/slurm/slurm.mustache @@ -88,6 +88,12 @@ _PSIJ_PPN={{.}} {{/processes_per_node}} {{/job.spec.resources}} +{{!redirect output here instead of through #SBATCH directive since SLURM_JOB_ID is not available +when the directives are evaluated; the reason for using the job id in the first place being the +same as for the exit code file.}} +exec &>> "{{psij.script_dir}}/$SLURM_JOB_ID.out" + + {{> batch_lib}} _PSIJ_NC=`scontrol show hostnames | wc -l` @@ -111,11 +117,6 @@ export PSIJ_NODEFILE {{> stagein}} update_status ACTIVE -{{!redirect output here instead of through #SBATCH directive since SLURM_JOB_ID is not available -when the directives are evaluated; the reason for using the job id in the first place being the -same as for the exit code file.}} -exec &>> "{{psij.script_dir}}/$SLURM_JOB_ID.out" - {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? @@ -124,3 +125,4 @@ _PSIJ_JOB_EC=$? {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$SLURM_JOB_ID.ec" +echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file diff --git a/src/psij/executors/local.py b/src/psij/executors/local.py index ab3d4bc5..31cace23 100644 --- a/src/psij/executors/local.py +++ b/src/psij/executors/local.py @@ -335,8 +335,8 @@ def _process_done(self, p: _ProcessEntry) -> None: # the exit code of the launcher is the exit code of the job, we must use a different # mechanism to distinguish between job errors and launcher errors. So we delegate to # the launcher implementation to figure out if the error belongs to the job or not - if p.launcher and p.out and p.launcher.is_launcher_failure(p.out): - message = p.launcher.get_launcher_failure_message(p.out) + if p.out and '_PSIJ_SCRIPT_DONE' not in p.out: + message = p.out state = JobState.FAILED # We need to ensure that the status updater has processed all updates that diff --git a/src/psij/executors/local/local.mustache b/src/psij/executors/local/local.mustache index 84941895..e2c0a8c2 100644 --- a/src/psij/executors/local/local.mustache +++ b/src/psij/executors/local/local.mustache @@ -22,4 +22,5 @@ set -e {{> stageout}} {{> cleanup}} +echo "_PSIJ_SCRIPT_DONE" exit "$_PSIJ_JOB_EC" \ No newline at end of file diff --git a/src/psij/job_launcher.py b/src/psij/job_launcher.py index c156cdc2..1e7aae3b 100644 --- a/src/psij/job_launcher.py +++ b/src/psij/job_launcher.py @@ -34,43 +34,6 @@ def get_launch_command(self, job: Job) -> List[str]: """ pass - @abstractmethod - def is_launcher_failure(self, output: str) -> bool: - """ - Determines whether the launcher invocation output contains a launcher failure or not. - - Parameters - ---------- - output - The output (combined stdout/stderr) from an invocation of the launcher command - - Returns - ------- - Returns `True` if the `output` parameter contains a string that represents a launncher - failure. - """ - pass - - @abstractmethod - def get_launcher_failure_message(self, output: str) -> str: - """ - Extracts the launcher error message from the output of this launcher's invocation. - - It is understood that the value of the `output` parameter is such that - :meth:`is_launcher_failure` returns `True` on it. - - Parameters - ---------- - output - The output (combined stdout/stderr) from an invocation of the launcher command. - - Returns - ------- - A string representing the part of the launcher output that describes the launcher - error. - """ - pass - @staticmethod def get_instance(name: str, version_constraint: Optional[str] = None, config: Optional[JobExecutorConfig] = None) -> 'Launcher': diff --git a/src/psij/launcher.py b/src/psij/launcher.py index 1aab4a0c..29e855ea 100644 --- a/src/psij/launcher.py +++ b/src/psij/launcher.py @@ -36,44 +36,6 @@ def get_launch_command(self, job: Job) -> List[str]: """ pass - @abstractmethod - def is_launcher_failure(self, output: str) -> bool: - """ - Determines whether the launcher invocation output contains a launcher failure or not. - - Parameters - ---------- - output - The output (combined stdout/stderr) from an invocation of the launcher command - - Returns - ------- - Returns `True` if the output of the launcher indicates that it has exited with a - non-zero exit code due to an error occurring in the launcher. - - """ - pass - - @abstractmethod - def get_launcher_failure_message(self, output: str) -> str: - """ - Extracts the launcher error message from the output of this launcher's invocation. - - It is understood that the output is such that - :func:`~psij.launcher.Launcher.is_launcher_failure` returns `True` on it. - - Parameters - ---------- - output - The output (combined stdout/stderr) from an invocation of the launcher command. - - Returns - ------- - A string representing the part of the launcher output that describes the launcher - error. - """ - pass - @staticmethod def get_instance(name: str, version_constraint: Optional[str] = None, config: Optional[JobExecutorConfig] = None) -> 'Launcher': diff --git a/src/psij/launchers/script_based_launcher.py b/src/psij/launchers/script_based_launcher.py index 4a263106..45519ff7 100644 --- a/src/psij/launchers/script_based_launcher.py +++ b/src/psij/launchers/script_based_launcher.py @@ -197,15 +197,3 @@ def get_additional_args(self, job: Job) -> List[str]: :param job: The job that is being launched. """ return [] - - def is_launcher_failure(self, output: str) -> bool: - """See :func:`~psij.Launcher.is_launcher_failure`.""" - lines = output.split('\n') - return lines[-2] != '_PSI_J_LAUNCHER_DONE' or lines[-1] != '' - - def get_launcher_failure_message(self, output: str) -> str: - """See :func:`~psij.Launcher.get_launcher_failure_message`.""" - # Errors can occur in the job script after the launcher is done (e.g., stageout), - # so we need to filter the launcher tag - lines = output.split('\n') - return '\n'.join(filter(lambda x: x != '_PSI_J_LAUNCHER_DONE', lines)) diff --git a/src/psij/launchers/scripts/aprun_launch.sh b/src/psij/launchers/scripts/aprun_launch.sh index 8134f575..bd586ade 100644 --- a/src/psij/launchers/scripts/aprun_launch.sh +++ b/src/psij/launchers/scripts/aprun_launch.sh @@ -16,5 +16,4 @@ log "Command done: $_PSI_J_EC" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_EC diff --git a/src/psij/launchers/scripts/jsrun_launch.sh b/src/psij/launchers/scripts/jsrun_launch.sh index 73efe3d8..80fdcff4 100755 --- a/src/psij/launchers/scripts/jsrun_launch.sh +++ b/src/psij/launchers/scripts/jsrun_launch.sh @@ -16,5 +16,4 @@ log "Command done: $_PSI_J_EC" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_EC diff --git a/src/psij/launchers/scripts/mpi_launch.sh b/src/psij/launchers/scripts/mpi_launch.sh index 7a76d108..bfb85fcb 100644 --- a/src/psij/launchers/scripts/mpi_launch.sh +++ b/src/psij/launchers/scripts/mpi_launch.sh @@ -25,5 +25,4 @@ log "Command done: $_PSI_J_EC" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_EC diff --git a/src/psij/launchers/scripts/multi_launch.sh b/src/psij/launchers/scripts/multi_launch.sh index 9311ba0c..2c9f3213 100644 --- a/src/psij/launchers/scripts/multi_launch.sh +++ b/src/psij/launchers/scripts/multi_launch.sh @@ -29,5 +29,4 @@ log "All completed" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_FAILED_EC diff --git a/src/psij/launchers/scripts/single_launch.sh b/src/psij/launchers/scripts/single_launch.sh index 112609b0..88e5d235 100644 --- a/src/psij/launchers/scripts/single_launch.sh +++ b/src/psij/launchers/scripts/single_launch.sh @@ -12,5 +12,4 @@ log "Command done: $_PSI_J_EC" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_EC diff --git a/src/psij/launchers/scripts/srun_launch.sh b/src/psij/launchers/scripts/srun_launch.sh index d3808f6d..d0ac39ca 100644 --- a/src/psij/launchers/scripts/srun_launch.sh +++ b/src/psij/launchers/scripts/srun_launch.sh @@ -16,5 +16,4 @@ log "Command done: $_PSI_J_EC" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_EC diff --git a/tests/plugins1/_batch_test/test/launcher.sh b/tests/plugins1/_batch_test/test/launcher.sh index 5d7011e6..13a188ed 100644 --- a/tests/plugins1/_batch_test/test/launcher.sh +++ b/tests/plugins1/_batch_test/test/launcher.sh @@ -51,5 +51,4 @@ log "All completed" post_launch -echo "_PSI_J_LAUNCHER_DONE" exit $_PSI_J_FAILED_EC diff --git a/tests/plugins1/_batch_test/test/test.mustache b/tests/plugins1/_batch_test/test/test.mustache index 8fec2597..f1f2dda0 100644 --- a/tests/plugins1/_batch_test/test/test.mustache +++ b/tests/plugins1/_batch_test/test/test.mustache @@ -57,3 +57,4 @@ _PSIJ_JOB_EC="$?" echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$PSIJ_BATCH_TEST_JOB_ID.ec" +echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file From b438a484f75cda91e143cfc754156bd8185f85a2 Mon Sep 17 00:00:00 2001 From: hategan Date: Thu, 29 Feb 2024 21:23:06 -0800 Subject: [PATCH 29/39] Create test directories in ~/.psij/test, since CNs may not have access to CWD (the CI runner runs things in /tmp). --- tests/_test_tools.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/_test_tools.py b/tests/_test_tools.py index cedf1391..7ee08150 100644 --- a/tests/_test_tools.py +++ b/tests/_test_tools.py @@ -117,7 +117,8 @@ def _temppath() -> Iterator[Path]: @contextmanager def _tempdir(keep: bool = False) -> Iterator[Path]: _make_test_dir() - d = tempfile.mkdtemp(dir=Path.cwd()) + test_dir = Path.home() / '.psij' / 'test' + d = tempfile.mkdtemp(dir=test_dir) try: yield Path(d) shutil.rmtree(d) From 5fd72d7ead18d5165dee074f739b8a34971658e7 Mon Sep 17 00:00:00 2001 From: hategan Date: Fri, 1 Mar 2024 09:11:34 -0800 Subject: [PATCH 30/39] Also canonicalize job dir before testing cleanup inclusion in it. --- src/psij/executors/batch/common/batch_lib.mustache | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index 6b706e63..93c18977 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -115,15 +115,16 @@ do_cleanup() { if [ "$FAILED" == "0" ] || [ "{{job.spec.cleanup_on_failure}}" != "0" ]; then TARGET=`readlink -m "$TARGET"` + DIR=`readlink -m "{{job.spec.directory}}"` [ "{{psij.debug}}" != "0" ] && update_status "LOG Cleaning up $TARGET" case "$TARGET" in - "{{job.spec.directory}}"*) + "$DIR"*) rm -rf "$TARGET" ;; *) - fail 121 "Cannot clean $TARGET outside of job directory {{job.spec.directory}}" + fail 121 "Cannot clean $TARGET outside of job directory $DIR" ;; esac fi From 30d0d3404582167cf08f56cbc7a1ac56755ae88f Mon Sep 17 00:00:00 2001 From: hategan Date: Sat, 9 Mar 2024 09:59:54 -0800 Subject: [PATCH 31/39] Fixed exit code --- src/psij/executors/batch/pbs/pbspro.mustache | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/psij/executors/batch/pbs/pbspro.mustache b/src/psij/executors/batch/pbs/pbspro.mustache index cc3622f5..6e9d9c6d 100644 --- a/src/psij/executors/batch/pbs/pbspro.mustache +++ b/src/psij/executors/batch/pbs/pbspro.mustache @@ -78,5 +78,5 @@ _PSIJ_JOB_EC=$? {{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} -echo "$?" > "{{psij.script_dir}}/$PBS_JOBID.ec" +echo $_PSIJ_JOB_EC > "{{psij.script_dir}}/$PBS_JOBID.ec" echo "_PSIJ_SCRIPT_DONE" \ No newline at end of file From 2190c8755377b0e324b050ef5d3b20bc466ba809 Mon Sep 17 00:00:00 2001 From: hategan Date: Tue, 12 Mar 2024 11:54:14 -0700 Subject: [PATCH 32/39] - renamed internal psij functions to have a `_psij_` prefix in order to avoid clashes with other things that might be defined in the environment. - use `StageOutFlags` to express when cleanup happens rather than a boolean flag - fixed equality operator in `JobSpec` - added `__str__`, `__eq__` and `__hash__` methods for staging objects --- .../executors/batch/cobalt/cobalt.mustache | 2 +- .../executors/batch/common/batch_lib.mustache | 98 +++++++++---------- .../executors/batch/common/cleanup.mustache | 4 +- .../executors/batch/common/stagein.mustache | 4 +- .../executors/batch/common/stageout.mustache | 4 +- src/psij/executors/batch/lsf/lsf.mustache | 2 +- .../executors/batch/pbs/pbs_classic.mustache | 2 +- src/psij/executors/batch/pbs/pbspro.mustache | 2 +- src/psij/executors/batch/slurm/slurm.mustache | 2 +- src/psij/executors/local/local.mustache | 2 +- src/psij/job_spec.py | 13 ++- src/psij/staging.py | 51 +++++++++- tests/plugins1/_batch_test/test/test.mustache | 2 +- tests/test_staging.py | 2 +- 14 files changed, 121 insertions(+), 69 deletions(-) diff --git a/src/psij/executors/batch/cobalt/cobalt.mustache b/src/psij/executors/batch/cobalt/cobalt.mustache index 4f67721b..eb5af85f 100644 --- a/src/psij/executors/batch/cobalt/cobalt.mustache +++ b/src/psij/executors/batch/cobalt/cobalt.mustache @@ -59,7 +59,7 @@ PSIJ_NODEFILE="$COBALT_NODEFILE" export PSIJ_NODEFILE {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? diff --git a/src/psij/executors/batch/common/batch_lib.mustache b/src/psij/executors/batch/common/batch_lib.mustache index 93c18977..0afc2a16 100644 --- a/src/psij/executors/batch/common/batch_lib.mustache +++ b/src/psij/executors/batch/common/batch_lib.mustache @@ -1,4 +1,4 @@ -update_status() { +_psij_update_status() { STATUS="$1" ADDRS={{psij.us_addrs}} @@ -7,55 +7,55 @@ update_status() { done } -fail() { - [ "{{psij.debug}}" != "0" ] && update_status "LOG Failing: $2" +_psij_fail() { + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG Failing: $2" echo $2 exit $1 } -check_remote() { +_psij_check_remote() { SCHEME="$1" HOSTPORT="$2" if [ "$SCHEME" != "" ] && [ "$SCHEME" != "file" ]; then - fail 121 "$SCHEME staging is not supported" + _psij_fail 121 "$SCHEME staging is not supported" fi if [ "$HOSTPORT" != "" ] && [ "$HOSTPORT" != "localhost" ]; then - fail 121 "The host, if specified, must be \"localhost\". Got \"$HOSTPORT\"." + _psij_fail 121 "The host, if specified, must be \"localhost\". Got \"$HOSTPORT\"." fi } -do_stagein() { +_psij_do_stagein() { SOURCE="$1" TARGET="$2" MODE="$3" SCHEME="$6" HOSTPORT="$7" - check_remote "$SCHEME" "$HOSTPORT" || exit $? + _psij_check_remote "$SCHEME" "$HOSTPORT" || exit $? - do_stage "$SOURCE" "$TARGET" "$MODE" 0 + _psij_do_stage "$SOURCE" "$TARGET" "$MODE" 0 } -do_stage() { +_psij_do_stage() { SOURCE="$1" TARGET="$2" MODE="$3" MISSING_OK="$4" - [ "{{psij.debug}}" != "0" ] && update_status "LOG Stage $SOURCE -> $TARGET, mode: $MODE, missingok: $MISSING_OK" + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG Stage $SOURCE -> $TARGET, mode: $MODE, missingok: $MISSING_OK" if [ ! -e "$SOURCE" ]; then if [ "$MISSING_OK" == "0" ]; then - [ "{{psij.debug}}" != "0" ] && update_status "LOG Missing source file: $SOURCE" - fail 121 "Missing source file: $SOURCE" + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG Missing source file: $SOURCE" + _psij_fail 121 "Missing source file: $SOURCE" else - [ "{{psij.debug}}" != "0" ] && update_status "LOG Skipping staging of missing file $SOURCE" + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG Skipping staging of missing file $SOURCE" return 0 fi fi - [ "{{psij.debug}}" != "0" ] && update_status "LOG Staging $SOURCE to $TARGET" + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG Staging $SOURCE to $TARGET" TARGET_DIR=`dirname "$TARGET"` @@ -64,22 +64,22 @@ do_stage() { fi if [ -d "$TARGET" ] && [ ! -d "$SOURCE" ]; then - fail 121 "Target is a directory: $TARGET" + _psij_fail 121 "Target is a directory: $TARGET" fi if [ "$MODE" == "1" ]; then # copy - cp -r -T "$SOURCE" "$TARGET" || fail 121 "Failed to copy \"$SOURCE\" to \"$TARGET\"" + cp -r -T "$SOURCE" "$TARGET" || _psij_fail 121 "Failed to copy \"$SOURCE\" to \"$TARGET\"" elif [ "$MODE" == "2" ]; then # link {{!we want the same semantics as cp and mv, which is "overwrite if exists"}} {{!we resolve the source since it may be a path relative to the job dir}} rm -f "$TARGET" SOURCE=`readlink -m $SOURCE` - ln -s "$SOURCE" "$TARGET" || fail 121 "Failed to link \"$SOURCE\" to \"$TARGET\"" + ln -s "$SOURCE" "$TARGET" || _psij_fail 121 "Failed to link \"$SOURCE\" to \"$TARGET\"" elif [ "$MODE" == "3" ]; then # move - mv -T -f "$SOURCE" "$TARGET" || fail 121 "Failed to move \"$SOURCE\" to \"$TARGET\"" + mv -T -f "$SOURCE" "$TARGET" || _psij_fail 121 "Failed to move \"$SOURCE\" to \"$TARGET\"" fi } @@ -88,7 +88,7 @@ _FLAG_ON_SUCCESS=2 _FLAG_ON_ERROR=4 _FLAG_ON_CANCEL=8 -do_stageout() { +_psij_do_stageout() { SOURCE="$1" TARGET="$2" MODE="$3" @@ -97,44 +97,42 @@ do_stageout() { SCHEME="$6" HOSTPORT="$7" - check_remote "$SCHEME" "$HOSTPORT" + _psij_check_remote "$SCHEME" "$HOSTPORT" - [ "{{psij.debug}}" != "0" ] && update_status "LOG do_stageout $SOURCE -> $TARGET, mode: $MODE, flags: $FLAGS, failed: $FAILED" + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG do_stageout $SOURCE -> $TARGET, mode: $MODE, flags: $FLAGS, failed: $FAILED" - if [ "$FAILED" == "0" ] && [ "$((FLAGS & _FLAG_ON_SUCCESS))" != "0" ]; then - do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) - elif [ "$FAILED" != "0" ] && [ "$((FLAGS & _FLAG_ON_ERROR))" != "0" ]; then - do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) + if [ "$FAILED" == "0" ] && [ "$((FLAGS & _FLAG_ON_SUCCESS))" == "0" ]; then + return 0 fi + if [ "$FAILED" != "0" ] && [ "$((FLAGS & _FLAG_ON_ERROR))" == "0" ]; then + return 0 + fi + _psij_do_stage "$SOURCE" "$TARGET" "$MODE" $((FLAGS & _FLAG_IF_PRESENT)) } -do_cleanup() { +_psij_do_cleanup() { TARGET="$1" - FAILED="$2" - - if [ "$FAILED" == "0" ] || [ "{{job.spec.cleanup_on_failure}}" != "0" ]; then - - TARGET=`readlink -m "$TARGET"` - DIR=`readlink -m "{{job.spec.directory}}"` - - [ "{{psij.debug}}" != "0" ] && update_status "LOG Cleaning up $TARGET" + FLAGS="$2" + FAILED="$3" - case "$TARGET" in - "$DIR"*) - rm -rf "$TARGET" - ;; - *) - fail 121 "Cannot clean $TARGET outside of job directory $DIR" - ;; - esac + if [ "$FAILED" == "0" ] && [ "$((FLAGS & _FLAG_ON_SUCCESS))" == "0" ]; then + return 0 fi -} + if [ "$FAILED" != "0" ] && [ "$((FLAGS & _FLAG_ON_ERROR))" == "0" ]; then + return 0 + fi + + TARGET=`readlink -m "$TARGET"` + DIR=`readlink -m "{{job.spec.directory}}"` -stagein() { - update_status STAGE_IN + [ "{{psij.debug}}" != "0" ] && _psij_update_status "LOG Cleaning up $TARGET" -{{#job.spec.stage_in}} - do_stagein "{{source.path}}" "{{target}}" {{mode}} \ - "{{{source.scheme}}}" "{{#source.hostname}}{{{.}}}{{#source.port}}:{{{.}}}{{/source.port}}{{/source.hostname}}" -{{/job.spec.stage_in}} + case "$TARGET" in + "$DIR"*) + rm -rf "$TARGET" + ;; + *) + _psij_fail 121 "Cannot clean $TARGET outside of job directory $DIR" + ;; + esac } diff --git a/src/psij/executors/batch/common/cleanup.mustache b/src/psij/executors/batch/common/cleanup.mustache index 0f161c4a..1f4a1cf2 100644 --- a/src/psij/executors/batch/common/cleanup.mustache +++ b/src/psij/executors/batch/common/cleanup.mustache @@ -1,5 +1,5 @@ -update_status CLEANUP +_psij_update_status CLEANUP {{#job.spec.cleanup}} -do_cleanup {{.}} $_PSIJ_JOB_EC +_psij_do_cleanup {{.}} {{job.spec.cleanup_flags}} $_PSIJ_JOB_EC {{/job.spec.cleanup}} diff --git a/src/psij/executors/batch/common/stagein.mustache b/src/psij/executors/batch/common/stagein.mustache index 7cad872b..ef02b3d4 100644 --- a/src/psij/executors/batch/common/stagein.mustache +++ b/src/psij/executors/batch/common/stagein.mustache @@ -1,5 +1,5 @@ -update_status STAGE_IN +_psij_update_status STAGE_IN {{#job.spec.stage_in}} -do_stagein "{{source.path}}" "{{target}}" {{mode}} \ +_psij_do_stagein "{{source.path}}" "{{target}}" {{mode}} \ "{{{source.scheme}}}" "{{#source.hostname}}{{{.}}}{{#source.port}}:{{{.}}}{{/source.port}}{{/source.hostname}}" {{/job.spec.stage_in}} diff --git a/src/psij/executors/batch/common/stageout.mustache b/src/psij/executors/batch/common/stageout.mustache index 8d63dd64..a747e13e 100644 --- a/src/psij/executors/batch/common/stageout.mustache +++ b/src/psij/executors/batch/common/stageout.mustache @@ -1,5 +1,5 @@ -update_status STAGE_OUT +_psij_update_status STAGE_OUT {{#job.spec.stage_out}} -do_stageout "{{source}}" "{{target.path}}" {{mode}} {{flags}} $_PSIJ_JOB_EC \ +_psij_do_stageout "{{source}}" "{{target.path}}" {{mode}} {{flags}} $_PSIJ_JOB_EC \ "{{{target.scheme}}}" "{{#target.hostname}}{{{.}}}{{#target.port}}:{{{.}}}{{/target.port}}{{/target.hostname}}" {{/job.spec.stage_out}} diff --git a/src/psij/executors/batch/lsf/lsf.mustache b/src/psij/executors/batch/lsf/lsf.mustache index 24641d25..e21820c3 100644 --- a/src/psij/executors/batch/lsf/lsf.mustache +++ b/src/psij/executors/batch/lsf/lsf.mustache @@ -83,7 +83,7 @@ PSIJ_NODEFILE="$LSB_HOSTS" export PSIJ_NODEFILE {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? diff --git a/src/psij/executors/batch/pbs/pbs_classic.mustache b/src/psij/executors/batch/pbs/pbs_classic.mustache index 94ddc87a..39c80a81 100644 --- a/src/psij/executors/batch/pbs/pbs_classic.mustache +++ b/src/psij/executors/batch/pbs/pbs_classic.mustache @@ -66,7 +66,7 @@ cd "{{.}}" {{/job.spec.directory}} {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? diff --git a/src/psij/executors/batch/pbs/pbspro.mustache b/src/psij/executors/batch/pbs/pbspro.mustache index 6e9d9c6d..4bf262a1 100644 --- a/src/psij/executors/batch/pbs/pbspro.mustache +++ b/src/psij/executors/batch/pbs/pbspro.mustache @@ -69,7 +69,7 @@ cd "{{.}}" {{/job.spec.directory}} {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? diff --git a/src/psij/executors/batch/slurm/slurm.mustache b/src/psij/executors/batch/slurm/slurm.mustache index d2bad977..7cedfbe8 100644 --- a/src/psij/executors/batch/slurm/slurm.mustache +++ b/src/psij/executors/batch/slurm/slurm.mustache @@ -115,7 +115,7 @@ fi export PSIJ_NODEFILE {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE {{#psij.launch_command}}{{.}} {{/psij.launch_command}} _PSIJ_JOB_EC=$? diff --git a/src/psij/executors/local/local.mustache b/src/psij/executors/local/local.mustache index e2c0a8c2..ecd1142e 100644 --- a/src/psij/executors/local/local.mustache +++ b/src/psij/executors/local/local.mustache @@ -9,7 +9,7 @@ cd "{{.}}" {{/job.spec.directory}} {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE set +e {{#job.spec.inherit_environment}}env \{{/job.spec.inherit_environment}}{{^job.spec.inherit_environment}}env --ignore-environment \{{/job.spec.inherit_environment}}{{#env}} diff --git a/src/psij/job_spec.py b/src/psij/job_spec.py index 67ea4980..8ebdb2b5 100644 --- a/src/psij/job_spec.py +++ b/src/psij/job_spec.py @@ -9,7 +9,7 @@ import psij.resource_spec import psij.job_attributes -from psij.staging import StageIn, StageOut +from psij.staging import StageIn, StageOut, StageOutFlags def _to_path(arg: Union[str, pathlib.Path, None]) -> Optional[pathlib.Path]: @@ -70,7 +70,7 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st stage_in: Optional[Set[StageIn]] = None, stage_out: Optional[Set[StageOut]] = None, cleanup: Optional[Set[Union[str, pathlib.Path]]] = None, - cleanup_on_failure: bool = True): + cleanup_flags: StageOutFlags = StageOutFlags.ALWAYS): """ :param executable: An executable, such as "/bin/date". :param arguments: The argument list to be passed to the executable. Unlike with execve(), @@ -107,6 +107,10 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st :param stage_in: Specifies a set of files to be staged in before the job is launched. :param stage_out: Specifies a set of files to be staged out after the job terminates. :param cleanup: Specifies a set of files to remove after the stage out process. + :param cleanup_flags: Specifies the conditions under which the files in `cleanup` should + be removed, such as when the job completes successfully. The flag + `StageOutFlags.IF_PRESENT` is ignored and no error condition is triggered if a file + specified by the `cleanup` argument is not present. All constructor parameters are accessible as properties. @@ -171,7 +175,7 @@ def __init__(self, executable: Optional[str] = None, arguments: Optional[List[st self.stage_in = stage_in self.stage_out = stage_out self._cleanup = _all_to_path(cleanup) - self.cleanup_on_failure = cleanup_on_failure + self.cleanup_flags = cleanup_flags # TODO: `resources` is of type `ResourceSpec`, not `ResourceSpecV1`. An # connector trying to access `job.spec.resources.process_count` @@ -284,7 +288,8 @@ def __eq__(self, o: object) -> bool: for prop_name in ['name', 'executable', 'arguments', 'directory', 'inherit_environment', 'environment', 'stdin_path', 'stdout_path', 'stderr_path', 'resources', - 'attributes', 'pre_launch', 'post_launch', 'launcher']: + 'attributes', 'pre_launch', 'post_launch', 'launcher', 'stage_in', + 'stage_out', 'cleanup', 'cleanup_flags']: if getattr(self, prop_name) != getattr(o, prop_name): return False diff --git a/src/psij/staging.py b/src/psij/staging.py index af2fec29..4042feae 100644 --- a/src/psij/staging.py +++ b/src/psij/staging.py @@ -112,9 +112,25 @@ def password(self) -> Optional[str]: return self.parts.password def __str__(self) -> str: - """Returns a string representation of this URL.""" + """Returns a string representation of this URI.""" return self.parts.geturl() + def __eq__(self, other: object) -> bool: + """ + Tests if the parameter `other` is equal to this `URI`. + + Returns `True` if `other` is a `URI` and if it represents the same + resource as this `URI`. + """ + if isinstance(other, URI): + return self.parts == other.parts + else: + return False + + def __hash__(self) -> int: + """Computes a hash of this object.""" + return hash(self.parts) + class StagingMode(Enum): """ @@ -223,6 +239,22 @@ def __init__(self, source: Union[URI, Path, str], target: Union[str, Path], self.target = target self.mode = mode + def __str__(self) -> str: + """Returns a string representation of this object.""" + return 'StageIn[%s -> %s, %s]' % (self.source, self.target, self.mode) + + def __eq__(self, other: object) -> bool: + """Compares `other` to this object.""" + if isinstance(other, StageIn): + return (self.source == other.source and self.target == other.target + and self.mode == other.mode) + else: + return False + + def __hash__(self) -> int: + """Computes a hash of this object.""" + return (hash(self.source) << 16) + (hash(self.target) << 8) + hash(self.mode) + def _normalize_flags(flags: StageOutFlags) -> StageOutFlags: if (flags & StageOutFlags.ALWAYS).value == 0: @@ -285,3 +317,20 @@ def flags(self) -> StageOutFlags: @flags.setter def flags(self, flags: StageOutFlags) -> None: self._flags = _normalize_flags(flags) + + def __str__(self) -> str: + """Returns a string representation of this object.""" + return 'StageOut[%s -> %s, %s, %s]' % (self.source, self.target, self.flags, self.mode) + + def __eq__(self, other: object) -> bool: + """Compares `other` to this object.""" + if isinstance(other, StageOut): + return (self.source == other.source and self.target == other.target + and self.mode == other.mode and self.flags == other.flags) + else: + return False + + def __hash__(self) -> int: + """Computes a hash of this object.""" + return ((hash(self.source) << 24) + (hash(self.target) << 16) + (hash(self.mode) << 8) + + hash(self.flags)) diff --git a/tests/plugins1/_batch_test/test/test.mustache b/tests/plugins1/_batch_test/test/test.mustache index f1f2dda0..88d0b4d8 100644 --- a/tests/plugins1/_batch_test/test/test.mustache +++ b/tests/plugins1/_batch_test/test/test.mustache @@ -45,7 +45,7 @@ done export PSIJ_NODEFILE {{> stagein}} -update_status ACTIVE +_psij_update_status ACTIVE {{#job.spec.inherit_environment}}env \{{/job.spec.inherit_environment}}{{^job.spec.inherit_environment}}env --ignore-environment \{{/job.spec.inherit_environment}}{{#env}} {{name}}="{{value}}" \{{/env}} diff --git a/tests/test_staging.py b/tests/test_staging.py index d381612d..76a16da4 100644 --- a/tests/test_staging.py +++ b/tests/test_staging.py @@ -217,7 +217,7 @@ def test_cleanup2(execparams: ExecutorTestParams) -> None: StageOut('out.txt', out_path, flags=StageOutFlags.IF_PRESENT), } job.spec.cleanup = {Path('out.txt')} - job.spec.cleanup_on_failure = False + job.spec.cleanup_flags = StageOutFlags.ON_SUCCESS ex = _get_executor_instance(execparams, job) ex.submit(job) status = job.wait(timeout=_get_timeout(execparams)) From ea0f32f9ba6cc2bc51d7a5b27ebe5d9557df11a9 Mon Sep 17 00:00:00 2001 From: hategan Date: Fri, 14 Feb 2025 14:57:31 -0800 Subject: [PATCH 33/39] Uninstall psij when installing depdendencies, since psij is provided by the current package. --- .github/workflows/python-package.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 135625c3..4494c35d 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -25,6 +25,9 @@ jobs: pip install -r requirements.txt pip install -r requirements-dev.txt pip install -r requirements-connector-radical.txt + # Radical depends on PSI/J. However, the version we are + # testing provide that dependency + pip uninstall -y psij-python - name: Typecheck and stylecheck run: | make typecheck From 17246f92a4716dbd046bb9f6379c5ba1f2a3145c Mon Sep 17 00:00:00 2001 From: hategan Date: Fri, 14 Feb 2025 15:07:40 -0800 Subject: [PATCH 34/39] Also uninstall psij when building docs --- .readthedocs.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.readthedocs.yaml b/.readthedocs.yaml index 6415f34f..422cc7af 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -9,6 +9,9 @@ build: os: ubuntu-22.04 tools: python: "3.11" + jobs: + pre_create_environment: + - pip uninstall -y psij-python python: install: From 2600db474be651bddbc2db11e256ea9577677688 Mon Sep 17 00:00:00 2001 From: hategan Date: Fri, 14 Feb 2025 15:14:10 -0800 Subject: [PATCH 35/39] Or maybe it's post_build --- .readthedocs.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.readthedocs.yaml b/.readthedocs.yaml index 422cc7af..38cee527 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -10,7 +10,7 @@ build: tools: python: "3.11" jobs: - pre_create_environment: + post_build: - pip uninstall -y psij-python python: From 0eaa937acb8fcfa46916e9353949e8f40e64df30 Mon Sep 17 00:00:00 2001 From: hategan Date: Fri, 14 Feb 2025 15:22:06 -0800 Subject: [PATCH 36/39] post_install? --- .readthedocs.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.readthedocs.yaml b/.readthedocs.yaml index 38cee527..85674529 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -10,7 +10,7 @@ build: tools: python: "3.11" jobs: - post_build: + post_install: - pip uninstall -y psij-python python: From d20fd45cfe04ec7b98af9dc6d9718fa9e6dfd90b Mon Sep 17 00:00:00 2001 From: hategan Date: Tue, 18 Feb 2025 23:58:12 -0800 Subject: [PATCH 37/39] When using the local executor, there is no need to send UDP packet on various interfaces when the loopback will suffice --- src/psij/executors/local.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/psij/executors/local.py b/src/psij/executors/local.py index 31cace23..c9756d33 100644 --- a/src/psij/executors/local.py +++ b/src/psij/executors/local.py @@ -284,7 +284,7 @@ def submit(self, job: Job) -> None: 'script_dir': str(self._work_dir), 'us_file': self._status_updater.update_file_name, 'us_port': self._status_updater.update_port, - 'us_addrs': ', '.join(self._status_updater.ips), + 'us_addrs': '127.0.0.1', 'debug': logger.isEnabledFor(logging.DEBUG), 'nodefile': nodefile } From aa7c1f4ade65b8f79351a12f49ab41c1df3fb448 Mon Sep 17 00:00:00 2001 From: hategan Date: Wed, 19 Feb 2025 14:38:44 -0800 Subject: [PATCH 38/39] Local executor was not cleaning its files. Also, add a mechanism to clean old submit scripts to avoid gazillions of small files in ~/.psij/work. --- .../batch/batch_scheduler_executor.py | 3 +- src/psij/executors/local.py | 12 +++++- src/psij/utils.py | 39 +++++++++++++++++++ 3 files changed, 52 insertions(+), 2 deletions(-) diff --git a/src/psij/executors/batch/batch_scheduler_executor.py b/src/psij/executors/batch/batch_scheduler_executor.py index a0b82ba7..a952234f 100644 --- a/src/psij/executors/batch/batch_scheduler_executor.py +++ b/src/psij/executors/batch/batch_scheduler_executor.py @@ -16,7 +16,7 @@ from psij import JobExecutor, JobExecutorConfig, Launcher, Job, SubmitException, \ JobStatus, JobState from psij.executors.batch.template_function_library import ALL as FUNCTION_LIBRARY -from psij.utils import _StatusUpdater +from psij.utils import _StatusUpdater, _FileCleaner UNKNOWN_ERROR = 'PSIJ: Unknown error' @@ -206,6 +206,7 @@ def __init__(self, url: Optional[str] = None, assert config self.work_directory = config.work_directory / self.name self.work_directory.mkdir(parents=True, exist_ok=True) + cast(_FileCleaner, _FileCleaner.get_instance()).clean(self.work_directory) def submit(self, job: Job) -> None: """See :func:`~psij.JobExecutor.submit`.""" diff --git a/src/psij/executors/local.py b/src/psij/executors/local.py index c9756d33..976217e1 100644 --- a/src/psij/executors/local.py +++ b/src/psij/executors/local.py @@ -19,7 +19,7 @@ from psij import JobExecutor from psij.executors.batch.batch_scheduler_executor import _env_to_mustache from psij.executors.batch.script_generator import TemplatedScriptGenerator -from psij.utils import SingletonThread, _StatusUpdater +from psij.utils import SingletonThread, _StatusUpdater, _FileCleaner from psij.executors.batch.template_function_library import ALL as FUNCTION_LIBRARY @@ -232,6 +232,7 @@ def __init__(self, url: Optional[str] = None, self._reaper = _ProcessReaper.get_instance() self._work_dir = Path.home() / '.psij' / 'work' / 'local' self._work_dir.mkdir(parents=True, exist_ok=True) + cast(_FileCleaner, _FileCleaner.get_instance()).clean(self._work_dir) self._status_updater = cast(_StatusUpdater, _StatusUpdater.get_instance()) self.generator = TemplatedScriptGenerator(config, Path(__file__).parent / 'local' / 'local.mustache') @@ -339,6 +340,8 @@ def _process_done(self, p: _ProcessEntry) -> None: message = p.out state = JobState.FAILED + if state.final: + self._clean_submit_file(p.job) # We need to ensure that the status updater has processed all updates that # have been sent up to this point self._status_updater.flush() @@ -346,6 +349,13 @@ def _process_done(self, p: _ProcessEntry) -> None: self._set_job_status(p.job, JobStatus(state, time=p.done_time, exit_code=p.exit_code, message=message)) + def _clean_submit_file(self, job: Job) -> None: + submit_file_path = self._work_dir / (job.id + '.job') + try: + submit_file_path.unlink() + except FileNotFoundError: + pass + def list(self) -> List[str]: """ Return a list of ids representing jobs that are running on the underlying implementation. diff --git a/src/psij/utils.py b/src/psij/utils.py index 970ca6c5..47357b35 100644 --- a/src/psij/utils.py +++ b/src/psij/utils.py @@ -2,11 +2,13 @@ import io import logging import os +import queue import random import socket import tempfile import threading import time +from datetime import datetime, timedelta from pathlib import Path from typing import Type, Dict, Optional, Tuple, Set, List @@ -17,6 +19,9 @@ logger = logging.getLogger(__name__) +_MAX_FILE_AGE_DAYS = 30 + + class SingletonThread(threading.Thread): """ A convenience class to return a thread that is guaranteed to be unique to this process. @@ -207,3 +212,37 @@ def _process_update_data(self, data: bytes) -> None: pass if job: executor._set_job_status(job, JobStatus(state)) + + +class _FileCleaner(SingletonThread): + def __init__(self) -> None: + super().__init__() + self.name = 'File Cleaner' + self.daemon = True + self.queue: queue.SimpleQueue[Path] = queue.SimpleQueue() + + def clean(self, path: Path) -> None: + self.queue.put(path) + + def run(self) -> None: + while True: + try: + path = self.queue.get(block=True, timeout=1) + try: + self._do_clean(path) + except Exception as ex: + print(f'Warning: cannot clean {path}: {ex}') + except queue.Empty: + pass + + def _do_clean(self, path: Path) -> None: + now = datetime.now() + max_age = timedelta(days=_MAX_FILE_AGE_DAYS) + for child in path.iterdir(): + m_time = datetime.fromtimestamp(child.lstat().st_mtime) + if now - m_time > max_age: + try: + child.unlink() + except FileNotFoundError: + # we try our best + pass From b3642f367233bfc0bcc23738dc3620eb768d9aca Mon Sep 17 00:00:00 2001 From: hategan Date: Wed, 2 Jul 2025 15:14:23 -0700 Subject: [PATCH 39/39] Attempt to add file staging to NQSV. --- src/psij/executors/batch/nqsv/nqsv.mustache | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/psij/executors/batch/nqsv/nqsv.mustache b/src/psij/executors/batch/nqsv/nqsv.mustache index a6184020..99031c47 100644 --- a/src/psij/executors/batch/nqsv/nqsv.mustache +++ b/src/psij/executors/batch/nqsv/nqsv.mustache @@ -72,13 +72,21 @@ do done < $PBS_NODEFILE export PSIJ_NODEFILE +{{> batch_lib}} + +{{> stagein}} +_psij_update_status ACTIVE + {{#psij.launch_command}}{{.}} {{/psij.launch_command}} -E=$? +_PSIJ_JOB_EC=$? + +{{> stageout}} +{{> cleanup}} {{!we redirect to a file tied to the native ID so that we can reach the file with attach().}} if [ "$J" = "0" ]; then - echo "$E" > "{{psij.script_dir}}/$ID.ec" + echo "$_PSIJ_JOB_EC" > "{{psij.script_dir}}/$ID.ec" fi exit $E