-
-
Notifications
You must be signed in to change notification settings - Fork 144
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add lsf #78
Merged
Merged
Add lsf #78
Changes from all commits
Commits
Show all changes
23 commits
Select commit
Hold shift + click to select a range
caf89e2
rebase
9a5e5c8
update local_cluster
1e47f4d
update thread to cores
b6d248a
add comma
7fbd76a
typos
72f195c
one more typo
f9970b3
add cores and memory parameters to config test
702b1ec
move submit_job to lsf.py
d115ff4
add self.shell in base classes
d5f691b
move shell to class variable
20616b6
update comment
c87f02e
slighter better explanation
5f370d5
update comment
462110a
clean up. add kill_jobs
7db4ec8
Merge branch 'master' of github.com:dask/dask-jobqueue into add-lsf
5740d45
rebase
a56c616
remove empty lines
2a61e9e
rename JOB_ID var to LSB_JOBID
c1f85fd
supress Job is submitted to <PROJECT> project.
8b872b3
suppress Job is submitted to <PROJECT> project in bsub command
fc29249
Cealn up. Use **kwargs in _call
ccc61cb
rm stop_jobs for lsy.py
22cbb60
updates docs with lsf
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,131 @@ | ||
from __future__ import absolute_import, division, print_function | ||
|
||
import logging | ||
import math | ||
|
||
import dask | ||
|
||
from .core import JobQueueCluster, docstrings | ||
|
||
logger = logging.getLogger(__name__) | ||
|
||
|
||
class LSFCluster(JobQueueCluster): | ||
__doc__ = docstrings.with_indents(""" Launch Dask on a LSF cluster | ||
|
||
Parameters | ||
---------- | ||
queue : str | ||
Destination queue for each worker job. Passed to `#BSUB -q` option. | ||
project : str | ||
Accounting string associated with each worker job. Passed to | ||
`#BSUB -P` option. | ||
ncpus : int | ||
Number of cpus. Passed to `#BSUB -n` option. | ||
mem : int | ||
Request memory in bytes. Passed to `#BSUB -M` option. | ||
walltime : str | ||
Walltime for each worker job in HH:MM. Passed to `#BSUB -W` option. | ||
job_extra : list | ||
List of other LSF options, for example -u. Each option will be | ||
prepended with the #LSF prefix. | ||
%(JobQueueCluster.parameters)s | ||
|
||
Examples | ||
-------- | ||
>>> from dask_jobqueue import LSFCluster | ||
>>> cluster = LSFcluster(queue='general', project='DaskonLSF', | ||
... cores=15, memory='25GB') | ||
>>> cluster.start_workers(10) # this may take a few seconds to launch | ||
|
||
>>> from dask.distributed import Client | ||
>>> client = Client(cluster) | ||
|
||
This also works with adaptive clusters. This automatically launches and | ||
kill workers based on load. | ||
|
||
>>> cluster.adapt() | ||
""", 4) | ||
|
||
# Override class variables | ||
submit_command = 'bsub <' | ||
cancel_command = 'bkill' | ||
scheduler_name = 'lsf' | ||
|
||
def __init__(self, queue=None, project=None, ncpus=None, mem=None, | ||
walltime=None, job_extra=None, **kwargs): | ||
if queue is None: | ||
queue = dask.config.get('jobqueue.%s.queue' % self.scheduler_name) | ||
if project is None: | ||
project = dask.config.get('jobqueue.%s.project' % self.scheduler_name) | ||
if ncpus is None: | ||
ncpus = dask.config.get('jobqueue.%s.ncpus' % self.scheduler_name) | ||
if mem is None: | ||
mem = dask.config.get('jobqueue.%s.mem' % self.scheduler_name) | ||
if walltime is None: | ||
walltime = dask.config.get('jobqueue.%s.walltime' % self.scheduler_name) | ||
if job_extra is None: | ||
job_extra = dask.config.get('jobqueue.%s.job-extra' % self.scheduler_name) | ||
|
||
# Instantiate args and parameters from parent abstract class | ||
super(LSFCluster, self).__init__(**kwargs) | ||
|
||
header_lines = [] | ||
# LSF header build | ||
if self.name is not None: | ||
header_lines.append('#BSUB -J %s' % self.name) | ||
header_lines.append('#BSUB -e %s.err' % self.name) | ||
header_lines.append('#BSUB -o %s.out' % self.name) | ||
if queue is not None: | ||
header_lines.append('#BSUB -q %s' % queue) | ||
if project is not None: | ||
header_lines.append('#BSUB -P %s' % project) | ||
if ncpus is None: | ||
# Compute default cores specifications | ||
ncpus = self.worker_cores | ||
logger.info("ncpus specification for LSF not set, " | ||
"initializing it to %s" % ncpus) | ||
if ncpus is not None: | ||
header_lines.append('#BSUB -n %s' % ncpus) | ||
if mem is None: | ||
# Compute default memory specifications | ||
mem = self.worker_memory | ||
logger.info("mem specification for LSF not set, " | ||
"initializing it to %s" % mem) | ||
if mem is not None: | ||
memory_string = lsf_format_bytes_ceil(mem) | ||
header_lines.append('#BSUB -M %s' % memory_string) | ||
if walltime is not None: | ||
header_lines.append('#BSUB -W %s' % walltime) | ||
header_lines.extend(['#BSUB %s' % arg for arg in job_extra]) | ||
header_lines.append('JOB_ID=${LSB_JOBID%.*}') | ||
|
||
# Declare class attribute that shall be overriden | ||
self.job_header = '\n'.join(header_lines) | ||
|
||
logger.debug("Job script: \n %s" % self.job_script()) | ||
|
||
def _job_id_from_submit_output(self, out): | ||
return out.split('<')[1].split('>')[0].strip() | ||
|
||
def _submit_job(self, script_filename): | ||
piped_cmd = [self.submit_command + ' ' + script_filename + ' 2> /dev/null'] | ||
return self._call(piped_cmd, shell=True) | ||
|
||
|
||
def lsf_format_bytes_ceil(n): | ||
""" Format bytes as text | ||
|
||
Convert bytes to megabytes which LSF requires. | ||
|
||
Parameters | ||
---------- | ||
n: int | ||
Bytes | ||
|
||
Examples | ||
-------- | ||
>>> lsf_format_bytes_ceil(1234567890) | ||
'1235' | ||
""" | ||
return '%d' % math.ceil(n / (1000**2)) |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just checking in, does LSF need/use all of these options? I'm slightly concerned that as we copy configs from different systems we may accrue more than is necessary.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The only ones I haven't used are
extra
andenv-extra
.extra
is for Additional arguments to pass todask-worker
so that is probably worth keeping.env-extra
is other commands to the script before launching the worker. I can't see myself using this butcore.py
checks for itdask-jobqueue/dask_jobqueue/core.py
Line 109 in f7c565a
A future PR could be to move that out of
core.py
and have users specify it in their individual classes. I'll let you decide that.