Skip to content

Commit

Permalink
Track pending, running, and failed jobs (#63)
Browse files Browse the repository at this point in the history
This tracks the pending, running, and failed jobs on the cluster. 
It does this by using a naming convention for the dask worker name,
and by installing a SchedulerPlugin to watch for when these names come in.

This helps to improve the adaptive experience with dask-jobqueue
  • Loading branch information
Joe Hamman authored and mrocklin committed Jul 16, 2018
1 parent 4816a25 commit f7c565a
Show file tree
Hide file tree
Showing 17 changed files with 271 additions and 126 deletions.
2 changes: 0 additions & 2 deletions ci/none.sh
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
#!/usr/bin/env bash

set -x

function jobqueue_before_install {
# Install miniconda
./ci/conda_setup.sh
Expand Down
4 changes: 1 addition & 3 deletions ci/pbs.sh
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
#!/usr/bin/env bash

set -x

function jobqueue_before_install {
docker version
docker-compose version
Expand All @@ -17,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 {
Expand Down
4 changes: 1 addition & 3 deletions ci/sge.sh
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
#!/usr/bin/env bash

set -x

function jobqueue_before_install {
docker version
docker-compose version
Expand All @@ -16,7 +14,7 @@ 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 {
Expand Down
4 changes: 1 addition & 3 deletions ci/slurm.sh
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
#!/usr/bin/env bash

set -x

function jobqueue_before_install {
docker version
docker-compose version
Expand All @@ -16,7 +14,7 @@ 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 {
Expand Down
3 changes: 1 addition & 2 deletions dask_jobqueue/config.py
Original file line number Diff line number Diff line change
@@ -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)

Expand Down
169 changes: 133 additions & 36 deletions dask_jobqueue/core.py
Original file line number Diff line number Diff line change
@@ -1,17 +1,22 @@
from contextlib import contextmanager
from __future__ import absolute_import, division, print_function

import logging
import math
import shlex
import socket
import subprocess
import sys
import warnings
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,
format_bytes)
from distributed.diagnostics.plugin import SchedulerPlugin
from distributed.utils import (
format_bytes, get_ip_interface, parse_bytes, tmpfile)

logger = logging.getLogger(__name__)
docstrings = docrep.DocstringProcessor()
Expand All @@ -28,6 +33,54 @@
""".strip()


def _job_id_from_worker_name(name):
''' utility to parse the job ID from the worker name
template: 'prefix--jobid--suffix'
'''
_, job_id, _ = name.split('--')
return job_id


class JobQueuePlugin(SchedulerPlugin):
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'''
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
self.running_jobs[job_id][w.name] = w

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)


@docstrings.get_sectionsf('JobQueueCluster')
class JobQueueCluster(Cluster):
""" Base class to launch Dask Clusters for Job queues
Expand Down Expand Up @@ -87,6 +140,8 @@ class JobQueueCluster(Cluster):
submit_command = None
cancel_command = None
scheduler_name = ''
_adaptive_options = {
'worker_key': lambda ws: _job_id_from_worker_name(ws.name)}

def __init__(self,
name=None,
Expand Down Expand Up @@ -155,15 +210,17 @@ def __init__(self,

self.local_cluster = LocalCluster(n_workers=0, ip=host, **kwargs)

# Keep information on process, cores, and memory, for use in subclasses
self.worker_memory = parse_bytes(memory)

# Keep information on process, threads and memory, for use in
# subclasses
self.worker_memory = parse_bytes(memory) if memory is not None else None
self.worker_processes = processes
self.worker_cores = cores
self.name = name

self.jobs = dict()
self.n = 0
# 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)
Expand All @@ -179,47 +236,60 @@ 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:
self._command_template += " --name %s" % name
self._command_template += "-%(n)d" # Keep %(n) to be replaced later
if death_timeout is not None:
self._command_template += " --death-timeout %s" % death_timeout
if local_directory is not None:
self._command_template += " --local-directory %s" % local_directory
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

@property
def worker_threads(self):
return int(self.worker_cores / self.worker_processes)

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):
""" Write job submission script to temporary file """
with tmpfile(extension='sh') as fn:
with open(fn, 'w') as f:
logger.debug("writing job script: \n%s" % 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 """
workers = []
for _ in range(n):
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:
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
logger.debug("started job: %s" % job)
self.pending_jobs[job] = {}

@property
def scheduler(self):
Expand Down Expand Up @@ -248,12 +318,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:
Expand All @@ -269,33 +339,60 @@ def _call(self, cmd):

def stop_workers(self, workers):
""" Stop a list of workers"""
logger.debug("Stopping workers: %s" % 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))
jobs = self._stop_pending_jobs() # stop pending jobs too
for w in workers:
with ignoring(KeyError):
del self.jobs[w]
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))

def stop_jobs(self, jobs):
""" Stop a list of jobs"""
logger.debug("Stopping jobs: %s" % jobs)
if jobs:
jobs = list(jobs)
self._call([self.cancel_command] + list(set(jobs)))

def scale_up(self, n, **kwargs):
""" Brings total worker count up to ``n`` """
return self.start_workers(n - len(self.jobs))
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 '''
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)
logger.debug("Scaling down. Workers: %s" % workers)
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 = 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 '
Expand Down
2 changes: 1 addition & 1 deletion dask_jobqueue/moab.py
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 5 additions & 1 deletion dask_jobqueue/pbs.py
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,10 @@ def __init__(self, queue=None, project=None, resource_spec=None, walltime=None,
# Instantiate args and parameters from parent abstract class
super(PBSCluster, self).__init__(**kwargs)

header_lines = []
# Try to find a project name from environment variable
project = project or os.environ.get('PBS_ACCOUNT')

header_lines = ['#!/usr/bin/env bash']
# PBS header build
if self.name is not None:
header_lines.append('#PBS -N %s' % self.name)
Expand All @@ -95,6 +98,7 @@ def __init__(self, queue=None, project=None, resource_spec=None, walltime=None,
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)
Expand Down
5 changes: 3 additions & 2 deletions dask_jobqueue/sge.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
from __future__ import absolute_import, division, print_function

import logging

import dask
Expand Down Expand Up @@ -56,8 +58,7 @@ def __init__(self, queue=None, project=None, resource_spec=None, walltime=None,

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:
Expand Down

0 comments on commit f7c565a

Please sign in to comment.