From 7c56b1d4697cef5740949718487bb4c07b46f772 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 18 May 2018 15:35:40 -0700 Subject: [PATCH 01/59] add job ids to dask workers --- dask_jobqueue/core.py | 6 ++++-- dask_jobqueue/pbs.py | 3 ++- dask_jobqueue/sge.py | 3 +-- dask_jobqueue/slurm.py | 3 ++- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index d2e6a53d..40d8bf45 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -133,8 +133,10 @@ def __init__(self, if memory is not None: self._command_template += " --memory-limit %s" % memory if name is not None: - self._command_template += " --name %s" % name - self._command_template += "-%(n)d" # Keep %(n) to be replaced later + self._command_template += " --name %s" % name # e.g. "dask-worker" + # Keep %(n) to be replaced later (worker id on this job) + # ${JOB_ID} is an environment variable describing this job + self._command_template += "${JOB_ID}-%(n)d" if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: diff --git a/dask_jobqueue/pbs.py b/dask_jobqueue/pbs.py index 350547ad..4d57bd4f 100644 --- a/dask_jobqueue/pbs.py +++ b/dask_jobqueue/pbs.py @@ -73,7 +73,7 @@ def __init__(self, # Try to find a project name from environment variable project = project or os.environ.get('PBS_ACCOUNT') - header_lines = [] + header_lines = ['#!/usr/bin/env bash'] # PBS header build if self.name is not None: header_lines.append('#PBS -N %s' % self.name) @@ -93,6 +93,7 @@ def __init__(self, if walltime is not None: header_lines.append('#PBS -l walltime=%s' % walltime) header_lines.extend(['#PBS %s' % arg for arg in job_extra]) + header_lines.append('JOB_ID=${PBS_JOBID}') # Declare class attribute that shall be overriden self.job_header = '\n'.join(header_lines) diff --git a/dask_jobqueue/sge.py b/dask_jobqueue/sge.py index e6a2a841..7c3817f7 100644 --- a/dask_jobqueue/sge.py +++ b/dask_jobqueue/sge.py @@ -52,8 +52,7 @@ def __init__(self, super(SGECluster, self).__init__(**kwargs) - header_lines = ['#!/bin/bash'] - + header_lines = ['#!/usr/bin/env bash'] if self.name is not None: header_lines.append('#$ -N %(name)s') if queue is not None: diff --git a/dask_jobqueue/slurm.py b/dask_jobqueue/slurm.py index d4878315..01b7dafe 100644 --- a/dask_jobqueue/slurm.py +++ b/dask_jobqueue/slurm.py @@ -71,7 +71,7 @@ def __init__(self, super(SLURMCluster, self).__init__(**kwargs) # Always ask for only one task - header_lines = [] + header_lines = ['#!/usr/bin/env bash'] # SLURM header build if self.name is not None: header_lines.append('#SBATCH -J %s' % self.name) @@ -99,6 +99,7 @@ def __init__(self, if walltime is not None: header_lines.append('#SBATCH -t %s' % walltime) + header_lines.append('JOB_ID=${SLURM_JOB_ID}') header_lines.extend(['#SBATCH %s' % arg for arg in job_extra]) # Declare class attribute that shall be overriden From 6b35688dd89aba3b0f45d7f9204d29346d31adf0 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 18 May 2018 16:39:29 -0600 Subject: [PATCH 02/59] pad job id --- dask_jobqueue/core.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 40d8bf45..3836142a 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -136,7 +136,7 @@ def __init__(self, self._command_template += " --name %s" % name # e.g. "dask-worker" # Keep %(n) to be replaced later (worker id on this job) # ${JOB_ID} is an environment variable describing this job - self._command_template += "${JOB_ID}-%(n)d" + self._command_template += "-${JOB_ID}-%(n)d" if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: From 507be825d0d279fa4432b86da7afbe3151889fd7 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 18 May 2018 16:03:10 -0700 Subject: [PATCH 03/59] parse PBS/slurm job ids --- dask_jobqueue/pbs.py | 2 +- dask_jobqueue/slurm.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dask_jobqueue/pbs.py b/dask_jobqueue/pbs.py index 4d57bd4f..3475d74a 100644 --- a/dask_jobqueue/pbs.py +++ b/dask_jobqueue/pbs.py @@ -93,7 +93,7 @@ def __init__(self, if walltime is not None: header_lines.append('#PBS -l walltime=%s' % walltime) header_lines.extend(['#PBS %s' % arg for arg in job_extra]) - header_lines.append('JOB_ID=${PBS_JOBID}') + header_lines.append('JOB_ID=${PBS_JOBID%.*}') # Declare class attribute that shall be overriden self.job_header = '\n'.join(header_lines) diff --git a/dask_jobqueue/slurm.py b/dask_jobqueue/slurm.py index 01b7dafe..73a012bd 100644 --- a/dask_jobqueue/slurm.py +++ b/dask_jobqueue/slurm.py @@ -99,7 +99,7 @@ def __init__(self, if walltime is not None: header_lines.append('#SBATCH -t %s' % walltime) - header_lines.append('JOB_ID=${SLURM_JOB_ID}') + header_lines.append('JOB_ID=${SLURM_JOB_ID%;*}') header_lines.extend(['#SBATCH %s' % arg for arg in job_extra]) # Declare class attribute that shall be overriden From 99d0f1f997164e5bc774b5f07852f7054d14a94a Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Mon, 21 May 2018 08:55:35 -0700 Subject: [PATCH 04/59] track workers individually (sort of) --- dask_jobqueue/core.py | 26 +++++++++++++++++--------- dask_jobqueue/sge.py | 2 ++ 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 3836142a..648de124 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -1,4 +1,7 @@ +from __future__ import absolute_import, division, print_function + import logging +import math import os import shlex import socket @@ -133,6 +136,7 @@ def __init__(self, if memory is not None: self._command_template += " --memory-limit %s" % memory if name is not None: + # worker names follow this template: {NAME}-{JOB_ID}-{WORKER_NUM} self._command_template += " --name %s" % name # e.g. "dask-worker" # Keep %(n) to be replaced later (worker id on this job) # ${JOB_ID} is an environment variable describing this job @@ -163,7 +167,8 @@ def job_file(self): def start_workers(self, n=1): """ Start workers and point them to our local scheduler """ workers = [] - for _ in range(n): + num_jobs = min(1, math.ceil(n / self.worker_processes)) + for _ in range(num_jobs): with self.job_file() as fn: out = self._call(shlex.split(self.submit_command) + [fn]) job = self._job_id_from_submit_output(out.decode()) @@ -198,12 +203,12 @@ def _calls(self, cmds): Also logs any stderr information """ logger.debug("Submitting the following calls to command line") + procs = [] for cmd in cmds: logger.debug(' '.join(cmd)) - procs = [subprocess.Popen(cmd, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE) - for cmd in cmds] + procs.append(subprocess.Popen(cmd, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE)) result = [] for proc in procs: @@ -234,10 +239,13 @@ def scale_up(self, n, **kwargs): def scale_down(self, workers): ''' Close the workers with the given addresses ''' - if isinstance(workers, dict): - names = {v['name'] for v in workers.values()} - job_ids = {name.split('-')[-2] for name in names} - self.stop_workers(job_ids) + if not isinstance(workers, dict): + raise ValueError( + 'Expected dictionary of workers, got %s' % type(workers)) + names = {v['name'] for v in workers.values()} + # This will close down the full group of workers + job_ids = {name.split('-')[-2] for name in names} + self.stop_workers(job_ids) def __enter__(self): return self diff --git a/dask_jobqueue/sge.py b/dask_jobqueue/sge.py index 7c3817f7..938fba3a 100644 --- a/dask_jobqueue/sge.py +++ b/dask_jobqueue/sge.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import, division, print_function + import logging import dask From 78a22ff9a26e0ae5d2775c8bd8e5cad94b7bdc09 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Mon, 21 May 2018 10:51:29 -0700 Subject: [PATCH 05/59] add _adaptive_options --- dask_jobqueue/core.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 648de124..b0feab87 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -80,6 +80,7 @@ class JobQueueCluster(Cluster): # Following class attributes should be overriden by extending classes. submit_command = None cancel_command = None + _adaptive_options = {'worker_key': lambda ws: ws.name.name.split('-')[-2]} def __init__(self, name=dask.config.get('jobqueue.name'), From 62e050c6401af514bf6f9f9fcd0336f3c17fb5b2 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 22 May 2018 11:17:16 -0700 Subject: [PATCH 06/59] generalize the parsing of the job id --- dask_jobqueue/core.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index b0feab87..8630c4a3 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -80,7 +80,8 @@ class JobQueueCluster(Cluster): # Following class attributes should be overriden by extending classes. submit_command = None cancel_command = None - _adaptive_options = {'worker_key': lambda ws: ws.name.name.split('-')[-2]} + _adaptive_options = { + 'worker_key': lambda ws: _job_id_from_worker_name(ws.name)} def __init__(self, name=dask.config.get('jobqueue.name'), @@ -245,7 +246,7 @@ def scale_down(self, workers): 'Expected dictionary of workers, got %s' % type(workers)) names = {v['name'] for v in workers.values()} # This will close down the full group of workers - job_ids = {name.split('-')[-2] for name in names} + job_ids = {_job_id_from_worker_name(name) for name in names} self.stop_workers(job_ids) def __enter__(self): @@ -260,3 +261,8 @@ def _job_id_from_submit_output(self, out): 'implemented when JobQueueCluster is ' 'inherited. It should convert the stdout ' 'from submit_command to the job id') + + +def _job_id_from_worker_name(name): + ''' utility to parse the job ID from the worker name''' + return name.name.split('-')[-2] From d121180a32e5c4716b266615a2f464d33746f82c Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 22 May 2018 11:29:40 -0700 Subject: [PATCH 07/59] fix typo --- dask_jobqueue/core.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 8630c4a3..60534e26 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -265,4 +265,4 @@ def _job_id_from_submit_output(self, out): def _job_id_from_worker_name(name): ''' utility to parse the job ID from the worker name''' - return name.name.split('-')[-2] + return name.split('-')[-2] From 2329bfea326046bd15c61b7bacaabd1f163a8f9b Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 22 May 2018 14:12:41 -0600 Subject: [PATCH 08/59] changes for review --- dask_jobqueue/core.py | 37 ++++++++++++++++--------------------- 1 file changed, 16 insertions(+), 21 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 60534e26..de903f51 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -104,7 +104,7 @@ def __init__(self, raise NotImplementedError('JobQueueCluster is an abstract class ' 'that should not be instanciated.') - #This attribute should be overriden + # This attribute should be overriden self.job_header = None if interface: @@ -122,7 +122,6 @@ def __init__(self, self.worker_threads = threads self.name = name - self.jobs = dict() self.n = 0 self._adaptive = None @@ -168,15 +167,14 @@ def job_file(self): def start_workers(self, n=1): """ Start workers and point them to our local scheduler """ - workers = [] - num_jobs = min(1, math.ceil(n / self.worker_processes)) + job_ids = [] + num_jobs = math.ceil(n / self.worker_processes) for _ in range(num_jobs): with self.job_file() as fn: out = self._call(shlex.split(self.submit_command) + [fn]) job = self._job_id_from_submit_output(out.decode()) - self.jobs[self.n] = job - workers.append(self.n) - return workers + job_ids.append(job) + return job_ids @property def scheduler(self): @@ -228,32 +226,29 @@ def stop_workers(self, workers): """ Stop a list of workers""" if not workers: return - workers = list(map(int, workers)) - jobs = [self.jobs[w] for w in workers] + jobs = {_job_id_from_worker_name(w.name) for w in workers} self._call([self.cancel_command] + list(jobs)) - for w in workers: - with ignoring(KeyError): - del self.jobs[w] def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ - return self.start_workers(n - len(self.jobs)) + return self.start_workers(n - len(self.scheduler.workers)) def scale_down(self, workers): ''' Close the workers with the given addresses ''' - if not isinstance(workers, dict): - raise ValueError( - 'Expected dictionary of workers, got %s' % type(workers)) - names = {v['name'] for v in workers.values()} - # This will close down the full group of workers - job_ids = {_job_id_from_worker_name(name) for name in names} - self.stop_workers(job_ids) + workers = [] + for w in workers: + try: + # Get the actual "Worker" + workers.append(self.scheduler.workers[w]) + except KeyError: + logger.debug('worker %s is already gone' % w) + self.stop_workers(workers) def __enter__(self): return self def __exit__(self, type, value, traceback): - self.stop_workers(self.jobs) + self.stop_workers(self.scheduler.workers) self.cluster.__exit__(type, value, traceback) def _job_id_from_submit_output(self, out): From 92eaf4ed6242658e56f1281643c93cf89163dfe2 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Wed, 23 May 2018 23:14:52 -0700 Subject: [PATCH 09/59] add pluggin (untested) --- dask_jobqueue/core.py | 79 +++++++++++++++++++++++++++++++++---------- 1 file changed, 62 insertions(+), 17 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index de903f51..476d6b00 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -7,13 +7,15 @@ import socket import subprocess import sys +from collections import OrderedDict from contextlib import contextmanager import dask import docrep from distributed import LocalCluster from distributed.deploy import Cluster -from distributed.utils import get_ip_interface, ignoring, parse_bytes, tmpfile +from distributed.utils import get_ip_interface, parse_bytes, tmpfile +from distributed.diagnostics.plugin import SchedulerPlugin dirname = os.path.dirname(sys.executable) @@ -21,6 +23,47 @@ docstrings = docrep.DocstringProcessor() +def _job_id_from_worker_name(name): + ''' utility to parse the job ID from the worker name''' + return name.split('-')[-1] + + +class Job(object): + def __init__(self, job_id, workers=None, status=None): + self.job_id = job_id + self.status = status + if workers is None: + workers = [] + self.workers = workers + + def update(self, workers=None, status=None): + if workers is not None: + self.workers.extend(workers) + if status is not None: + self.status = status + + def __repr__(self): + return "<%s: %r, status: %r, workers: %r>" % ( + self.__class__.__name__, self.job_id, self.status, self.workers) + + +class JobQueuePlugin(SchedulerPlugin): + def __init__(self): + self.pending_jobs = OrderedDict() + self.running_jobs = OrderedDict() + self.finished_jobs = OrderedDict() + + def add_worker(self, scheduler, worker=None, name=None, **kwargs): + job_id = _job_id_from_worker_name(worker.name) + self.running_jobs[job_id] = self.pending_jobs.pop(job_id) + self.running_jobs[job_id].update(workers=[worker], status='running') + + def remove_worker(self, scheduler=None, worker=None, **kwargs): + job_id = _job_id_from_worker_name(worker.name) + self.finished_jobs[job_id] = self.running_jobs.pop(job_id) + self.finished_jobs[job_id].update(status='finished') + + @docstrings.get_sectionsf('JobQueueCluster') class JobQueueCluster(Cluster): """ Base class to launch Dask Clusters for Job queues @@ -115,6 +158,16 @@ def __init__(self, self.cluster = LocalCluster(n_workers=0, ip=host, **kwargs) + # plugin for tracking job status + self._scheduler_plugin = JobQueuePlugin() + self.cluster.scheduler.add_plugin(self._scheduler_plugin) + self.pending_jobs = self._scheduler_plugin.pending_jobs + self.running_jobs = self._scheduler_plugin.running_jobs + self.finished_jobs = self._scheduler_plugin.finished_jobs + + # counter to keep track of how many jobs have been submitted + self._n = 0 + # Keep information on process, threads and memory, for use in # subclasses self.worker_memory = parse_bytes(memory) @@ -122,7 +175,6 @@ def __init__(self, self.worker_threads = threads self.name = name - self.n = 0 self._adaptive = None self._env_header = '\n'.join(env_extra) @@ -137,11 +189,9 @@ def __init__(self, if memory is not None: self._command_template += " --memory-limit %s" % memory if name is not None: - # worker names follow this template: {NAME}-{JOB_ID}-{WORKER_NUM} + # worker names follow this template: {NAME}-{JOB_ID} self._command_template += " --name %s" % name # e.g. "dask-worker" - # Keep %(n) to be replaced later (worker id on this job) - # ${JOB_ID} is an environment variable describing this job - self._command_template += "-${JOB_ID}-%(n)d" + self._command_template += "-${JOB_ID}" if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: @@ -151,8 +201,8 @@ def __init__(self, def job_script(self): """ Construct a job submission script """ - self.n += 1 - template = self._command_template % {'n': self.n} + self._n += 1 + template = self._command_template % {'n': self._n} return self._script_template % {'job_header': self.job_header, 'env_header': self._env_header, 'worker_command': template} @@ -167,14 +217,12 @@ def job_file(self): def start_workers(self, n=1): """ Start workers and point them to our local scheduler """ - job_ids = [] num_jobs = math.ceil(n / self.worker_processes) for _ in range(num_jobs): with self.job_file() as fn: out = self._call(shlex.split(self.submit_command) + [fn]) job = self._job_id_from_submit_output(out.decode()) - job_ids.append(job) - return job_ids + self.pending_jobs[job] = Job(job, status='pending') @property def scheduler(self): @@ -231,7 +279,9 @@ def stop_workers(self, workers): def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ - return self.start_workers(n - len(self.scheduler.workers)) + pending_workers = self.worker_processes * len(self.pending_jobs) + active_and_pending = len(self.scheduler.workers) + pending_workers + return self.start_workers(n - active_and_pending) def scale_down(self, workers): ''' Close the workers with the given addresses ''' @@ -256,8 +306,3 @@ def _job_id_from_submit_output(self, out): 'implemented when JobQueueCluster is ' 'inherited. It should convert the stdout ' 'from submit_command to the job id') - - -def _job_id_from_worker_name(name): - ''' utility to parse the job ID from the worker name''' - return name.split('-')[-2] From 9084a35929bbc3e8f58804f9cad0f691c89a5a59 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Thu, 24 May 2018 13:28:57 -0700 Subject: [PATCH 10/59] a few fixes + tests --- dask_jobqueue/core.py | 16 +++--- dask_jobqueue/tests/test_jobqueue_core.py | 22 ++++++-- dask_jobqueue/tests/test_pbs.py | 64 +++++++++++++++++++---- dask_jobqueue/tests/test_slurm.py | 45 ++++++++++------ 4 files changed, 110 insertions(+), 37 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 476d6b00..e6713431 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -36,9 +36,9 @@ def __init__(self, job_id, workers=None, status=None): workers = [] self.workers = workers - def update(self, workers=None, status=None): - if workers is not None: - self.workers.extend(workers) + def update(self, worker=None, status=None): + if worker is not None: + self.workers.append(worker) if status is not None: self.status = status @@ -55,13 +55,17 @@ def __init__(self): def add_worker(self, scheduler, worker=None, name=None, **kwargs): job_id = _job_id_from_worker_name(worker.name) - self.running_jobs[job_id] = self.pending_jobs.pop(job_id) - self.running_jobs[job_id].update(workers=[worker], status='running') + if job_id not in self.running_jobs: + self.running_jobs[job_id] = self.pending_jobs.pop(job_id) + self.running_jobs[job_id].update(worker=worker, status='running') def remove_worker(self, scheduler=None, worker=None, **kwargs): job_id = _job_id_from_worker_name(worker.name) - self.finished_jobs[job_id] = self.running_jobs.pop(job_id) + if job_id not in self.finished_jobs: + self.finished_jobs[job_id] = self.running_jobs.pop(job_id) self.finished_jobs[job_id].update(status='finished') + if self.finished_jobs[job_id].workers: + self.finished_jobs[job_id].workers = [] @docstrings.get_sectionsf('JobQueueCluster') diff --git a/dask_jobqueue/tests/test_jobqueue_core.py b/dask_jobqueue/tests/test_jobqueue_core.py index 3f9a8a85..411ec5e8 100644 --- a/dask_jobqueue/tests/test_jobqueue_core.py +++ b/dask_jobqueue/tests/test_jobqueue_core.py @@ -1,11 +1,7 @@ import pytest from dask_jobqueue import JobQueueCluster - - -def test_jq_core_placeholder(): - # to test that CI is working - pass +from dask_jobqueue.core import Job def test_errors(): @@ -13,3 +9,19 @@ def test_errors(): JobQueueCluster() assert 'abstract class' in str(info.value) + + +def test_job_update(): + job = Job(job_id='1234', status='pending') + assert job.job_id == '1234' + assert job.status == 'pending' + assert len(job.workers) == 0 + + job.update(status='running') + assert job.status == 'running' + assert len(job.workers) == 0 + + job.update(worker='worker1') + job.update(worker='worker2') + assert job.status == 'running' + assert len(job.workers) == 2 diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 39e810e0..76156071 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -102,14 +102,62 @@ def test_basic(loop): @pytest.mark.env("pbs") # noqa: F811 def test_adaptive(loop): - with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', local_directory='/tmp', - job_extra=['-V'], loop=loop) as cluster: + with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', + local_directory='/tmp', ob_extra=['-V'], + loop=loop) as cluster: cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) assert future.result(60) == 11 - assert cluster.jobs + start = time() + while not len(cluster.pending_jobs): + sleep(0.100) + assert time() < start + 10 + + start = time() + while not len(cluster.running_jobs): + sleep(0.100) + assert time() < start + 10 + + start = time() + processes = cluster.worker_processes + while len(client.scheduler_info()['workers']) != processes: + sleep(0.1) + assert time() < start + 10 + + del future + + start = time() + while len(client.scheduler_info()['workers']) > 0: + sleep(0.100) + assert time() < start + 10 + + start = time() + while cluster.pending_jobs or cluster.running_jobs: + sleep(0.100) + assert time() < start + 10 + + +@pytest.mark.env("pbs") # noqa: F811 +def test_adaptive_grouped(loop): + with PBSCluster(walltime='00:02:00', processes=2, threads=1, memory='2GB', + local_directory='/tmp', ob_extra=['-V'], + loop=loop) as cluster: + cluster.adapt(minimum=1) + with Client(cluster) as client: + future = client.submit(lambda x: x + 1, 10) + assert future.result(60) == 11 + + start = time() + while not len(cluster.pending_jobs): + sleep(0.100) + assert time() < start + 10 + + start = time() + while not len(cluster.running_jobs): + sleep(0.100) + assert time() < start + 10 start = time() processes = cluster.worker_processes @@ -124,9 +172,7 @@ def test_adaptive(loop): sleep(0.100) assert time() < start + 10 - # There is probably a bug to fix in the adaptive methods of the JobQueueCluster - # Currently cluster.jobs is not cleaned up. - #start = time() - #while cluster.jobs: - # sleep(0.100) - # assert time() < start + 10 + start = time() + while cluster.pending_jobs or cluster.running_jobs: + sleep(0.100) + assert time() < start + 10 diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index c2bba71c..12599acf 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -8,7 +8,8 @@ def test_header(): - with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: + with SLURMCluster(walltime='00:02:00', processes=4, threads=2, + memory='7GB') as cluster: assert '#SBATCH' in cluster.job_header assert '#SBATCH -J dask-worker' in cluster.job_header @@ -19,7 +20,8 @@ def test_header(): assert '#SBATCH -p' not in cluster.job_header assert '#SBATCH -A' not in cluster.job_header - with SLURMCluster(queue='regular', project='DaskOnPBS', processes=4, threads=2, memory='7GB', + with SLURMCluster(queue='regular', project='DaskOnPBS', processes=4, + threads=2, memory='7GB', job_cpu=16, job_mem='100G') as cluster: assert '#SBATCH --cpus-per-task=16' in cluster.job_header @@ -42,7 +44,8 @@ def test_header(): def test_job_script(): - with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: + with SLURMCluster(walltime='00:02:00', processes=4, threads=2, + memory='7GB') as cluster: job_script = cluster.job_script() assert '#SBATCH' in job_script @@ -59,8 +62,10 @@ def test_job_script(): assert '/dask-worker tcp://' in job_script assert '--nthreads 2 --nprocs 4 --memory-limit 7GB' in job_script - with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB', - env_extra=['export LANG="en_US.utf8"', 'export LANGUAGE="en_US.utf8"', + with SLURMCluster(walltime='00:02:00', processes=4, threads=2, + memory='7GB', + env_extra=['export LANG="en_US.utf8"', + 'export LANGUAGE="en_US.utf8"', 'export LC_ALL="en_US.utf8"'] ) as cluster: job_script = cluster.job_script() @@ -83,8 +88,8 @@ def test_job_script(): @pytest.mark.env("slurm") # noqa: F811 def test_basic(loop): - with SLURMCluster(walltime='00:02:00', threads=2, processes=1, memory='4GB', - job_extra=['-D /'], loop=loop) as cluster: + with SLURMCluster(walltime='00:02:00', threads=2, processes=1, + memory='4GB', job_extra=['-D /'], loop=loop) as cluster: with Client(cluster) as client: workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) @@ -108,18 +113,26 @@ def test_basic(loop): @pytest.mark.env("slurm") # noqa: F811 def test_adaptive(loop): - with SLURMCluster(walltime='00:02:00', threads=2, processes=1, memory='4GB', - job_extra=['-D /'], loop=loop) as cluster: + with SLURMCluster(walltime='00:02:00', threads=2, processes=1, + memory='4GB', job_extra=['-D /'], loop=loop) as cluster: cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) assert future.result(60) == 11 - assert cluster.jobs + start = time() + while not len(cluster.pending_jobs): + sleep(0.100) + assert time() < start + 10 + + start = time() + while not len(cluster.running_jobs): + sleep(0.100) + assert time() < start + 10 start = time() processes = cluster.worker_processes - while (len(client.scheduler_info()['workers']) != processes): + while len(client.scheduler_info()['workers']) != processes: sleep(0.1) assert time() < start + 10 @@ -130,9 +143,7 @@ def test_adaptive(loop): sleep(0.100) assert time() < start + 10 - # There is probably a bug to fix in the adaptive methods of the JobQueueCluster - # Currently cluster.jobs is not cleaned up. - # start = time() - # while cluster.jobs: - # sleep(0.100) - # assert time() < start + 10 + start = time() + while cluster.pending_jobs or cluster.running_jobs: + sleep(0.100) + assert time() < start + 10 From 47768924161dc83bedefe1c5cc1597d5416d0860 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Mon, 28 May 2018 22:15:23 -0700 Subject: [PATCH 11/59] respond to first round of comments --- dask_jobqueue/core.py | 22 +++++++++++----------- dask_jobqueue/tests/test_pbs.py | 4 ++-- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index e6713431..1c35a2a1 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -61,11 +61,11 @@ def add_worker(self, scheduler, worker=None, name=None, **kwargs): def remove_worker(self, scheduler=None, worker=None, **kwargs): job_id = _job_id_from_worker_name(worker.name) - if job_id not in self.finished_jobs: + if self.running_jobs[job_id].workers: + self.running_jobs[job_id].workers.remove(worker) + if not self.running_jobs[job_id].workers: self.finished_jobs[job_id] = self.running_jobs.pop(job_id) - self.finished_jobs[job_id].update(status='finished') - if self.finished_jobs[job_id].workers: - self.finished_jobs[job_id].workers = [] + self.finished_jobs[job_id].update(status='finished') @docstrings.get_sectionsf('JobQueueCluster') @@ -206,10 +206,10 @@ def __init__(self, def job_script(self): """ Construct a job submission script """ self._n += 1 - template = self._command_template % {'n': self._n} - return self._script_template % {'job_header': self.job_header, - 'env_header': self._env_header, - 'worker_command': template} + pieces = {'job_header': self.job_header, + 'env_header': self._env_header, + 'worker_command': self._command_template} + return self._script_template % pieces @contextmanager def job_file(self): @@ -283,8 +283,8 @@ def stop_workers(self, workers): def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ - pending_workers = self.worker_processes * len(self.pending_jobs) - active_and_pending = len(self.scheduler.workers) + pending_workers + active_and_pending = (self.worker_processes * + (self.pending_jobs + self.running_jobs)) return self.start_workers(n - active_and_pending) def scale_down(self, workers): @@ -302,7 +302,7 @@ def __enter__(self): return self def __exit__(self, type, value, traceback): - self.stop_workers(self.scheduler.workers) + self.stop_workers(self.pending_jobs + self.running_jobs) self.cluster.__exit__(type, value, traceback) def _job_id_from_submit_output(self, out): diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 76156071..fbdc073d 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -103,7 +103,7 @@ def test_basic(loop): @pytest.mark.env("pbs") # noqa: F811 def test_adaptive(loop): with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', - local_directory='/tmp', ob_extra=['-V'], + local_directory='/tmp', job_extra=['-V'], loop=loop) as cluster: cluster.adapt() with Client(cluster) as client: @@ -142,7 +142,7 @@ def test_adaptive(loop): @pytest.mark.env("pbs") # noqa: F811 def test_adaptive_grouped(loop): with PBSCluster(walltime='00:02:00', processes=2, threads=1, memory='2GB', - local_directory='/tmp', ob_extra=['-V'], + local_directory='/tmp', job_extra=['-V'], loop=loop) as cluster: cluster.adapt(minimum=1) with Client(cluster) as client: From ef62f5949be19ba4455d1056b3b1d25860590dc2 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 29 May 2018 10:59:58 -0700 Subject: [PATCH 12/59] fix list addition --- dask_jobqueue/core.py | 12 +++++++++--- dask_jobqueue/tests/test_sge.py | 4 ++-- dask_jobqueue/tests/test_slurm.py | 4 ++-- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 1c35a2a1..46c8ec68 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -279,12 +279,17 @@ def stop_workers(self, workers): if not workers: return jobs = {_job_id_from_worker_name(w.name) for w in workers} + self.stop_jobs(jobs) + + def stop_jobs(self, jobs): + """ Stop a list of jobs""" self._call([self.cancel_command] + list(jobs)) def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ - active_and_pending = (self.worker_processes * - (self.pending_jobs + self.running_jobs)) + active_and_pending = sum([len(j.workers) for j in + self.running_jobs.values()]) + active_and_pending += self.worker_processes * self.pending_jobs return self.start_workers(n - active_and_pending) def scale_down(self, workers): @@ -302,7 +307,8 @@ def __enter__(self): return self def __exit__(self, type, value, traceback): - self.stop_workers(self.pending_jobs + self.running_jobs) + jobs = list(self.pending_jobs.keys()) + list(self.running_jobs.keys()) + self.stop_jobs(jobs) self.cluster.__exit__(type, value, traceback) def _job_id_from_submit_output(self, out): diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 4bf6c0de..2a3d819f 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -16,7 +16,7 @@ def test_basic(loop): # noqa: F811 workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) assert future.result(60) == 11 - assert cluster.jobs + assert cluster.running_jobs info = client.scheduler_info() w = list(info['workers'].values())[0] @@ -30,4 +30,4 @@ def test_basic(loop): # noqa: F811 sleep(0.100) assert time() < start + 10 - assert not cluster.jobs + assert not cluster.running_jobs diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 12599acf..e0ae085f 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -94,7 +94,7 @@ def test_basic(loop): workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) assert future.result(60) == 11 - assert cluster.jobs + assert cluster.running_jobs info = client.scheduler_info() w = list(info['workers'].values())[0] @@ -108,7 +108,7 @@ def test_basic(loop): sleep(0.100) assert time() < start + 10 - assert not cluster.jobs + assert not cluster.running_jobs @pytest.mark.env("slurm") # noqa: F811 From a4e007a53da421ee2315036a5e7fa20618055abf Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 29 May 2018 11:06:40 -0700 Subject: [PATCH 13/59] mark test modules (again) --- dask_jobqueue/tests/test_pbs.py | 2 ++ dask_jobqueue/tests/test_slurm.py | 2 ++ 2 files changed, 4 insertions(+) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index fbdc073d..66ea77ee 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -6,6 +6,8 @@ from dask_jobqueue import PBSCluster +pytestmark = pytest.mark.env("pbs") + def test_header(): with PBSCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index e0ae085f..0cd07ebd 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -6,6 +6,8 @@ from dask_jobqueue import SLURMCluster +pytestmark = pytest.mark.env("slurm") + def test_header(): with SLURMCluster(walltime='00:02:00', processes=4, threads=2, From c19e4daa6120d134a5a569837fcfb9f3cef64dba Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 29 May 2018 15:10:05 -0600 Subject: [PATCH 14/59] fixes while testing on pbs --- dask_jobqueue/core.py | 43 +++++++++++++++++++++++++++++++------------ 1 file changed, 31 insertions(+), 12 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 46c8ec68..36e351d7 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -16,6 +16,7 @@ from distributed.deploy import Cluster from distributed.utils import get_ip_interface, parse_bytes, tmpfile from distributed.diagnostics.plugin import SchedulerPlugin +from sortedcontainers import SortedDict dirname = os.path.dirname(sys.executable) @@ -25,20 +26,19 @@ def _job_id_from_worker_name(name): ''' utility to parse the job ID from the worker name''' - return name.split('-')[-1] + return name.split('-')[-2] class Job(object): - def __init__(self, job_id, workers=None, status=None): + def __init__(self, job_id, status=None): self.job_id = job_id self.status = status - if workers is None: - workers = [] - self.workers = workers + self.workers = SortedDict() def update(self, worker=None, status=None): + ''' update the status this job''' if worker is not None: - self.workers.append(worker) + self.workers[worker.address] = worker if status is not None: self.status = status @@ -54,15 +54,34 @@ def __init__(self): self.finished_jobs = OrderedDict() def add_worker(self, scheduler, worker=None, name=None, **kwargs): - job_id = _job_id_from_worker_name(worker.name) + ''' Run when a new worker enters the cluster''' + w = scheduler.workers[worker] + job_id = _job_id_from_worker_name(w.name) + + # if this is the first worker for this job, move job to running if job_id not in self.running_jobs: self.running_jobs[job_id] = self.pending_jobs.pop(job_id) - self.running_jobs[job_id].update(worker=worker, status='running') + self.running_jobs[job_id].update(status='running') + + # add worker to dict of workers in this job + self.running_jobs[job_id].update(worker=w) def remove_worker(self, scheduler=None, worker=None, **kwargs): - job_id = _job_id_from_worker_name(worker.name) - if self.running_jobs[job_id].workers: - self.running_jobs[job_id].workers.remove(worker) + ''' Run when a worker leaves the cluster''' + + job_id = None + for job_id, job in self.running_jobs.items(): + if worker in job.workers: + break + + if job_id is None: + raise ValueError('did not find a job that owned this worker') + + # remove worker from dict of workers on this job id + if worker in self.running_jobs[job_id].workers: + del self.running_jobs[job_id].workers[worker] + + # if there are no more workers, move job to finished status if not self.running_jobs[job_id].workers: self.finished_jobs[job_id] = self.running_jobs.pop(job_id) self.finished_jobs[job_id].update(status='finished') @@ -289,7 +308,7 @@ def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ active_and_pending = sum([len(j.workers) for j in self.running_jobs.values()]) - active_and_pending += self.worker_processes * self.pending_jobs + active_and_pending += self.worker_processes * len(self.pending_jobs) return self.start_workers(n - active_and_pending) def scale_down(self, workers): From 5d5fd85aec07934afc2e624622a22fb78ca03838 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 29 May 2018 16:29:04 -0600 Subject: [PATCH 15/59] remove extra if block --- dask_jobqueue/core.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 36e351d7..e042baa6 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -78,8 +78,7 @@ def remove_worker(self, scheduler=None, worker=None, **kwargs): raise ValueError('did not find a job that owned this worker') # remove worker from dict of workers on this job id - if worker in self.running_jobs[job_id].workers: - del self.running_jobs[job_id].workers[worker] + del self.running_jobs[job_id].workers[worker] # if there are no more workers, move job to finished status if not self.running_jobs[job_id].workers: From 115b0c17ff1a146f157085b88f4681f6aec95a2f Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 29 May 2018 16:33:25 -0600 Subject: [PATCH 16/59] use for/else --- dask_jobqueue/core.py | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index e042baa6..9e071c56 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -68,18 +68,16 @@ def add_worker(self, scheduler, worker=None, name=None, **kwargs): def remove_worker(self, scheduler=None, worker=None, **kwargs): ''' Run when a worker leaves the cluster''' - - job_id = None + # the worker may have already been removed from the scheduler so we + # need to check in running_jobs for a job that has this worker for job_id, job in self.running_jobs.items(): if worker in job.workers: + # remove worker from dict of workers on this job id + del self.running_jobs[job_id].workers[worker] break - - if job_id is None: + else: raise ValueError('did not find a job that owned this worker') - # remove worker from dict of workers on this job id - del self.running_jobs[job_id].workers[worker] - # if there are no more workers, move job to finished status if not self.running_jobs[job_id].workers: self.finished_jobs[job_id] = self.running_jobs.pop(job_id) From cde3ca4a91ae38f7f67eb2dc6e1a1501081465d3 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Wed, 30 May 2018 15:55:03 -0700 Subject: [PATCH 17/59] fix two failing tests --- dask_jobqueue/tests/test_jobqueue_core.py | 10 ++++++++-- dask_jobqueue/tests/test_pbs.py | 9 +++++---- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/dask_jobqueue/tests/test_jobqueue_core.py b/dask_jobqueue/tests/test_jobqueue_core.py index 411ec5e8..7859a043 100644 --- a/dask_jobqueue/tests/test_jobqueue_core.py +++ b/dask_jobqueue/tests/test_jobqueue_core.py @@ -4,6 +4,12 @@ from dask_jobqueue.core import Job +class TestWorker(object): + def __init__(self, name, address): + self.name = name + self.address = address + + def test_errors(): with pytest.raises(NotImplementedError) as info: JobQueueCluster() @@ -21,7 +27,7 @@ def test_job_update(): assert job.status == 'running' assert len(job.workers) == 0 - job.update(worker='worker1') - job.update(worker='worker2') + job.update(worker=TestWorker('worker1', '127.0.0.1:1234')) + job.update(worker=TestWorker('worker2', '127.0.0.1:1235')) assert job.status == 'running' assert len(job.workers) == 2 diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 66ea77ee..a08639ef 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -79,13 +79,14 @@ def test_job_script(): @pytest.mark.env("pbs") # noqa: F811 def test_basic(loop): - with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', local_directory='/tmp', - job_extra=['-V'], loop=loop) as cluster: + with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', + local_directory='/tmp', job_extra=['-V'], + loop=loop) as cluster: with Client(cluster) as client: workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) assert future.result(60) == 11 - assert cluster.jobs + assert cluster.running_jobs info = client.scheduler_info() w = list(info['workers'].values())[0] @@ -99,7 +100,7 @@ def test_basic(loop): sleep(0.100) assert time() < start + 10 - assert not cluster.jobs + assert not cluster.running_jobs @pytest.mark.env("pbs") # noqa: F811 From 66db52d0de56c7be9a37c49fd9820f7c701cbeca Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Sat, 2 Jun 2018 20:27:31 -0700 Subject: [PATCH 18/59] respond to review comments --- dask_jobqueue/core.py | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 9e071c56..aa27ecbd 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -181,12 +181,6 @@ def __init__(self, # plugin for tracking job status self._scheduler_plugin = JobQueuePlugin() self.cluster.scheduler.add_plugin(self._scheduler_plugin) - self.pending_jobs = self._scheduler_plugin.pending_jobs - self.running_jobs = self._scheduler_plugin.running_jobs - self.finished_jobs = self._scheduler_plugin.finished_jobs - - # counter to keep track of how many jobs have been submitted - self._n = 0 # Keep information on process, threads and memory, for use in # subclasses @@ -210,8 +204,7 @@ def __init__(self, self._command_template += " --memory-limit %s" % memory if name is not None: # worker names follow this template: {NAME}-{JOB_ID} - self._command_template += " --name %s" % name # e.g. "dask-worker" - self._command_template += "-${JOB_ID}" + self._command_template += " --name %s-${JOB_ID}" % name if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: @@ -219,9 +212,23 @@ def __init__(self, if extra is not None: self._command_template += extra + @property + def pending_jobs(self): + """ Jobs pending in the queue """ + return self._scheduler_plugin.pending_jobs + + @property + def running_jobs(self): + """ Jobs with currenly active workers """ + return self._scheduler_plugin.running_jobs + + @property + def finished_jobs(self): + """ Jobs that have finished """ + return self._scheduler_plugin.finished_jobs + def job_script(self): """ Construct a job submission script """ - self._n += 1 pieces = {'job_header': self.job_header, 'env_header': self._env_header, 'worker_command': self._command_template} @@ -313,7 +320,7 @@ def scale_down(self, workers): workers = [] for w in workers: try: - # Get the actual "Worker" + # Get the actual WorkerState workers.append(self.scheduler.workers[w]) except KeyError: logger.debug('worker %s is already gone' % w) From ea7d56d56b65d6f81c8947d45e631bdb084c156e Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Mon, 4 Jun 2018 07:16:09 -0700 Subject: [PATCH 19/59] fix bug in scale down --- dask_jobqueue/core.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index aa27ecbd..2df33280 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -317,14 +317,14 @@ def scale_up(self, n, **kwargs): def scale_down(self, workers): ''' Close the workers with the given addresses ''' - workers = [] + worker_states = [] for w in workers: try: # Get the actual WorkerState - workers.append(self.scheduler.workers[w]) + worker_states.append(self.scheduler.workers[w]) except KeyError: logger.debug('worker %s is already gone' % w) - self.stop_workers(workers) + self.stop_workers(worker_states) def __enter__(self): return self From a6d31d22ea554f3f882b5d4e05d9d269b8e59449 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Mon, 4 Jun 2018 14:26:47 -0700 Subject: [PATCH 20/59] fix marks --- dask_jobqueue/tests/test_pbs.py | 2 -- dask_jobqueue/tests/test_sge.py | 3 +-- dask_jobqueue/tests/test_slurm.py | 2 -- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index a08639ef..3afbe4da 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -6,8 +6,6 @@ from dask_jobqueue import PBSCluster -pytestmark = pytest.mark.env("pbs") - def test_header(): with PBSCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 2a3d819f..949e79c1 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -6,9 +6,8 @@ from dask_jobqueue import SGECluster -pytestmark = pytest.mark.env("sge") - +@pytest.mark.env("sge") # noqa: F811 def test_basic(loop): # noqa: F811 with SGECluster(walltime='00:02:00', threads=2, memory='7GB', loop=loop) as cluster: diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 0cd07ebd..e0ae085f 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -6,8 +6,6 @@ from dask_jobqueue import SLURMCluster -pytestmark = pytest.mark.env("slurm") - def test_header(): with SLURMCluster(walltime='00:02:00', processes=4, threads=2, From ab4363a88c16ec0bc98a180594ea4ccd73bb202c Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Wed, 20 Jun 2018 20:51:22 -0700 Subject: [PATCH 21/59] more docs --- docs/faq.rst | 13 +++++++++++++ docs/index.rst | 7 ++++--- 2 files changed, 17 insertions(+), 3 deletions(-) create mode 100644 docs/faq.rst diff --git a/docs/faq.rst b/docs/faq.rst new file mode 100644 index 00000000..e2e1da64 --- /dev/null +++ b/docs/faq.rst @@ -0,0 +1,13 @@ +.. _faq: + +Frequently Asked Questions +========================== + +What is the difference between a Job and a Worker? +-------------------------------------------------- + +In dask-distributed, a ``Worker`` is a Python object and node in a dask +``Cluster`` that serves two purposes, 1) serve data, and 2) perform\ +computations. Jobs are resources submitted to, and managed by, the job queueing +system (e.g. PBS, SGE, etc.). In dask-jobqueue, a single Job may include one or +more Workers. diff --git a/docs/index.rst b/docs/index.rst index 9d06b994..5ac96691 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -25,7 +25,7 @@ Example from dask_jobqueue import PBSCluster cluster = PBSCluster(processes=6, threads=4, memory="16GB") - cluster.start_workers(10) + cluster.start_workers(12) # this equates to 2 PBS jobs, each with 6 workers from dask.distributed import Client client = Client(cluster) @@ -42,7 +42,7 @@ resources when not actively computing. .. code-block:: python - cluster.adapt(minimum=1, maximum=100) + cluster.adapt(minimum=6, maximum=90) .. toctree:: :maxdepth: 1 @@ -50,6 +50,7 @@ resources when not actively computing. install.rst examples.rst + faq.rst history.rst api.rst @@ -73,7 +74,7 @@ When you ask for more workers, such as with the ``scale`` command .. code-block:: python - cluster.scale(10) + cluster.scale(36) The cluster generates a traditional job script and submits that an appropriate number of times to the job queue. You can see the job script that it will From 604a5636349dcbbc92247dd85defc50c8fb98f63 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Wed, 20 Jun 2018 21:51:37 -0700 Subject: [PATCH 22/59] debugging ci --- dask_jobqueue/tests/test_sge.py | 1 + dask_jobqueue/tests/test_slurm.py | 1 + 2 files changed, 2 insertions(+) diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 949e79c1..cadb79cc 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -11,6 +11,7 @@ def test_basic(loop): # noqa: F811 with SGECluster(walltime='00:02:00', threads=2, memory='7GB', loop=loop) as cluster: + print(cluster.job_script()) with Client(cluster, loop=loop) as client: workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index e0ae085f..0667af13 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -48,6 +48,7 @@ def test_job_script(): memory='7GB') as cluster: job_script = cluster.job_script() + print(job_script) assert '#SBATCH' in job_script assert '#SBATCH -J dask-worker' in job_script assert '#SBATCH -n 1' in job_script From 1e0455e985749601e57479ddc9fc17885ebeb210 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Wed, 20 Jun 2018 22:03:51 -0700 Subject: [PATCH 23/59] more ci --- dask_jobqueue/tests/test_jobqueue_core.py | 6 +++--- dask_jobqueue/tests/test_slurm.py | 10 ++++++++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/dask_jobqueue/tests/test_jobqueue_core.py b/dask_jobqueue/tests/test_jobqueue_core.py index 7859a043..e5333fe1 100644 --- a/dask_jobqueue/tests/test_jobqueue_core.py +++ b/dask_jobqueue/tests/test_jobqueue_core.py @@ -4,7 +4,7 @@ from dask_jobqueue.core import Job -class TestWorker(object): +class DummyWorker(object): def __init__(self, name, address): self.name = name self.address = address @@ -27,7 +27,7 @@ def test_job_update(): assert job.status == 'running' assert len(job.workers) == 0 - job.update(worker=TestWorker('worker1', '127.0.0.1:1234')) - job.update(worker=TestWorker('worker2', '127.0.0.1:1235')) + job.update(worker=DummyWorker('worker1', '127.0.0.1:1234')) + job.update(worker=DummyWorker('worker2', '127.0.0.1:1235')) assert job.status == 'running' assert len(job.workers) == 2 diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 0667af13..1caecf28 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -11,6 +11,8 @@ def test_header(): with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: + print(cluster.job_header) + assert '#SBATCH' in cluster.job_header assert '#SBATCH -J dask-worker' in cluster.job_header assert '#SBATCH -n 1' in cluster.job_header @@ -20,19 +22,23 @@ def test_header(): assert '#SBATCH -p' not in cluster.job_header assert '#SBATCH -A' not in cluster.job_header - with SLURMCluster(queue='regular', project='DaskOnPBS', processes=4, + with SLURMCluster(queue='regular', project='DaskOnSlurm', processes=4, threads=2, memory='7GB', job_cpu=16, job_mem='100G') as cluster: + print(cluster.job_header) + assert '#SBATCH --cpus-per-task=16' in cluster.job_header assert '#SBATCH --cpus-per-task=8' not in cluster.job_header assert '#SBATCH --mem=100G' in cluster.job_header assert '#SBATCH -t ' in cluster.job_header - assert '#SBATCH -A DaskOnPBS' in cluster.job_header + assert '#SBATCH -A DaskOnSlurm' in cluster.job_header assert '#SBATCH -p regular' in cluster.job_header with SLURMCluster() as cluster: + print(cluster.job_header) + assert '#SBATCH' in cluster.job_header assert '#SBATCH -J ' in cluster.job_header assert '#SBATCH -n 1' in cluster.job_header From ace37ad836cf34486a465f785e19c000b2da6fe8 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Wed, 20 Jun 2018 22:18:40 -0700 Subject: [PATCH 24/59] only stop jobs if there are jobs to stop --- dask_jobqueue/core.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 2df33280..d733bb68 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -306,7 +306,8 @@ def stop_workers(self, workers): def stop_jobs(self, jobs): """ Stop a list of jobs""" - self._call([self.cancel_command] + list(jobs)) + if jobs: + self._call([self.cancel_command] + list(jobs)) def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ From 56e29903ba1c7e3f3788497929e303daa38d0100 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Thu, 21 Jun 2018 07:00:31 -0700 Subject: [PATCH 25/59] refactor remove_worker method --- dask_jobqueue/core.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index d733bb68..1303ad54 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -59,6 +59,7 @@ def add_worker(self, scheduler, worker=None, name=None, **kwargs): job_id = _job_id_from_worker_name(w.name) # if this is the first worker for this job, move job to running + print(self.running_jobs) if job_id not in self.running_jobs: self.running_jobs[job_id] = self.pending_jobs.pop(job_id) self.running_jobs[job_id].update(status='running') @@ -74,14 +75,13 @@ def remove_worker(self, scheduler=None, worker=None, **kwargs): if worker in job.workers: # remove worker from dict of workers on this job id del self.running_jobs[job_id].workers[worker] - break - else: - raise ValueError('did not find a job that owned this worker') - # if there are no more workers, move job to finished status - if not self.running_jobs[job_id].workers: - self.finished_jobs[job_id] = self.running_jobs.pop(job_id) - self.finished_jobs[job_id].update(status='finished') + # if there are no more workers, move job to finished status + if not self.running_jobs[job_id].workers: + self.finished_jobs[job_id] = self.running_jobs.pop(job_id) + self.finished_jobs[job_id].update(status='finished') + + break @docstrings.get_sectionsf('JobQueueCluster') From 914244cd9517e6bf10eedddc3ffb0c98ea8eafdb Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Thu, 21 Jun 2018 07:20:51 -0700 Subject: [PATCH 26/59] debug --- dask_jobqueue/core.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 1303ad54..ebbb0c3b 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -26,6 +26,7 @@ def _job_id_from_worker_name(name): ''' utility to parse the job ID from the worker name''' + print('_job_id_from_worker_name: ', name) return name.split('-')[-2] From 359be5929d07a3cacb17fc0f8730cfeeb2a3ee31 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Thu, 21 Jun 2018 07:40:24 -0700 Subject: [PATCH 27/59] print debug info --- dask_jobqueue/core.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index ebbb0c3b..25110a5d 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -21,6 +21,7 @@ dirname = os.path.dirname(sys.executable) logger = logging.getLogger(__name__) +logger.setLevel(10) docstrings = docrep.DocstringProcessor() From 1441634692d2800c12f93111ce2ae064d03b5d98 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Thu, 21 Jun 2018 08:10:47 -0700 Subject: [PATCH 28/59] longer waits in tests --- dask_jobqueue/tests/test_pbs.py | 24 +++++++++++++----------- dask_jobqueue/tests/test_sge.py | 4 +++- dask_jobqueue/tests/test_slurm.py | 14 ++++++++------ 3 files changed, 24 insertions(+), 18 deletions(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 3afbe4da..f28197e8 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -6,6 +6,8 @@ from dask_jobqueue import PBSCluster +from . import QUEUE_WAIT + def test_header(): with PBSCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: @@ -96,7 +98,7 @@ def test_basic(loop): start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT assert not cluster.running_jobs @@ -114,30 +116,30 @@ def test_adaptive(loop): start = time() while not len(cluster.pending_jobs): sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() while not len(cluster.running_jobs): sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() processes = cluster.worker_processes while len(client.scheduler_info()['workers']) != processes: sleep(0.1) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT del future start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() while cluster.pending_jobs or cluster.running_jobs: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT @pytest.mark.env("pbs") # noqa: F811 @@ -153,27 +155,27 @@ def test_adaptive_grouped(loop): start = time() while not len(cluster.pending_jobs): sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() while not len(cluster.running_jobs): sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() processes = cluster.worker_processes while len(client.scheduler_info()['workers']) != processes: sleep(0.1) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT del future start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() while cluster.pending_jobs or cluster.running_jobs: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index cadb79cc..7e9d7d74 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -6,6 +6,8 @@ from dask_jobqueue import SGECluster +from . import QUEUE_WAIT + @pytest.mark.env("sge") # noqa: F811 def test_basic(loop): # noqa: F811 @@ -28,6 +30,6 @@ def test_basic(loop): # noqa: F811 start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT assert not cluster.running_jobs diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 1caecf28..52882c4d 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -6,6 +6,8 @@ from dask_jobqueue import SLURMCluster +from . import QUEUE_WAIT + def test_header(): with SLURMCluster(walltime='00:02:00', processes=4, threads=2, @@ -113,7 +115,7 @@ def test_basic(loop): start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT assert not cluster.running_jobs @@ -130,27 +132,27 @@ def test_adaptive(loop): start = time() while not len(cluster.pending_jobs): sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() while not len(cluster.running_jobs): sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() processes = cluster.worker_processes while len(client.scheduler_info()['workers']) != processes: sleep(0.1) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT del future start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT start = time() while cluster.pending_jobs or cluster.running_jobs: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT From 0bf53d1f0dfb588c4cda873cb55e6f42e0fbc588 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 18 May 2018 15:35:40 -0700 Subject: [PATCH 29/59] refactor use of jobids, add scheduler plugin --- dask_jobqueue/core.py | 168 +++++++++++++++++----- dask_jobqueue/jobqueue.yaml | 2 +- dask_jobqueue/pbs.py | 3 +- dask_jobqueue/sge.py | 5 +- dask_jobqueue/slurm.py | 3 +- dask_jobqueue/tests/__init__.py | 2 + dask_jobqueue/tests/test_jobqueue_core.py | 24 +++- dask_jobqueue/tests/test_pbs.py | 100 ++++++++++--- dask_jobqueue/tests/test_sge.py | 11 +- dask_jobqueue/tests/test_slurm.py | 74 +++++----- docs/faq.rst | 13 ++ docs/index.rst | 7 +- 12 files changed, 309 insertions(+), 103 deletions(-) create mode 100644 dask_jobqueue/tests/__init__.py create mode 100644 docs/faq.rst diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 7c1cca8f..27a60568 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -1,16 +1,22 @@ +from __future__ import absolute_import, division, print_function + import logging +import math import os import shlex import socket import subprocess import sys +from collections import OrderedDict from contextlib import contextmanager import dask import docrep from distributed import LocalCluster from distributed.deploy import Cluster -from distributed.utils import get_ip_interface, ignoring, parse_bytes, tmpfile +from distributed.utils import get_ip_interface, parse_bytes, tmpfile +from distributed.diagnostics.plugin import SchedulerPlugin +from sortedcontainers import SortedDict dirname = os.path.dirname(sys.executable) @@ -18,6 +24,75 @@ docstrings = docrep.DocstringProcessor() +def _job_id_from_worker_name(name): + ''' utility to parse the job ID from the worker name + + template: 'prefix-jobid[-proc]' + ''' + pieces = name.split('-') + if len(pieces) == 2: + return pieces[-1] + else: + return pieces[-2] + + +class Job(object): + def __init__(self, job_id, status=None): + self.job_id = job_id + self.status = status + self.workers = SortedDict() + + def update(self, worker=None, status=None): + ''' update the status this job''' + if worker is not None: + self.workers[worker.address] = worker + if status is not None: + self.status = status + + def __repr__(self): + return "<%s: %r, status: %r, workers: %r>" % ( + self.__class__.__name__, self.job_id, self.status, self.workers) + + +class JobQueuePlugin(SchedulerPlugin): + def __init__(self): + self.pending_jobs = OrderedDict() + self.running_jobs = OrderedDict() + self.finished_jobs = OrderedDict() + + def add_worker(self, scheduler, worker=None, name=None, **kwargs): + ''' Run when a new worker enters the cluster''' + w = scheduler.workers[worker] + job_id = _job_id_from_worker_name(w.name) + + # if this is the first worker for this job, move job to running + if job_id not in self.running_jobs: + if job_id not in self.pending_jobs: + raise KeyError( + '%s not in pending jobs: %s' % (job_id, self.pending_jobs)) + self.running_jobs[job_id] = self.pending_jobs.pop(job_id) + self.running_jobs[job_id].update(status='running') + + # add worker to dict of workers in this job + self.running_jobs[job_id].update(worker=w) + + def remove_worker(self, scheduler=None, worker=None, **kwargs): + ''' Run when a worker leaves the cluster''' + # the worker may have already been removed from the scheduler so we + # need to check in running_jobs for a job that has this worker + for job_id, job in self.running_jobs.items(): + if worker in job.workers: + # remove worker from dict of workers on this job id + del self.running_jobs[job_id].workers[worker] + + # if there are no more workers, move job to finished status + if not self.running_jobs[job_id].workers: + self.finished_jobs[job_id] = self.running_jobs.pop(job_id) + self.finished_jobs[job_id].update(status='finished') + + break + + @docstrings.get_sectionsf('JobQueueCluster') class JobQueueCluster(Cluster): """ Base class to launch Dask Clusters for Job queues @@ -77,6 +152,8 @@ class JobQueueCluster(Cluster): # Following class attributes should be overriden by extending classes. submit_command = None cancel_command = None + _adaptive_options = { + 'worker_key': lambda ws: _job_id_from_worker_name(ws.name)} def __init__(self, name=dask.config.get('jobqueue.name'), @@ -99,7 +176,10 @@ def __init__(self, raise NotImplementedError('JobQueueCluster is an abstract class ' 'that should not be instanciated.') - #This attribute should be overriden + if '-' in name: + raise ValueError('name (%s) can not include the `-` character') + + # This attribute should be overriden self.job_header = None if interface: @@ -110,6 +190,10 @@ def __init__(self, self.cluster = LocalCluster(n_workers=0, ip=host, **kwargs) + # plugin for tracking job status + self._scheduler_plugin = JobQueuePlugin() + self.cluster.scheduler.add_plugin(self._scheduler_plugin) + # Keep information on process, threads and memory, for use in # subclasses self.worker_memory = parse_bytes(memory) if memory is not None else None @@ -117,8 +201,6 @@ def __init__(self, self.worker_threads = threads self.name = name - self.jobs = dict() - self.n = 0 self._adaptive = None self._env_header = '\n'.join(env_extra) @@ -133,8 +215,8 @@ def __init__(self, if memory is not None: self._command_template += " --memory-limit %s" % memory if name is not None: - self._command_template += " --name %s" % name - self._command_template += "-%(n)d" # Keep %(n) to be replaced later + # worker names follow this template: {NAME}-{JOB_ID} + self._command_template += " --name %s-${JOB_ID}" % name if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: @@ -142,13 +224,27 @@ def __init__(self, if extra is not None: self._command_template += extra + @property + def pending_jobs(self): + """ Jobs pending in the queue """ + return self._scheduler_plugin.pending_jobs + + @property + def running_jobs(self): + """ Jobs with currenly active workers """ + return self._scheduler_plugin.running_jobs + + @property + def finished_jobs(self): + """ Jobs that have finished """ + return self._scheduler_plugin.finished_jobs + def job_script(self): """ Construct a job submission script """ - self.n += 1 - template = self._command_template % {'n': self.n} - return self._script_template % {'job_header': self.job_header, - 'env_header': self._env_header, - 'worker_command': template} + pieces = {'job_header': self.job_header, + 'env_header': self._env_header, + 'worker_command': self._command_template} + return self._script_template % pieces @contextmanager def job_file(self): @@ -160,14 +256,13 @@ def job_file(self): def start_workers(self, n=1): """ Start workers and point them to our local scheduler """ - workers = [] - for _ in range(n): + num_jobs = math.ceil(n / self.worker_processes) + for _ in range(num_jobs): with self.job_file() as fn: out = self._call(shlex.split(self.submit_command) + [fn]) job = self._job_id_from_submit_output(out.decode()) - self.jobs[self.n] = job - workers.append(self.n) - return workers + self._scheduler_plugin.pending_jobs[job] = Job( + job, status='pending') @property def scheduler(self): @@ -196,12 +291,12 @@ def _calls(self, cmds): Also logs any stderr information """ logger.debug("Submitting the following calls to command line") + procs = [] for cmd in cmds: logger.debug(' '.join(cmd)) - procs = [subprocess.Popen(cmd, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE) - for cmd in cmds] + procs.append(subprocess.Popen(cmd, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE)) result = [] for proc in procs: @@ -219,29 +314,38 @@ def stop_workers(self, workers): """ Stop a list of workers""" if not workers: return - workers = list(map(int, workers)) - jobs = [self.jobs[w] for w in workers] - self._call([self.cancel_command] + list(jobs)) - for w in workers: - with ignoring(KeyError): - del self.jobs[w] + jobs = {_job_id_from_worker_name(w.name) for w in workers} + self.stop_jobs(jobs) + + def stop_jobs(self, jobs): + """ Stop a list of jobs""" + if jobs: + self._call([self.cancel_command] + list(jobs)) def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ - return self.start_workers(n - len(self.jobs)) + active_and_pending = sum([len(j.workers) for j in + self.running_jobs.values()]) + active_and_pending += self.worker_processes * len(self.pending_jobs) + return self.start_workers(n - active_and_pending) def scale_down(self, workers): ''' Close the workers with the given addresses ''' - if isinstance(workers, dict): - names = {v['name'] for v in workers.values()} - job_ids = {name.split('-')[-2] for name in names} - self.stop_workers(job_ids) + worker_states = [] + for w in workers: + try: + # Get the actual WorkerState + worker_states.append(self.scheduler.workers[w]) + except KeyError: + logger.debug('worker %s is already gone' % w) + self.stop_workers(worker_states) def __enter__(self): return self def __exit__(self, type, value, traceback): - self.stop_workers(self.jobs) + jobs = list(self.pending_jobs.keys()) + list(self.running_jobs.keys()) + self.stop_jobs(jobs) self.cluster.__exit__(type, value, traceback) def _job_id_from_submit_output(self, out): diff --git a/dask_jobqueue/jobqueue.yaml b/dask_jobqueue/jobqueue.yaml index 018936f4..bd103fec 100644 --- a/dask_jobqueue/jobqueue.yaml +++ b/dask_jobqueue/jobqueue.yaml @@ -1,5 +1,5 @@ jobqueue: - name: dask-worker + name: dask_worker threads: 2 processes: 4 memory: 8GB diff --git a/dask_jobqueue/pbs.py b/dask_jobqueue/pbs.py index 350547ad..3475d74a 100644 --- a/dask_jobqueue/pbs.py +++ b/dask_jobqueue/pbs.py @@ -73,7 +73,7 @@ def __init__(self, # Try to find a project name from environment variable project = project or os.environ.get('PBS_ACCOUNT') - header_lines = [] + header_lines = ['#!/usr/bin/env bash'] # PBS header build if self.name is not None: header_lines.append('#PBS -N %s' % self.name) @@ -93,6 +93,7 @@ def __init__(self, if walltime is not None: header_lines.append('#PBS -l walltime=%s' % walltime) header_lines.extend(['#PBS %s' % arg for arg in job_extra]) + header_lines.append('JOB_ID=${PBS_JOBID%.*}') # Declare class attribute that shall be overriden self.job_header = '\n'.join(header_lines) diff --git a/dask_jobqueue/sge.py b/dask_jobqueue/sge.py index e6a2a841..938fba3a 100644 --- a/dask_jobqueue/sge.py +++ b/dask_jobqueue/sge.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import, division, print_function + import logging import dask @@ -52,8 +54,7 @@ def __init__(self, super(SGECluster, self).__init__(**kwargs) - header_lines = ['#!/bin/bash'] - + header_lines = ['#!/usr/bin/env bash'] if self.name is not None: header_lines.append('#$ -N %(name)s') if queue is not None: diff --git a/dask_jobqueue/slurm.py b/dask_jobqueue/slurm.py index d4878315..73a012bd 100644 --- a/dask_jobqueue/slurm.py +++ b/dask_jobqueue/slurm.py @@ -71,7 +71,7 @@ def __init__(self, super(SLURMCluster, self).__init__(**kwargs) # Always ask for only one task - header_lines = [] + header_lines = ['#!/usr/bin/env bash'] # SLURM header build if self.name is not None: header_lines.append('#SBATCH -J %s' % self.name) @@ -99,6 +99,7 @@ def __init__(self, if walltime is not None: header_lines.append('#SBATCH -t %s' % walltime) + header_lines.append('JOB_ID=${SLURM_JOB_ID%;*}') header_lines.extend(['#SBATCH %s' % arg for arg in job_extra]) # Declare class attribute that shall be overriden diff --git a/dask_jobqueue/tests/__init__.py b/dask_jobqueue/tests/__init__.py new file mode 100644 index 00000000..575ccfb2 --- /dev/null +++ b/dask_jobqueue/tests/__init__.py @@ -0,0 +1,2 @@ + +QUEUE_WAIT = 60 # seconds diff --git a/dask_jobqueue/tests/test_jobqueue_core.py b/dask_jobqueue/tests/test_jobqueue_core.py index 3f9a8a85..e5333fe1 100644 --- a/dask_jobqueue/tests/test_jobqueue_core.py +++ b/dask_jobqueue/tests/test_jobqueue_core.py @@ -1,11 +1,13 @@ import pytest from dask_jobqueue import JobQueueCluster +from dask_jobqueue.core import Job -def test_jq_core_placeholder(): - # to test that CI is working - pass +class DummyWorker(object): + def __init__(self, name, address): + self.name = name + self.address = address def test_errors(): @@ -13,3 +15,19 @@ def test_errors(): JobQueueCluster() assert 'abstract class' in str(info.value) + + +def test_job_update(): + job = Job(job_id='1234', status='pending') + assert job.job_id == '1234' + assert job.status == 'pending' + assert len(job.workers) == 0 + + job.update(status='running') + assert job.status == 'running' + assert len(job.workers) == 0 + + job.update(worker=DummyWorker('worker1', '127.0.0.1:1234')) + job.update(worker=DummyWorker('worker2', '127.0.0.1:1235')) + assert job.status == 'running' + assert len(job.workers) == 2 diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 39e810e0..c50e03c3 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -6,12 +6,14 @@ from dask_jobqueue import PBSCluster +from . import QUEUE_WAIT + def test_header(): with PBSCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: assert '#PBS' in cluster.job_header - assert '#PBS -N dask-worker' in cluster.job_header + assert '#PBS -N dask_worker' in cluster.job_header assert '#PBS -l select=1:ncpus=8:mem=27GB' in cluster.job_header assert '#PBS -l walltime=00:02:00' in cluster.job_header assert '#PBS -q' not in cluster.job_header @@ -21,7 +23,7 @@ def test_header(): resource_spec='select=1:ncpus=24:mem=100GB') as cluster: assert '#PBS -q regular' in cluster.job_header - assert '#PBS -N dask-worker' in cluster.job_header + assert '#PBS -N dask_worker' in cluster.job_header assert '#PBS -l select=1:ncpus=24:mem=100GB' in cluster.job_header assert '#PBS -l select=1:ncpus=8:mem=27GB' not in cluster.job_header assert '#PBS -l walltime=' in cluster.job_header @@ -51,7 +53,7 @@ def test_job_script(): job_script = cluster.job_script() assert '#PBS' in job_script - assert '#PBS -N dask-worker' in job_script + assert '#PBS -N dask_worker' in job_script assert '#PBS -l select=1:ncpus=8:mem=27GB' in job_script assert '#PBS -l walltime=00:02:00' in job_script assert '#PBS -q' not in job_script @@ -65,7 +67,7 @@ def test_job_script(): job_script = cluster.job_script() assert '#PBS -q regular' in job_script - assert '#PBS -N dask-worker' in job_script + assert '#PBS -N dask_worker' in job_script assert '#PBS -l select=1:ncpus=24:mem=100GB' in job_script assert '#PBS -l select=1:ncpus=8:mem=27GB' not in job_script assert '#PBS -l walltime=' in job_script @@ -77,13 +79,14 @@ def test_job_script(): @pytest.mark.env("pbs") # noqa: F811 def test_basic(loop): - with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', local_directory='/tmp', - job_extra=['-V'], loop=loop) as cluster: + with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', + local_directory='/tmp', job_extra=['-V'], + loop=loop) as cluster: with Client(cluster) as client: workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) - assert future.result(60) == 11 - assert cluster.jobs + assert future.result(QUEUE_WAIT) == 11 + assert cluster.running_jobs info = client.scheduler_info() w = list(info['workers'].values())[0] @@ -95,38 +98,89 @@ def test_basic(loop): start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT - assert not cluster.jobs + assert not cluster.running_jobs @pytest.mark.env("pbs") # noqa: F811 def test_adaptive(loop): - with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', local_directory='/tmp', - job_extra=['-V'], loop=loop) as cluster: + with PBSCluster(walltime='00:02:00', processes=1, threads=2, memory='2GB', + local_directory='/tmp', job_extra=['-V'], + loop=loop) as cluster: cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) - assert future.result(60) == 11 + assert future.result(QUEUE_WAIT) == 11 + + start = time() + while not len(cluster.pending_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT + + start = time() + while not len(cluster.running_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT + + start = time() + processes = cluster.worker_processes + while len(client.scheduler_info()['workers']) != processes: + sleep(0.1) + assert time() < start + QUEUE_WAIT + + del future + + start = time() + while len(client.scheduler_info()['workers']) > 0: + sleep(0.100) + assert time() < start + QUEUE_WAIT + + start = time() + while cluster.pending_jobs or cluster.running_jobs: + sleep(0.100) + assert time() < start + QUEUE_WAIT + + +@pytest.mark.env("pbs") # noqa: F811 +def test_adaptive_grouped(loop): + with PBSCluster(walltime='00:02:00', processes=2, threads=1, memory='2GB', + local_directory='/tmp', job_extra=['-V'], + loop=loop) as cluster: + cluster.adapt(minimum=1) + with Client(cluster) as client: + future = client.submit(lambda x: x + 1, 10) + assert future.result(QUEUE_WAIT) == 11 + + start = time() + while not len(cluster.pending_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT - assert cluster.jobs + start = time() + while not len(cluster.running_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT start = time() processes = cluster.worker_processes while len(client.scheduler_info()['workers']) != processes: sleep(0.1) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT del future start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 - - # There is probably a bug to fix in the adaptive methods of the JobQueueCluster - # Currently cluster.jobs is not cleaned up. - #start = time() - #while cluster.jobs: - # sleep(0.100) - # assert time() < start + 10 + assert time() < start + QUEUE_WAIT + + start = time() + while cluster.pending_jobs or cluster.running_jobs: + sleep(0.100) + assert time() < start + QUEUE_WAIT + + +def test_valid_worker_name(): + with pytest.raises(ValueError): + PBSCluster(name='dask-worker') diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 4bf6c0de..830022db 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -6,17 +6,18 @@ from dask_jobqueue import SGECluster -pytestmark = pytest.mark.env("sge") +from . import QUEUE_WAIT +@pytest.mark.env("sge") # noqa: F811 def test_basic(loop): # noqa: F811 with SGECluster(walltime='00:02:00', threads=2, memory='7GB', loop=loop) as cluster: with Client(cluster, loop=loop) as client: workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) - assert future.result(60) == 11 - assert cluster.jobs + assert future.result(QUEUE_WAIT) == 11 + assert cluster.running_jobs info = client.scheduler_info() w = list(info['workers'].values())[0] @@ -28,6 +29,6 @@ def test_basic(loop): # noqa: F811 start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT - assert not cluster.jobs + assert not cluster.running_jobs diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index c2bba71c..3269445b 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -6,12 +6,14 @@ from dask_jobqueue import SLURMCluster +from . import QUEUE_WAIT -def test_header(): - with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: +def test_header(): + with SLURMCluster(walltime='00:02:00', processes=4, threads=2, + memory='7GB') as cluster: assert '#SBATCH' in cluster.job_header - assert '#SBATCH -J dask-worker' in cluster.job_header + assert '#SBATCH -J dask_worker' in cluster.job_header assert '#SBATCH -n 1' in cluster.job_header assert '#SBATCH --cpus-per-task=8' in cluster.job_header assert '#SBATCH --mem=27G' in cluster.job_header @@ -19,18 +21,17 @@ def test_header(): assert '#SBATCH -p' not in cluster.job_header assert '#SBATCH -A' not in cluster.job_header - with SLURMCluster(queue='regular', project='DaskOnPBS', processes=4, threads=2, memory='7GB', + with SLURMCluster(queue='regular', project='DaskOnSlurm', processes=4, + threads=2, memory='7GB', job_cpu=16, job_mem='100G') as cluster: - assert '#SBATCH --cpus-per-task=16' in cluster.job_header assert '#SBATCH --cpus-per-task=8' not in cluster.job_header assert '#SBATCH --mem=100G' in cluster.job_header assert '#SBATCH -t ' in cluster.job_header - assert '#SBATCH -A DaskOnPBS' in cluster.job_header + assert '#SBATCH -A DaskOnSlurm' in cluster.job_header assert '#SBATCH -p regular' in cluster.job_header with SLURMCluster() as cluster: - assert '#SBATCH' in cluster.job_header assert '#SBATCH -J ' in cluster.job_header assert '#SBATCH -n 1' in cluster.job_header @@ -42,11 +43,12 @@ def test_header(): def test_job_script(): - with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB') as cluster: + with SLURMCluster(walltime='00:02:00', processes=4, threads=2, + memory='7GB') as cluster: job_script = cluster.job_script() assert '#SBATCH' in job_script - assert '#SBATCH -J dask-worker' in job_script + assert '#SBATCH -J dask_worker' in job_script assert '#SBATCH -n 1' in job_script assert '#SBATCH --cpus-per-task=8' in job_script assert '#SBATCH --mem=27G' in job_script @@ -59,13 +61,15 @@ def test_job_script(): assert '/dask-worker tcp://' in job_script assert '--nthreads 2 --nprocs 4 --memory-limit 7GB' in job_script - with SLURMCluster(walltime='00:02:00', processes=4, threads=2, memory='7GB', - env_extra=['export LANG="en_US.utf8"', 'export LANGUAGE="en_US.utf8"', + with SLURMCluster(walltime='00:02:00', processes=4, threads=2, + memory='7GB', + env_extra=['export LANG="en_US.utf8"', + 'export LANGUAGE="en_US.utf8"', 'export LC_ALL="en_US.utf8"'] ) as cluster: job_script = cluster.job_script() assert '#SBATCH' in job_script - assert '#SBATCH -J dask-worker' in job_script + assert '#SBATCH -J dask_worker' in job_script assert '#SBATCH -n 1' in job_script assert '#SBATCH --cpus-per-task=8' in job_script assert '#SBATCH --mem=27G' in job_script @@ -83,13 +87,13 @@ def test_job_script(): @pytest.mark.env("slurm") # noqa: F811 def test_basic(loop): - with SLURMCluster(walltime='00:02:00', threads=2, processes=1, memory='4GB', - job_extra=['-D /'], loop=loop) as cluster: + with SLURMCluster(walltime='00:02:00', threads=2, processes=1, + memory='4GB', job_extra=['-D /'], loop=loop) as cluster: with Client(cluster) as client: workers = cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) - assert future.result(60) == 11 - assert cluster.jobs + assert future.result(QUEUE_WAIT) == 11 + assert cluster.running_jobs info = client.scheduler_info() w = list(info['workers'].values())[0] @@ -101,38 +105,44 @@ def test_basic(loop): start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT - assert not cluster.jobs + assert not cluster.running_jobs @pytest.mark.env("slurm") # noqa: F811 def test_adaptive(loop): - with SLURMCluster(walltime='00:02:00', threads=2, processes=1, memory='4GB', - job_extra=['-D /'], loop=loop) as cluster: + with SLURMCluster(walltime='00:02:00', threads=2, processes=1, + memory='4GB', job_extra=['-D /'], loop=loop) as cluster: cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) - assert future.result(60) == 11 + assert future.result(QUEUE_WAIT) == 11 + + start = time() + while not len(cluster.pending_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT - assert cluster.jobs + start = time() + while not len(cluster.running_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT start = time() processes = cluster.worker_processes - while (len(client.scheduler_info()['workers']) != processes): + while len(client.scheduler_info()['workers']) != processes: sleep(0.1) - assert time() < start + 10 + assert time() < start + QUEUE_WAIT del future start = time() while len(client.scheduler_info()['workers']) > 0: sleep(0.100) - assert time() < start + 10 - - # There is probably a bug to fix in the adaptive methods of the JobQueueCluster - # Currently cluster.jobs is not cleaned up. - # start = time() - # while cluster.jobs: - # sleep(0.100) - # assert time() < start + 10 + assert time() < start + QUEUE_WAIT + + start = time() + while cluster.pending_jobs or cluster.running_jobs: + sleep(0.100) + assert time() < start + QUEUE_WAIT diff --git a/docs/faq.rst b/docs/faq.rst new file mode 100644 index 00000000..e2e1da64 --- /dev/null +++ b/docs/faq.rst @@ -0,0 +1,13 @@ +.. _faq: + +Frequently Asked Questions +========================== + +What is the difference between a Job and a Worker? +-------------------------------------------------- + +In dask-distributed, a ``Worker`` is a Python object and node in a dask +``Cluster`` that serves two purposes, 1) serve data, and 2) perform\ +computations. Jobs are resources submitted to, and managed by, the job queueing +system (e.g. PBS, SGE, etc.). In dask-jobqueue, a single Job may include one or +more Workers. diff --git a/docs/index.rst b/docs/index.rst index 9d06b994..5ac96691 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -25,7 +25,7 @@ Example from dask_jobqueue import PBSCluster cluster = PBSCluster(processes=6, threads=4, memory="16GB") - cluster.start_workers(10) + cluster.start_workers(12) # this equates to 2 PBS jobs, each with 6 workers from dask.distributed import Client client = Client(cluster) @@ -42,7 +42,7 @@ resources when not actively computing. .. code-block:: python - cluster.adapt(minimum=1, maximum=100) + cluster.adapt(minimum=6, maximum=90) .. toctree:: :maxdepth: 1 @@ -50,6 +50,7 @@ resources when not actively computing. install.rst examples.rst + faq.rst history.rst api.rst @@ -73,7 +74,7 @@ When you ask for more workers, such as with the ``scale`` command .. code-block:: python - cluster.scale(10) + cluster.scale(36) The cluster generates a traditional job script and submits that an appropriate number of times to the job queue. You can see the job script that it will From cc2628fa962c3db3a6ed93133e1e689167905633 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 08:13:33 -0600 Subject: [PATCH 30/59] debug stop workers --- dask_jobqueue/core.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 27a60568..8d85fd64 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -312,9 +312,11 @@ def _call(self, cmd): def stop_workers(self, workers): """ Stop a list of workers""" + print('stop_workers', workers) if not workers: return jobs = {_job_id_from_worker_name(w.name) for w in workers} + print('jobs to stop', jobs) self.stop_jobs(jobs) def stop_jobs(self, jobs): From 90dd7309a773bb95efb252346fdaf9f598a49097 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 09:24:11 -0600 Subject: [PATCH 31/59] fix tests of stop_workers --- dask_jobqueue/tests/test_pbs.py | 6 +++--- dask_jobqueue/tests/test_sge.py | 6 +++--- dask_jobqueue/tests/test_slurm.py | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index c50e03c3..a33a6883 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -83,13 +83,13 @@ def test_basic(loop): local_directory='/tmp', job_extra=['-V'], loop=loop) as cluster: with Client(cluster) as client: - workers = cluster.start_workers(2) + cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs - info = client.scheduler_info() - w = list(info['workers'].values())[0] + workers = list(client.scheduler_info()['workers'].values()) + w = workers[0] assert w['memory_limit'] == 2e9 assert w['ncores'] == 2 diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 830022db..70bad3e2 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -14,13 +14,13 @@ def test_basic(loop): # noqa: F811 with SGECluster(walltime='00:02:00', threads=2, memory='7GB', loop=loop) as cluster: with Client(cluster, loop=loop) as client: - workers = cluster.start_workers(2) + cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs - info = client.scheduler_info() - w = list(info['workers'].values())[0] + workers = list(client.scheduler_info()['workers'].values()) + w = workers[0] assert w['memory_limit'] == 7e9 assert w['ncores'] == 2 diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 3269445b..e8986f0c 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -90,13 +90,13 @@ def test_basic(loop): with SLURMCluster(walltime='00:02:00', threads=2, processes=1, memory='4GB', job_extra=['-D /'], loop=loop) as cluster: with Client(cluster) as client: - workers = cluster.start_workers(2) + cluster.start_workers(2) future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs - info = client.scheduler_info() - w = list(info['workers'].values())[0] + workers = list(client.scheduler_info()['workers'].values()) + w = workers[0] assert w['memory_limit'] == 4e9 assert w['ncores'] == 2 From 18dfe3112f57ecc769a97446890299f928c1dc42 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 09:41:06 -0600 Subject: [PATCH 32/59] cleanup --- dask_jobqueue/core.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 8d85fd64..5a14f00a 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -312,11 +312,9 @@ def _call(self, cmd): def stop_workers(self, workers): """ Stop a list of workers""" - print('stop_workers', workers) if not workers: return jobs = {_job_id_from_worker_name(w.name) for w in workers} - print('jobs to stop', jobs) self.stop_jobs(jobs) def stop_jobs(self, jobs): @@ -329,7 +327,7 @@ def scale_up(self, n, **kwargs): active_and_pending = sum([len(j.workers) for j in self.running_jobs.values()]) active_and_pending += self.worker_processes * len(self.pending_jobs) - return self.start_workers(n - active_and_pending) + self.start_workers(n - active_and_pending) def scale_down(self, workers): ''' Close the workers with the given addresses ''' From b3032753fe48cba7da734d2af1730e834c3c6459 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 10:00:46 -0600 Subject: [PATCH 33/59] more flexible stop workers --- dask_jobqueue/core.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index efb0f911..5a159a38 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -312,7 +312,12 @@ def stop_workers(self, workers): """ Stop a list of workers""" if not workers: return - jobs = {_job_id_from_worker_name(w.name) for w in workers} + jobs = [] + for w in workers: + if isinstance(w, dict): + jobs.append(_job_id_from_worker_name(w['name'])) + else: + jobs.append(_job_id_from_worker_name(w.name)) self.stop_jobs(jobs) def stop_jobs(self, jobs): From 667369ec9cd57f407c88ab06cd162b162b328ef8 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 22:02:03 -0600 Subject: [PATCH 34/59] remove Job class --- dask_jobqueue/core.py | 45 ++++------------------- dask_jobqueue/tests/test_jobqueue_core.py | 23 ------------ dask_jobqueue/tests/test_pbs.py | 2 + dask_jobqueue/tests/test_sge.py | 1 + dask_jobqueue/tests/test_slurm.py | 1 + 5 files changed, 12 insertions(+), 60 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 5a159a38..01f966f5 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -15,7 +15,6 @@ from distributed.deploy import Cluster from distributed.utils import get_ip_interface, parse_bytes, tmpfile from distributed.diagnostics.plugin import SchedulerPlugin -from sortedcontainers import SortedDict logger = logging.getLogger(__name__) docstrings = docrep.DocstringProcessor() @@ -33,24 +32,6 @@ def _job_id_from_worker_name(name): return pieces[-2] -class Job(object): - def __init__(self, job_id, status=None): - self.job_id = job_id - self.status = status - self.workers = SortedDict() - - def update(self, worker=None, status=None): - ''' update the status this job''' - if worker is not None: - self.workers[worker.address] = worker - if status is not None: - self.status = status - - def __repr__(self): - return "<%s: %r, status: %r, workers: %r>" % ( - self.__class__.__name__, self.job_id, self.status, self.workers) - - class JobQueuePlugin(SchedulerPlugin): def __init__(self): self.pending_jobs = OrderedDict() @@ -64,30 +45,21 @@ def add_worker(self, scheduler, worker=None, name=None, **kwargs): # if this is the first worker for this job, move job to running if job_id not in self.running_jobs: - if job_id not in self.pending_jobs: - raise KeyError( - '%s not in pending jobs: %s' % (job_id, self.pending_jobs)) self.running_jobs[job_id] = self.pending_jobs.pop(job_id) - self.running_jobs[job_id].update(status='running') # add worker to dict of workers in this job - self.running_jobs[job_id].update(worker=w) + self.running_jobs[job_id][w.name] = w def remove_worker(self, scheduler=None, worker=None, **kwargs): ''' Run when a worker leaves the cluster''' - # the worker may have already been removed from the scheduler so we - # need to check in running_jobs for a job that has this worker - for job_id, job in self.running_jobs.items(): - if worker in job.workers: - # remove worker from dict of workers on this job id - del self.running_jobs[job_id].workers[worker] + job_id = _job_id_from_worker_name(worker) - # if there are no more workers, move job to finished status - if not self.running_jobs[job_id].workers: - self.finished_jobs[job_id] = self.running_jobs.pop(job_id) - self.finished_jobs[job_id].update(status='finished') + # remove worker from this job + del self.running_jobs[job_id][worker] - break + # once there are no more workers, move this job to finished_jobs + if not self.running_jobs[job_id]: + self.finished_jobs[job_id] = self.running_jobs.pop(job_id) @docstrings.get_sectionsf('JobQueueCluster') @@ -259,8 +231,7 @@ def start_workers(self, n=1): with self.job_file() as fn: out = self._call(shlex.split(self.submit_command) + [fn]) job = self._job_id_from_submit_output(out.decode()) - self._scheduler_plugin.pending_jobs[job] = Job( - job, status='pending') + self._scheduler_plugin.pending_jobs[job] = {} @property def scheduler(self): diff --git a/dask_jobqueue/tests/test_jobqueue_core.py b/dask_jobqueue/tests/test_jobqueue_core.py index e5333fe1..eb2d5494 100644 --- a/dask_jobqueue/tests/test_jobqueue_core.py +++ b/dask_jobqueue/tests/test_jobqueue_core.py @@ -1,13 +1,6 @@ import pytest from dask_jobqueue import JobQueueCluster -from dask_jobqueue.core import Job - - -class DummyWorker(object): - def __init__(self, name, address): - self.name = name - self.address = address def test_errors(): @@ -15,19 +8,3 @@ def test_errors(): JobQueueCluster() assert 'abstract class' in str(info.value) - - -def test_job_update(): - job = Job(job_id='1234', status='pending') - assert job.job_id == '1234' - assert job.status == 'pending' - assert len(job.workers) == 0 - - job.update(status='running') - assert job.status == 'running' - assert len(job.workers) == 0 - - job.update(worker=DummyWorker('worker1', '127.0.0.1:1234')) - job.update(worker=DummyWorker('worker2', '127.0.0.1:1235')) - assert job.status == 'running' - assert len(job.workers) == 2 diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index a7d72a71..5e79e03f 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -85,6 +85,7 @@ def test_basic(loop): loop=loop) as cluster: with Client(cluster) as client: cluster.start_workers(2) + assert cluster.pending_jobs or cluster.running_jobs future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs @@ -112,6 +113,7 @@ def test_adaptive(loop): cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) + assert cluster.pending_jobs or cluster.running_jobs assert future.result(QUEUE_WAIT) == 11 start = time() diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 70bad3e2..d2616c11 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -15,6 +15,7 @@ def test_basic(loop): # noqa: F811 loop=loop) as cluster: with Client(cluster, loop=loop) as client: cluster.start_workers(2) + assert cluster.pending_jobs or cluster.running_jobs future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 7cf155e9..808cbb8a 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -92,6 +92,7 @@ def test_basic(loop): memory='4GB', job_extra=['-D /'], loop=loop) as cluster: with Client(cluster) as client: cluster.start_workers(2) + assert cluster.pending_jobs or cluster.running_jobs future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs From 292b59538e847c78a26ddbf55be691e6b1146ffe Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 22:28:07 -0600 Subject: [PATCH 35/59] fix for worker name (again) --- dask_jobqueue/core.py | 3 ++- dask_jobqueue/jobqueue.yaml | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 389016d4..c41f9dba 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -167,7 +167,8 @@ def __init__(self, env_extra = dask.config.get('jobqueue.%s.env-extra' % self.scheduler_name) if '-' in name: - raise ValueError('name (%s) can not include the `-` character') + raise ValueError( + 'name (%s) can not include the `-` character' % name) #This attribute should be overriden self.job_header = None diff --git a/dask_jobqueue/jobqueue.yaml b/dask_jobqueue/jobqueue.yaml index 5686abaa..d5abfd1e 100644 --- a/dask_jobqueue/jobqueue.yaml +++ b/dask_jobqueue/jobqueue.yaml @@ -20,7 +20,7 @@ jobqueue: job-extra: [] sge: - name: dask-worker + name: dask_worker # Dask worker options threads: 2 @@ -40,7 +40,7 @@ jobqueue: resource-spec: null slurm: - name: dask-worker + name: dask_worker # Dask worker options threads: 2 @@ -61,7 +61,7 @@ jobqueue: job-extra: {} moab: - name: dask-worker + name: dask_worker # Dask worker options threads: 2 From 627f8733375f85fd6e62f753e71563ca9ced763d Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 22:52:22 -0600 Subject: [PATCH 36/59] debug --- dask_jobqueue/core.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index c41f9dba..14788e34 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -26,6 +26,7 @@ def _job_id_from_worker_name(name): template: 'prefix-jobid[-proc]' ''' pieces = name.split('-') + print(name, pieces) if len(pieces) == 2: return pieces[-1] else: From f2b2a920870b1ffd94e8163baa01a3d0568aec64 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 26 Jun 2018 23:14:52 -0600 Subject: [PATCH 37/59] perform a backflip --- dask_jobqueue/core.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 14788e34..e06d3ffa 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -38,11 +38,13 @@ def __init__(self): self.pending_jobs = OrderedDict() self.running_jobs = OrderedDict() self.finished_jobs = OrderedDict() + self.all_workers = {} def add_worker(self, scheduler, worker=None, name=None, **kwargs): ''' Run when a new worker enters the cluster''' w = scheduler.workers[worker] job_id = _job_id_from_worker_name(w.name) + self.all_workers[worker] = (w.name, job_id) # if this is the first worker for this job, move job to running if job_id not in self.running_jobs: @@ -53,10 +55,10 @@ def add_worker(self, scheduler, worker=None, name=None, **kwargs): def remove_worker(self, scheduler=None, worker=None, **kwargs): ''' Run when a worker leaves the cluster''' - job_id = _job_id_from_worker_name(worker) + name, job_id = self.all_workers[worker] # remove worker from this job - del self.running_jobs[job_id][worker] + del self.running_jobs[job_id][name] # once there are no more workers, move this job to finished_jobs if not self.running_jobs[job_id]: From c988f1e0f2e3a045f591f0843b857fb98847071b Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 10:03:52 -0700 Subject: [PATCH 38/59] isort after merge conflict resolution --- dask_jobqueue/config.py | 3 +-- dask_jobqueue/core.py | 5 ++--- dask_jobqueue/tests/test_pbs.py | 4 ++-- dask_jobqueue/tests/test_slurm.py | 2 +- docs/conf.py | 3 +-- 5 files changed, 7 insertions(+), 10 deletions(-) diff --git a/dask_jobqueue/config.py b/dask_jobqueue/config.py index 7ab63bb3..f27b2a04 100644 --- a/dask_jobqueue/config.py +++ b/dask_jobqueue/config.py @@ -1,11 +1,10 @@ -from __future__ import print_function, division, absolute_import +from __future__ import absolute_import, division, print_function import os import dask import yaml - fn = os.path.join(os.path.dirname(__file__), 'jobqueue.yaml') dask.config.ensure_file(source=fn) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index ec7f980f..e92b87b2 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -5,7 +5,6 @@ import subprocess import sys import warnings - from collections import OrderedDict from contextlib import contextmanager @@ -13,9 +12,9 @@ import docrep from distributed import LocalCluster from distributed.deploy import Cluster -from distributed.utils import (get_ip_interface, ignoring, parse_bytes, tmpfile, - format_bytes) from distributed.diagnostics.plugin import SchedulerPlugin +from distributed.utils import ( + format_bytes, get_ip_interface, ignoring, parse_bytes, tmpfile) logger = logging.getLogger(__name__) docstrings = docrep.DocstringProcessor() diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 8b10d140..df851d0d 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -2,11 +2,11 @@ from time import sleep, time import dask +import pytest from dask.distributed import Client from distributed.utils_test import loop # noqa: F401 -import pytest -from dask_jobqueue import PBSCluster, MoabCluster +from dask_jobqueue import MoabCluster, PBSCluster from . import QUEUE_WAIT diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index cfc75984..6d6eeaf4 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -1,7 +1,7 @@ +import sys from time import sleep, time import pytest -import sys from distributed import Client from distributed.utils_test import loop # noqa: F401 diff --git a/docs/conf.py b/docs/conf.py index 33f5c430..4b091427 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -24,9 +24,8 @@ copyright = u'2018, Anaconda, Inc. and contributors' -# The short X.Y version. -# version = '0.1.0' from dask_jobqueue import __version__ as version + # The full version, including alpha/beta/rc tags. # release = '0.1.0' release = version From 619047f75e834f9dbaf4fa9de8b6425be6c48640 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 10:24:48 -0700 Subject: [PATCH 39/59] more docs stuff --- dask_jobqueue/core.py | 2 +- dask_jobqueue/moab.py | 2 +- dask_jobqueue/pbs.py | 2 +- dask_jobqueue/sge.py | 2 +- dask_jobqueue/slurm.py | 2 +- docs/faq.rst | 13 ------------- docs/index.rst | 11 ++++++++++- 7 files changed, 15 insertions(+), 19 deletions(-) delete mode 100644 docs/faq.rst diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index e92b87b2..a2dede57 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -14,7 +14,7 @@ from distributed.deploy import Cluster from distributed.diagnostics.plugin import SchedulerPlugin from distributed.utils import ( - format_bytes, get_ip_interface, ignoring, parse_bytes, tmpfile) + format_bytes, get_ip_interface, parse_bytes, tmpfile) logger = logging.getLogger(__name__) docstrings = docrep.DocstringProcessor() diff --git a/dask_jobqueue/moab.py b/dask_jobqueue/moab.py index 7fbc0bc6..36e06595 100644 --- a/dask_jobqueue/moab.py +++ b/dask_jobqueue/moab.py @@ -30,7 +30,7 @@ class MoabCluster(PBSCluster): memory='16G', resource_spec='96G', job_extra=['-d /home/First.Last', '-M none'], local_directory=os.getenv('TMPDIR', '/tmp')) - >>> cluster.start_workers(10) # this may take a few seconds to launch + >>> cluster.start_workers(10) # submit enough jobs to deploy 10 workers >>> from dask.distributed import Client >>> client = Client(cluster) diff --git a/dask_jobqueue/pbs.py b/dask_jobqueue/pbs.py index 0eea7458..dd6188f2 100644 --- a/dask_jobqueue/pbs.py +++ b/dask_jobqueue/pbs.py @@ -35,7 +35,7 @@ class PBSCluster(JobQueueCluster): -------- >>> from dask_jobqueue import PBSCluster >>> cluster = PBSCluster(queue='regular', project='DaskOnPBS') - >>> cluster.start_workers(10) # this may take a few seconds to launch + >>> cluster.start_workers(10) # submit enough jobs to deploy 10 workers >>> from dask.distributed import Client >>> client = Client(cluster) diff --git a/dask_jobqueue/sge.py b/dask_jobqueue/sge.py index 995ac680..261350b4 100644 --- a/dask_jobqueue/sge.py +++ b/dask_jobqueue/sge.py @@ -30,7 +30,7 @@ class SGECluster(JobQueueCluster): -------- >>> from dask_jobqueue import SGECluster >>> cluster = SGECluster(queue='regular') - >>> cluster.start_workers(10) # this may take a few seconds to launch + >>> cluster.start_workers(10) # submit enough jobs to deploy 10 workers >>> from dask.distributed import Client >>> client = Client(cluster) diff --git a/dask_jobqueue/slurm.py b/dask_jobqueue/slurm.py index 6f7a4f65..f43e12e5 100644 --- a/dask_jobqueue/slurm.py +++ b/dask_jobqueue/slurm.py @@ -40,7 +40,7 @@ class SLURMCluster(JobQueueCluster): env_extra=['export LANG="en_US.utf8"', 'export LANGUAGE="en_US.utf8"', 'export LC_ALL="en_US.utf8"']) - >>> cluster.start_workers(10) # this may take a few seconds to launch + >>> cluster.start_workers(10) # submit enough jobs (2 in this case) to deploy 10 workers >>> from dask.distributed import Client >>> client = Client(cluster) diff --git a/docs/faq.rst b/docs/faq.rst deleted file mode 100644 index e2e1da64..00000000 --- a/docs/faq.rst +++ /dev/null @@ -1,13 +0,0 @@ -.. _faq: - -Frequently Asked Questions -========================== - -What is the difference between a Job and a Worker? --------------------------------------------------- - -In dask-distributed, a ``Worker`` is a Python object and node in a dask -``Cluster`` that serves two purposes, 1) serve data, and 2) perform\ -computations. Jobs are resources submitted to, and managed by, the job queueing -system (e.g. PBS, SGE, etc.). In dask-jobqueue, a single Job may include one or -more Workers. diff --git a/docs/index.rst b/docs/index.rst index 4667d033..ab445924 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -76,7 +76,7 @@ will specify how many jobs to deploy using the scale method. .. code-block:: python - cluster.scale(20) # launch twenty jobs of the specification provided above + cluster.scale(12) # launch 12 workers (2 jobs of 6 workers each) of the specification provided above Configuration Files ~~~~~~~~~~~~~~~~~~~ @@ -199,3 +199,12 @@ When the cluster object goes away, either because you delete it or because you close your Python program, it will send a signal to the workers to shut down. If for some reason this signal does not get through then workers will kill themselves after 60 seconds of waiting for a non-existent scheduler. + +Workers vs Jobs +--------------- + +In dask-distributed, a ``Worker`` is a Python object and node in a dask +``Cluster`` that serves two purposes, 1) serve data, and 2) perform +computations. ``Jobs`` are resources submitted to, and managed by, the job +queueing system (e.g. PBS, SGE, etc.). In dask-jobqueue, a single ``Job`` may +include one or more ``Workers``. From 8db65eb873f84b1d80428c6ad3766a6cb294caa3 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 10:41:37 -0700 Subject: [PATCH 40/59] update worker name template to use bracket delims --- dask_jobqueue/core.py | 15 +++------------ dask_jobqueue/jobqueue.yaml | 8 ++++---- dask_jobqueue/tests/test_pbs.py | 7 ++----- dask_jobqueue/tests/test_sge.py | 3 ++- dask_jobqueue/tests/test_slurm.py | 1 + 5 files changed, 12 insertions(+), 22 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index a2dede57..cdf4d041 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -34,14 +34,9 @@ def _job_id_from_worker_name(name): ''' utility to parse the job ID from the worker name - template: 'prefix-jobid[-proc]' + template: 'prefix[jobid]' ''' - pieces = name.split('-') - print(name, pieces) - if len(pieces) == 2: - return pieces[-1] - else: - return pieces[-2] + return name.split('[', 1)[1].split(']')[0] class JobQueuePlugin(SchedulerPlugin): @@ -194,10 +189,6 @@ def __init__(self, raise ValueError("You must specify how much memory to use per job " "like ``memory='24 GB'``") - if '-' in name: - raise ValueError( - 'name (%s) can not include the `-` character' % name) - #This attribute should be overriden self.job_header = None @@ -234,7 +225,7 @@ def __init__(self, if name is not None: # worker names follow this template: {NAME}-{JOB_ID} - self._command_template += " --name %s-${JOB_ID}" % name + self._command_template += " --name %s[${JOB_ID}]" % name if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: diff --git a/dask_jobqueue/jobqueue.yaml b/dask_jobqueue/jobqueue.yaml index a03ae767..b495f132 100644 --- a/dask_jobqueue/jobqueue.yaml +++ b/dask_jobqueue/jobqueue.yaml @@ -1,6 +1,6 @@ jobqueue: pbs: - name: dask_worker + name: dask-worker # Dask worker options cores: null # Total number of cores per job @@ -21,7 +21,7 @@ jobqueue: job-extra: [] sge: - name: dask_worker + name: dask-worker # Dask worker options cores: null # Total number of cores per job @@ -42,7 +42,7 @@ jobqueue: resource-spec: null slurm: - name: dask_worker + name: dask-worker # Dask worker options cores: null # Total number of cores per job @@ -64,7 +64,7 @@ jobqueue: job-extra: {} moab: - name: dask_worker + name: dask-worker # Dask worker options cores: null # Total number of cores per job diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index df851d0d..21078d12 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -144,6 +144,7 @@ def test_adaptive(loop): while cluster.pending_jobs or cluster.running_jobs: sleep(0.100) assert time() < start + QUEUE_WAIT + assert cluster.finished_jobs @pytest.mark.env("pbs") # noqa: F811 @@ -183,11 +184,7 @@ def test_adaptive_grouped(loop): while cluster.pending_jobs or cluster.running_jobs: sleep(0.100) assert time() < start + QUEUE_WAIT - - -def test_valid_worker_name(): - with pytest.raises(ValueError): - PBSCluster(name='dask-worker') + assert cluster.finished_jobs def test_config(loop): # noqa: F811 diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 3d6b89c9..9e1ad833 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -25,7 +25,7 @@ def test_basic(loop): # noqa: F811 assert w['memory_limit'] == 7e9 assert w['ncores'] == 2 - cluster.stop_workers(workers) + cluster.stop_workers(info['workers'].values()) start = time() while len(client.scheduler_info()['workers']) > 0: @@ -33,3 +33,4 @@ def test_basic(loop): # noqa: F811 assert time() < start + QUEUE_WAIT assert not cluster.running_jobs + assert cluster.finished_jobs diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 6d6eeaf4..4fbd7b9f 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -148,3 +148,4 @@ def test_adaptive(loop): while cluster.pending_jobs or cluster.running_jobs: sleep(0.100) assert time() < start + QUEUE_WAIT + assert cluster.finished_jobs From ef162984ef1ae06d7088b70e35d22a474daf3ff5 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 10:47:36 -0700 Subject: [PATCH 41/59] add a few more comments --- dask_jobqueue/core.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index cdf4d041..02def2e8 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -34,7 +34,7 @@ def _job_id_from_worker_name(name): ''' utility to parse the job ID from the worker name - template: 'prefix[jobid]' + template: 'prefix[jobid]suffix' ''' return name.split('[', 1)[1].split(']')[0] @@ -224,7 +224,9 @@ def __init__(self, self._command_template += " --memory-limit %s" % mem if name is not None: - # worker names follow this template: {NAME}-{JOB_ID} + # worker names follow this template: {NAME}[{JOB_ID}] + # {JOB_ID} is an environment variable defined by the individual + # job scrips/schedulers self._command_template += " --name %s[${JOB_ID}]" % name if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout From 38039189f81941c38866d42cf21343da6aa718f3 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 10:59:58 -0700 Subject: [PATCH 42/59] roll back changes in tests --- dask_jobqueue/tests/test_pbs.py | 10 +++++----- dask_jobqueue/tests/test_slurm.py | 4 ++-- docs/index.rst | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 21078d12..f385a312 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -16,7 +16,7 @@ def test_header(Cluster): with Cluster(walltime='00:02:00', processes=4, cores=8, memory='28GB') as cluster: assert '#PBS' in cluster.job_header - assert '#PBS -N dask_worker' in cluster.job_header + assert '#PBS -N dask-worker' in cluster.job_header assert '#PBS -l select=1:ncpus=8:mem=27GB' in cluster.job_header assert '#PBS -l walltime=00:02:00' in cluster.job_header assert '#PBS -q' not in cluster.job_header @@ -26,7 +26,7 @@ def test_header(Cluster): memory='28GB', resource_spec='select=1:ncpus=24:mem=100GB') as cluster: assert '#PBS -q regular' in cluster.job_header - assert '#PBS -N dask_worker' in cluster.job_header + assert '#PBS -N dask-worker' in cluster.job_header assert '#PBS -l select=1:ncpus=24:mem=100GB' in cluster.job_header assert '#PBS -l select=1:ncpus=8:mem=27GB' not in cluster.job_header assert '#PBS -l walltime=' in cluster.job_header @@ -57,7 +57,7 @@ def test_job_script(Cluster): job_script = cluster.job_script() assert '#PBS' in job_script - assert '#PBS -N dask_worker' in job_script + assert '#PBS -N dask-worker' in job_script assert '#PBS -l select=1:ncpus=8:mem=27GB' in job_script assert '#PBS -l walltime=00:02:00' in job_script assert '#PBS -q' not in job_script @@ -71,7 +71,7 @@ def test_job_script(Cluster): job_script = cluster.job_script() assert '#PBS -q regular' in job_script - assert '#PBS -N dask_worker' in job_script + assert '#PBS -N dask-worker' in job_script assert '#PBS -l select=1:ncpus=24:mem=100GB' in job_script assert '#PBS -l select=1:ncpus=8:mem=27GB' not in job_script assert '#PBS -l walltime=' in job_script @@ -152,7 +152,7 @@ def test_adaptive_grouped(loop): with PBSCluster(walltime='00:02:00', processes=2, threads=1, memory='2GB', local_directory='/tmp', job_extra=['-V'], loop=loop) as cluster: - cluster.adapt(minimum=1) + cluster.adapt(minimum=1) # at least 1 worker with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 4fbd7b9f..ea036cd1 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -14,7 +14,7 @@ def test_header(): with SLURMCluster(walltime='00:02:00', processes=4, cores=8, memory='28GB') as cluster: assert '#SBATCH' in cluster.job_header - assert '#SBATCH -J dask_worker' in cluster.job_header + assert '#SBATCH -J dask-worker' in cluster.job_header assert '#SBATCH -n 1' in cluster.job_header assert '#SBATCH --cpus-per-task=8' in cluster.job_header assert '#SBATCH --mem=27G' in cluster.job_header @@ -70,7 +70,7 @@ def test_job_script(): ) as cluster: job_script = cluster.job_script() assert '#SBATCH' in job_script - assert '#SBATCH -J dask_worker' in job_script + assert '#SBATCH -J dask-worker' in job_script assert '#SBATCH -n 1' in job_script assert '#SBATCH --cpus-per-task=8' in job_script assert '#SBATCH --mem=27G' in job_script diff --git a/docs/index.rst b/docs/index.rst index ab445924..ead3e5d8 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -37,7 +37,7 @@ save resources when not actively computing. .. code-block:: python - cluster.adapt(minimum=6, maximum=90) + cluster.adapt(minimum=6, maximum=90) # auto-scale between 6 and 90 workers Configuration From aad58d4cf051591ad29418f540e19a0a69ed8330 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 12:41:30 -0700 Subject: [PATCH 43/59] fix slurm tests and missing scheduler plugin after merge conflict resolution --- dask_jobqueue/core.py | 4 ++++ dask_jobqueue/tests/test_slurm.py | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 02def2e8..3ce882fe 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -207,6 +207,10 @@ def __init__(self, self.worker_cores = cores self.name = name + # plugin for tracking job status + self._scheduler_plugin = JobQueuePlugin() + self.local_cluster.scheduler.add_plugin(self._scheduler_plugin) + self._adaptive = None self._env_header = '\n'.join(env_extra) diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index ea036cd1..d46cb5fe 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -22,7 +22,7 @@ def test_header(): assert '#SBATCH -p' not in cluster.job_header assert '#SBATCH -A' not in cluster.job_header - with SLURMCluster(queue='regular', project='DaskOnPBS', processes=4, + with SLURMCluster(queue='regular', project='DaskOnSlurm', processes=4, cores=8, memory='28GB', job_cpu=16, job_mem='100G') as cluster: assert '#SBATCH --cpus-per-task=16' in cluster.job_header From fa1b71717ed8d08332ac2454434da9f4f0dae614 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Tue, 3 Jul 2018 13:28:47 -0700 Subject: [PATCH 44/59] fix threads in test --- dask_jobqueue/tests/test_pbs.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index f385a312..256fd5b1 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -149,7 +149,7 @@ def test_adaptive(loop): @pytest.mark.env("pbs") # noqa: F811 def test_adaptive_grouped(loop): - with PBSCluster(walltime='00:02:00', processes=2, threads=1, memory='2GB', + with PBSCluster(walltime='00:02:00', processes=1, cores=2, memory='2GB', local_directory='/tmp', job_extra=['-V'], loop=loop) as cluster: cluster.adapt(minimum=1) # at least 1 worker From aeea2e57bdbbef957889ad411ed2ce8dcf6279e9 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 6 Jul 2018 14:02:12 -0700 Subject: [PATCH 45/59] debug on travis --- dask_jobqueue/tests/test_sge.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 9e1ad833..b12f7c76 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -16,6 +16,15 @@ def test_basic(loop): # noqa: F811 with Client(cluster, loop=loop) as client: cluster.start_workers(2) assert cluster.pending_jobs or cluster.running_jobs + + start = time() + while not cluster.running_jobs: + sleep(0.100) + if time() > start + QUEUE_WAIT: + print('never got a job into the running catagory') + print('cluster.pending_jobs', cluster.pending_jobs) + print('cluster.running_jobs', cluster.running_jobs) + future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs From b93a7c444239d40f623ed33f9d3abda3747f3d63 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 6 Jul 2018 14:07:11 -0700 Subject: [PATCH 46/59] simplify tests --- dask_jobqueue/tests/test_pbs.py | 14 ++------------ dask_jobqueue/tests/test_sge.py | 2 +- dask_jobqueue/tests/test_slurm.py | 16 ++++------------ 3 files changed, 7 insertions(+), 25 deletions(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 256fd5b1..2451b3e4 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -118,12 +118,7 @@ def test_adaptive(loop): assert future.result(QUEUE_WAIT) == 11 start = time() - while not len(cluster.pending_jobs): - sleep(0.100) - assert time() < start + QUEUE_WAIT - - start = time() - while not len(cluster.running_jobs): + while not cluster.running_jobs: sleep(0.100) assert time() < start + QUEUE_WAIT @@ -158,12 +153,7 @@ def test_adaptive_grouped(loop): assert future.result(QUEUE_WAIT) == 11 start = time() - while not len(cluster.pending_jobs): - sleep(0.100) - assert time() < start + QUEUE_WAIT - - start = time() - while not len(cluster.running_jobs): + while not cluster.running_jobs: sleep(0.100) assert time() < start + QUEUE_WAIT diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index b12f7c76..52dda059 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -37,7 +37,7 @@ def test_basic(loop): # noqa: F811 cluster.stop_workers(info['workers'].values()) start = time() - while len(client.scheduler_info()['workers']) > 0: + while client.scheduler_info()['workers']: sleep(0.100) assert time() < start + QUEUE_WAIT diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index d46cb5fe..50a9b669 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -105,7 +105,7 @@ def test_basic(loop): cluster.stop_workers(workers) start = time() - while len(client.scheduler_info()['workers']) > 0: + while client.scheduler_info()['workers']: sleep(0.100) assert time() < start + QUEUE_WAIT @@ -120,16 +120,8 @@ def test_adaptive(loop): with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 - - start = time() - while not len(cluster.pending_jobs): - sleep(0.100) - assert time() < start + QUEUE_WAIT - - start = time() - while not len(cluster.running_jobs): - sleep(0.100) - assert time() < start + QUEUE_WAIT + assert (cluster.pending_jobs or cluster.running_jobs or + cluster.finished_jobs) start = time() processes = cluster.worker_processes @@ -140,7 +132,7 @@ def test_adaptive(loop): del future start = time() - while len(client.scheduler_info()['workers']) > 0: + while client.scheduler_info()['workers']: sleep(0.100) assert time() < start + QUEUE_WAIT From 0a2e304ea7209ba790b1a5dfa542d945ebd02492 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 6 Jul 2018 17:25:29 -0700 Subject: [PATCH 47/59] more debugging and more robust sleeps --- ci/none.sh | 2 -- ci/pbs.sh | 2 -- ci/sge.sh | 2 -- ci/slurm.sh | 2 -- dask_jobqueue/tests/test_pbs.py | 24 ++++++++---------------- dask_jobqueue/tests/test_sge.py | 8 -------- 6 files changed, 8 insertions(+), 32 deletions(-) diff --git a/ci/none.sh b/ci/none.sh index 1c67925e..fb1f9647 100644 --- a/ci/none.sh +++ b/ci/none.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -set -x - function jobqueue_before_install { # Install miniconda ./ci/conda_setup.sh diff --git a/ci/pbs.sh b/ci/pbs.sh index 715c45d1..d3f778be 100644 --- a/ci/pbs.sh +++ b/ci/pbs.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -set -x - function jobqueue_before_install { docker version docker-compose version diff --git a/ci/sge.sh b/ci/sge.sh index c4ed8a54..21975b0b 100644 --- a/ci/sge.sh +++ b/ci/sge.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -set -x - function jobqueue_before_install { docker version docker-compose version diff --git a/ci/slurm.sh b/ci/slurm.sh index d61a89b4..ece07dd8 100644 --- a/ci/slurm.sh +++ b/ci/slurm.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -set -x - function jobqueue_before_install { docker version docker-compose version diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 2451b3e4..b7b50a1d 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -114,14 +114,14 @@ def test_adaptive(loop): cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) - assert cluster.pending_jobs or cluster.running_jobs - assert future.result(QUEUE_WAIT) == 11 start = time() - while not cluster.running_jobs: + while not (cluster.pending_jobs or cluster.running_jobs): sleep(0.100) assert time() < start + QUEUE_WAIT + assert future.result(QUEUE_WAIT) == 11 + start = time() processes = cluster.worker_processes while len(client.scheduler_info()['workers']) != processes: @@ -149,6 +149,11 @@ def test_adaptive_grouped(loop): loop=loop) as cluster: cluster.adapt(minimum=1) # at least 1 worker with Client(cluster) as client: + start = time() + while not (cluster.pending_jobs or cluster.running_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT + future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 @@ -163,19 +168,6 @@ def test_adaptive_grouped(loop): sleep(0.1) assert time() < start + QUEUE_WAIT - del future - - start = time() - while len(client.scheduler_info()['workers']) > 0: - sleep(0.100) - assert time() < start + QUEUE_WAIT - - start = time() - while cluster.pending_jobs or cluster.running_jobs: - sleep(0.100) - assert time() < start + QUEUE_WAIT - assert cluster.finished_jobs - def test_config(loop): # noqa: F811 with dask.config.set({'jobqueue.pbs.walltime': '00:02:00', diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 52dda059..68ad7338 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -17,14 +17,6 @@ def test_basic(loop): # noqa: F811 cluster.start_workers(2) assert cluster.pending_jobs or cluster.running_jobs - start = time() - while not cluster.running_jobs: - sleep(0.100) - if time() > start + QUEUE_WAIT: - print('never got a job into the running catagory') - print('cluster.pending_jobs', cluster.pending_jobs) - print('cluster.running_jobs', cluster.running_jobs) - future = client.submit(lambda x: x + 1, 10) assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs From 1a1fe75849eb364c6fdadaa79a8562298f403179 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 6 Jul 2018 17:45:30 -0700 Subject: [PATCH 48/59] unify basic test --- dask_jobqueue/tests/test_pbs.py | 2 +- dask_jobqueue/tests/test_sge.py | 13 ++++++------- dask_jobqueue/tests/test_slurm.py | 4 ++-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index b7b50a1d..238326c0 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -100,7 +100,7 @@ def test_basic(loop): cluster.stop_workers(workers) start = time() - while len(client.scheduler_info()['workers']) > 0: + while client.scheduler_info()['workers']: sleep(0.100) assert time() < start + QUEUE_WAIT diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index 68ad7338..fbb2eaec 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -11,7 +11,7 @@ @pytest.mark.env("sge") # noqa: F811 def test_basic(loop): # noqa: F811 - with SGECluster(walltime='00:02:00', cores=8, processes=4, memory='28GB', + with SGECluster(walltime='00:02:00', cores=8, processes=4, memory='2GB', loop=loop) as cluster: with Client(cluster, loop=loop) as client: cluster.start_workers(2) @@ -21,12 +21,12 @@ def test_basic(loop): # noqa: F811 assert future.result(QUEUE_WAIT) == 11 assert cluster.running_jobs - info = client.scheduler_info() - for w in info['workers'].values(): - assert w['memory_limit'] == 7e9 - assert w['ncores'] == 2 + workers = list(client.scheduler_info()['workers'].values()) + w = workers[0] + assert w['memory_limit'] == 2e9 + assert w['ncores'] == 2 - cluster.stop_workers(info['workers'].values()) + cluster.stop_workers(workers) start = time() while client.scheduler_info()['workers']: @@ -34,4 +34,3 @@ def test_basic(loop): # noqa: F811 assert time() < start + QUEUE_WAIT assert not cluster.running_jobs - assert cluster.finished_jobs diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 50a9b669..2ffdc387 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -88,7 +88,7 @@ def test_job_script(): @pytest.mark.env("slurm") # noqa: F811 def test_basic(loop): - with SLURMCluster(walltime='00:02:00', cores=2, processes=1, memory='4GB', + with SLURMCluster(walltime='00:02:00', cores=2, processes=1, memory='2GB', job_extra=['-D /'], loop=loop) as cluster: with Client(cluster) as client: cluster.start_workers(2) @@ -99,7 +99,7 @@ def test_basic(loop): workers = list(client.scheduler_info()['workers'].values()) w = workers[0] - assert w['memory_limit'] == 4e9 + assert w['memory_limit'] == 2e9 assert w['ncores'] == 2 cluster.stop_workers(workers) From 8c32872d1e6109948c97f867734005d5e5647d24 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 6 Jul 2018 17:46:10 -0700 Subject: [PATCH 49/59] unify adaptive tests --- dask_jobqueue/tests/test_slurm.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index 2ffdc387..dc31d821 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -114,14 +114,18 @@ def test_basic(loop): @pytest.mark.env("slurm") # noqa: F811 def test_adaptive(loop): - with SLURMCluster(walltime='00:02:00', cores=2, processes=1, memory='4GB', + with SLURMCluster(walltime='00:02:00', cores=2, processes=1, memory='2GB', job_extra=['-D /'], loop=loop) as cluster: cluster.adapt() with Client(cluster) as client: future = client.submit(lambda x: x + 1, 10) + + start = time() + while not (cluster.pending_jobs or cluster.running_jobs): + sleep(0.100) + assert time() < start + QUEUE_WAIT + assert future.result(QUEUE_WAIT) == 11 - assert (cluster.pending_jobs or cluster.running_jobs or - cluster.finished_jobs) start = time() processes = cluster.worker_processes @@ -132,7 +136,7 @@ def test_adaptive(loop): del future start = time() - while client.scheduler_info()['workers']: + while len(client.scheduler_info()['workers']) > 0: sleep(0.100) assert time() < start + QUEUE_WAIT From ee89f2022e38ac165c99c08a70601cb3c7ebcbb8 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Mon, 9 Jul 2018 14:43:38 -0600 Subject: [PATCH 50/59] debug statements and some nice fixups --- dask_jobqueue/core.py | 16 ++++++++++------ dask_jobqueue/slurm.py | 5 +++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 6a7d30a3..ef319264 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -17,7 +17,7 @@ format_bytes, get_ip_interface, parse_bytes, tmpfile) logger = logging.getLogger(__name__) -logger.setLevel(10) +logger.setLevel(logging.INFO) docstrings = docrep.DocstringProcessor() @@ -331,6 +331,7 @@ def _call(self, cmd): def stop_workers(self, workers): """ Stop a list of workers""" + logger.debug("Stopping workers: %s" % workers) if not workers: return jobs = [] @@ -339,22 +340,25 @@ def stop_workers(self, workers): jobs.append(_job_id_from_worker_name(w['name'])) else: jobs.append(_job_id_from_worker_name(w.name)) - self.stop_jobs(jobs) + self.stop_jobs(set(jobs)) def stop_jobs(self, jobs): """ Stop a list of jobs""" + logger.debug("Stopping jobs: %s" % jobs) if jobs: - self._call([self.cancel_command] + list(jobs)) + self._call([self.cancel_command] + list(set(jobs))) def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ - active_and_pending = sum([len(j.workers) for j in - self.running_jobs.values()]) + logger.debug("Scaling up to %d workers." % n) + active_and_pending = sum([len(j) for j in self.running_jobs.values()]) active_and_pending += self.worker_processes * len(self.pending_jobs) + logger.debug("Found %d active/pending workers." % active_and_pending) self.start_workers(n - active_and_pending) def scale_down(self, workers): ''' Close the workers with the given addresses ''' + logger.debug("Scaling down. Workers: %s" % workers) worker_states = [] for w in workers: try: @@ -369,7 +373,7 @@ def __enter__(self): def __exit__(self, type, value, traceback): jobs = list(self.pending_jobs.keys()) + list(self.running_jobs.keys()) - self.stop_jobs(jobs) + self.stop_jobs(set(jobs)) self.local_cluster.__exit__(type, value, traceback) def _job_id_from_submit_output(self, out): diff --git a/dask_jobqueue/slurm.py b/dask_jobqueue/slurm.py index 872038e8..6ffbe79a 100644 --- a/dask_jobqueue/slurm.py +++ b/dask_jobqueue/slurm.py @@ -51,7 +51,7 @@ class SLURMCluster(JobQueueCluster): >>> cluster.adapt() """, 4) - #Override class variables + # Override class variables submit_command = 'sbatch --parsable' cancel_command = 'scancel' scheduler_name = 'slurm' @@ -98,9 +98,10 @@ def __init__(self, queue=None, project=None, walltime=None, if walltime is not None: header_lines.append('#SBATCH -t %s' % walltime) - header_lines.append('JOB_ID=${SLURM_JOB_ID%;*}') header_lines.extend(['#SBATCH %s' % arg for arg in job_extra]) + header_lines.append('JOB_ID=${SLURM_JOB_ID%;*}') + # Declare class attribute that shall be overriden self.job_header = '\n'.join(header_lines) From 4abcea1bf93ba509ca35a7f5b7893339dc243e10 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 13 Jul 2018 15:47:10 -0500 Subject: [PATCH 51/59] future div --- dask_jobqueue/core.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index ef319264..95d91ad8 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import, division, print_function + import logging import math import shlex @@ -17,7 +19,6 @@ format_bytes, get_ip_interface, parse_bytes, tmpfile) logger = logging.getLogger(__name__) -logger.setLevel(logging.INFO) docstrings = docrep.DocstringProcessor() From 0c7425a456d2d1425f551ad40c3a6130d059b3d7 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 13 Jul 2018 15:53:31 -0500 Subject: [PATCH 52/59] add logging stuff --- dask_jobqueue/core.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 95d91ad8..bb61e80b 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -19,6 +19,7 @@ format_bytes, get_ip_interface, parse_bytes, tmpfile) logger = logging.getLogger(__name__) +logger.setLevel(logging.DEBUG) docstrings = docrep.DocstringProcessor() @@ -50,12 +51,15 @@ def __init__(self): def add_worker(self, scheduler, worker=None, name=None, **kwargs): ''' Run when a new worker enters the cluster''' + logger.debug("adding worker %s" % worker) w = scheduler.workers[worker] job_id = _job_id_from_worker_name(w.name) + logger.debug("job id for new worker: %s" % job_id) self.all_workers[worker] = (w.name, job_id) # if this is the first worker for this job, move job to running if job_id not in self.running_jobs: + logger.debug("this is a new job") self.running_jobs[job_id] = self.pending_jobs.pop(job_id) # add worker to dict of workers in this job @@ -63,13 +67,16 @@ def add_worker(self, scheduler, worker=None, name=None, **kwargs): def remove_worker(self, scheduler=None, worker=None, **kwargs): ''' Run when a worker leaves the cluster''' + logger.debug("removing worker %s" % worker) name, job_id = self.all_workers[worker] + logger.debug("removing worker name (%s) and job_id (%s)" % (name, job_id)) # remove worker from this job del self.running_jobs[job_id][name] # once there are no more workers, move this job to finished_jobs if not self.running_jobs[job_id]: + logger.debug("that was the last worker for job %s" % job_id) self.finished_jobs[job_id] = self.running_jobs.pop(job_id) From 5d4552d011d1227b471a2c2352644975d3f4bdd8 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 13 Jul 2018 16:03:48 -0500 Subject: [PATCH 53/59] -s for pytest --- ci/pbs.sh | 2 +- ci/sge.sh | 2 +- ci/slurm.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ci/pbs.sh b/ci/pbs.sh index d3f778be..44d02a73 100644 --- a/ci/pbs.sh +++ b/ci/pbs.sh @@ -19,7 +19,7 @@ function jobqueue_install { } function jobqueue_script { - docker exec -it -u pbsuser pbs_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E pbs" + docker exec -it -u pbsuser pbs_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E pbs -s" } function jobqueue_after_script { diff --git a/ci/sge.sh b/ci/sge.sh index 21975b0b..733d68b3 100644 --- a/ci/sge.sh +++ b/ci/sge.sh @@ -18,7 +18,7 @@ function jobqueue_install { } function jobqueue_script { - docker exec -it sge_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E sge" + docker exec -it sge_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E sge -s" } function jobqueue_after_script { diff --git a/ci/slurm.sh b/ci/slurm.sh index ece07dd8..0ff19dc5 100644 --- a/ci/slurm.sh +++ b/ci/slurm.sh @@ -18,7 +18,7 @@ function jobqueue_install { } function jobqueue_script { - docker exec -it slurmctld /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E slurm" + docker exec -it slurmctld /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E slurm -s" } function jobqueue_after_script { From 8a150c997a547a03e735f57e0a421762dde1e93a Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Fri, 13 Jul 2018 16:46:13 -0500 Subject: [PATCH 54/59] use --job_id-- for name --- dask_jobqueue/core.py | 9 +++------ dask_jobqueue/tests/test_pbs.py | 4 +++- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index bb61e80b..73b97888 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -39,7 +39,8 @@ def _job_id_from_worker_name(name): template: 'prefix[jobid]suffix' ''' - return name.split('[', 1)[1].split(']')[0] + _, job_id, _ = name.split('--') + return job_id class JobQueuePlugin(SchedulerPlugin): @@ -235,12 +236,8 @@ def __init__(self, mem = format_bytes(self.worker_memory / self.worker_processes) mem = mem.replace(' ', '') self._command_template += " --memory-limit %s" % mem + self._command_template += " --name %s--${JOB_ID}--" % name - if name is not None: - # worker names follow this template: {NAME}[{JOB_ID}] - # {JOB_ID} is an environment variable defined by the individual - # job scrips/schedulers - self._command_template += " --name %s[${JOB_ID}]" % name if death_timeout is not None: self._command_template += " --death-timeout %s" % death_timeout if local_directory is not None: diff --git a/dask_jobqueue/tests/test_pbs.py b/dask_jobqueue/tests/test_pbs.py index 238326c0..d9a2f08b 100644 --- a/dask_jobqueue/tests/test_pbs.py +++ b/dask_jobqueue/tests/test_pbs.py @@ -13,7 +13,8 @@ @pytest.mark.parametrize('Cluster', [PBSCluster, MoabCluster]) def test_header(Cluster): - with Cluster(walltime='00:02:00', processes=4, cores=8, memory='28GB') as cluster: + with Cluster(walltime='00:02:00', processes=4, cores=8, memory='28GB', + name='dask-worker') as cluster: assert '#PBS' in cluster.job_header assert '#PBS -N dask-worker' in cluster.job_header @@ -21,6 +22,7 @@ def test_header(Cluster): assert '#PBS -l walltime=00:02:00' in cluster.job_header assert '#PBS -q' not in cluster.job_header assert '#PBS -A' not in cluster.job_header + assert '--name dask-worker--${JOB_ID}--' in cluster.job_script() with Cluster(queue='regular', project='DaskOnPBS', processes=4, cores=8, memory='28GB', resource_spec='select=1:ncpus=24:mem=100GB') as cluster: From ca0c72779c79f4452ba4ac7345bca25bac583e28 Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Fri, 13 Jul 2018 17:00:08 -0500 Subject: [PATCH 55/59] fix memory in sge tests --- dask_jobqueue/tests/test_sge.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask_jobqueue/tests/test_sge.py b/dask_jobqueue/tests/test_sge.py index e7a05f30..ffdd6760 100644 --- a/dask_jobqueue/tests/test_sge.py +++ b/dask_jobqueue/tests/test_sge.py @@ -24,7 +24,7 @@ def test_basic(loop): # noqa: F811 workers = list(client.scheduler_info()['workers'].values()) w = workers[0] - assert w['memory_limit'] == 2e9 + assert w['memory_limit'] == 2e9 / 4 assert w['ncores'] == 2 cluster.stop_workers(workers) From ce007dfa3e0553a7460bc54ef0678431f8c4a65e Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 13 Jul 2018 19:06:37 -0500 Subject: [PATCH 56/59] remove pending jobs when scaling down --- dask_jobqueue/core.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 73b97888..79c4d39a 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -276,11 +276,13 @@ def job_file(self): """ Write job submission script to temporary file """ with tmpfile(extension='sh') as fn: with open(fn, 'w') as f: + logger.debug(self.job_script()) f.write(self.job_script()) yield fn def start_workers(self, n=1): """ Start workers and point them to our local scheduler """ + logger.debug('starting %s workers' % n) num_jobs = math.ceil(n / self.worker_processes) for _ in range(num_jobs): with self.job_file() as fn: @@ -340,18 +342,29 @@ def stop_workers(self, workers): if not workers: return jobs = [] + if self.pending_jobs: + import pdb + pdb.set_trace() for w in workers: if isinstance(w, dict): jobs.append(_job_id_from_worker_name(w['name'])) else: jobs.append(_job_id_from_worker_name(w.name)) - self.stop_jobs(set(jobs)) + self.stop_jobs(set(jobs + list(self.pending_jobs.keys()))) def stop_jobs(self, jobs): """ Stop a list of jobs""" logger.debug("Stopping jobs: %s" % jobs) + # why set with empty string in jobs[0] if jobs: + jobs = list(jobs) + if len(jobs) == 1 and not jobs[0]: + import pdb + pdb.set_trace() self._call([self.cancel_command] + list(set(jobs))) + else: + import pdb + pdb.set_trace() def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ From c23ce7cbdb0a268a7da9c3276fa346dc9b9f9ceb Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Fri, 13 Jul 2018 19:28:37 -0500 Subject: [PATCH 57/59] remove pending jobs --- ci/pbs.sh | 2 +- dask_jobqueue/core.py | 15 ++++----------- dask_jobqueue/tests/test_slurm.py | 1 - 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/ci/pbs.sh b/ci/pbs.sh index 44d02a73..87a87f97 100644 --- a/ci/pbs.sh +++ b/ci/pbs.sh @@ -15,7 +15,7 @@ function jobqueue_before_install { } function jobqueue_install { - docker exec -it pbs_master /bin/bash -c "cd /dask-jobqueue; python setup.py install" + docker exec -it pbs_master /bin/bash -c "cd /dask-jobqueue; pip install -e ." } function jobqueue_script { diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index 79c4d39a..afd4b4f6 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -70,7 +70,8 @@ def remove_worker(self, scheduler=None, worker=None, **kwargs): ''' Run when a worker leaves the cluster''' logger.debug("removing worker %s" % worker) name, job_id = self.all_workers[worker] - logger.debug("removing worker name (%s) and job_id (%s)" % (name, job_id)) + logger.debug("removing worker name (%s) and" + "job_id (%s)" % (name, job_id)) # remove worker from this job del self.running_jobs[job_id][name] @@ -341,10 +342,8 @@ def stop_workers(self, workers): logger.debug("Stopping workers: %s" % workers) if not workers: return - jobs = [] - if self.pending_jobs: - import pdb - pdb.set_trace() + jobs = list(self.pending_jobs.keys()) # stop pending jobs too + logger.debug("Stopping pending jobs %s" % jobs) for w in workers: if isinstance(w, dict): jobs.append(_job_id_from_worker_name(w['name'])) @@ -358,13 +357,7 @@ def stop_jobs(self, jobs): # why set with empty string in jobs[0] if jobs: jobs = list(jobs) - if len(jobs) == 1 and not jobs[0]: - import pdb - pdb.set_trace() self._call([self.cancel_command] + list(set(jobs))) - else: - import pdb - pdb.set_trace() def scale_up(self, n, **kwargs): """ Brings total worker count up to ``n`` """ diff --git a/dask_jobqueue/tests/test_slurm.py b/dask_jobqueue/tests/test_slurm.py index a52e8199..dc31d821 100644 --- a/dask_jobqueue/tests/test_slurm.py +++ b/dask_jobqueue/tests/test_slurm.py @@ -49,7 +49,6 @@ def test_job_script(): memory='28GB') as cluster: job_script = cluster.job_script() - print(job_script) assert '#SBATCH' in job_script assert '#SBATCH -J dask-worker' in job_script assert '--memory-limit 7.00GB ' in job_script From 76184678d961b2bd371961f330740bfe6bd9b644 Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Sat, 14 Jul 2018 22:23:13 -0700 Subject: [PATCH 58/59] cleanup after lots of debugging --- dask_jobqueue/core.py | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index afd4b4f6..e24f2bfd 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -19,7 +19,6 @@ format_bytes, get_ip_interface, parse_bytes, tmpfile) logger = logging.getLogger(__name__) -logger.setLevel(logging.DEBUG) docstrings = docrep.DocstringProcessor() @@ -37,7 +36,7 @@ def _job_id_from_worker_name(name): ''' utility to parse the job ID from the worker name - template: 'prefix[jobid]suffix' + template: 'prefix--jobid--suffix' ''' _, job_id, _ = name.split('--') return job_id @@ -277,7 +276,7 @@ def job_file(self): """ Write job submission script to temporary file """ with tmpfile(extension='sh') as fn: with open(fn, 'w') as f: - logger.debug(self.job_script()) + logger.debug("writing job script: \n%s" % self.job_script()) f.write(self.job_script()) yield fn @@ -289,7 +288,8 @@ def start_workers(self, n=1): with self.job_file() as fn: out = self._call(shlex.split(self.submit_command) + [fn]) job = self._job_id_from_submit_output(out.decode()) - self._scheduler_plugin.pending_jobs[job] = {} + logger.debug("started job: %s" % job) + self.pending_jobs[job] = {} @property def scheduler(self): @@ -342,19 +342,17 @@ def stop_workers(self, workers): logger.debug("Stopping workers: %s" % workers) if not workers: return - jobs = list(self.pending_jobs.keys()) # stop pending jobs too - logger.debug("Stopping pending jobs %s" % jobs) + jobs = self._stop_pending_jobs() # stop pending jobs too for w in workers: if isinstance(w, dict): jobs.append(_job_id_from_worker_name(w['name'])) else: jobs.append(_job_id_from_worker_name(w.name)) - self.stop_jobs(set(jobs + list(self.pending_jobs.keys()))) + self.stop_jobs(set(jobs)) def stop_jobs(self, jobs): """ Stop a list of jobs""" logger.debug("Stopping jobs: %s" % jobs) - # why set with empty string in jobs[0] if jobs: jobs = list(jobs) self._call([self.cancel_command] + list(set(jobs))) @@ -383,10 +381,18 @@ def __enter__(self): return self def __exit__(self, type, value, traceback): - jobs = list(self.pending_jobs.keys()) + list(self.running_jobs.keys()) + jobs = self._stop_pending_jobs() + jobs += list(self.running_jobs.keys()) self.stop_jobs(set(jobs)) self.local_cluster.__exit__(type, value, traceback) + def _stop_pending_jobs(self): + jobs = list(self.pending_jobs.keys()) + logger.debug("Stopping pending jobs %s" % jobs) + for job_id in jobs: + del self.pending_jobs[job_id] + return jobs + def _job_id_from_submit_output(self, out): raise NotImplementedError('_job_id_from_submit_output must be ' 'implemented when JobQueueCluster is ' From d5e42b35665c0d12b97c70369c86dfcc16dca6db Mon Sep 17 00:00:00 2001 From: Joseph Hamman Date: Sun, 15 Jul 2018 19:42:42 -0700 Subject: [PATCH 59/59] additional cleanup --- ci/pbs.sh | 2 +- ci/sge.sh | 4 ++-- ci/slurm.sh | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/ci/pbs.sh b/ci/pbs.sh index 87a87f97..d35465ab 100644 --- a/ci/pbs.sh +++ b/ci/pbs.sh @@ -19,7 +19,7 @@ function jobqueue_install { } function jobqueue_script { - docker exec -it -u pbsuser pbs_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E pbs -s" + docker exec -it -u pbsuser pbs_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E pbs" } function jobqueue_after_script { diff --git a/ci/sge.sh b/ci/sge.sh index 733d68b3..50cf5518 100644 --- a/ci/sge.sh +++ b/ci/sge.sh @@ -14,11 +14,11 @@ function jobqueue_before_install { } function jobqueue_install { - docker exec -it sge_master /bin/bash -c "cd /dask-jobqueue; python setup.py install" + docker exec -it sge_master /bin/bash -c "cd /dask-jobqueue; pip install -e ." } function jobqueue_script { - docker exec -it sge_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E sge -s" + docker exec -it sge_master /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E sge" } function jobqueue_after_script { diff --git a/ci/slurm.sh b/ci/slurm.sh index 0ff19dc5..cdc88852 100644 --- a/ci/slurm.sh +++ b/ci/slurm.sh @@ -14,11 +14,11 @@ function jobqueue_before_install { } function jobqueue_install { - docker exec -it slurmctld /bin/bash -c "cd /dask-jobqueue; python setup.py install" + docker exec -it slurmctld /bin/bash -c "cd /dask-jobqueue; pip install -e ." } function jobqueue_script { - docker exec -it slurmctld /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E slurm -s" + docker exec -it slurmctld /bin/bash -c "cd /dask-jobqueue; py.test dask_jobqueue --verbose -E slurm" } function jobqueue_after_script {