Skip to content

Commit

Permalink
Change project to account where appropriate. Update docs (#586)
Browse files Browse the repository at this point in the history
  • Loading branch information
guillaumeeb committed Sep 6, 2022
1 parent 1167281 commit 045d168
Show file tree
Hide file tree
Showing 14 changed files with 203 additions and 42 deletions.
2 changes: 1 addition & 1 deletion dask_jobqueue/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -285,7 +285,7 @@ def __init__(
if header_skip is not None:
warn = (
"header_skip has been renamed to job_directives_skip. "
"You are still using it (even if only set to (); please also check config files). "
"You are still using it (even if only set to []; please also check config files). "
"If you did not set job_directives_skip yet, header_skip will be respected for now, "
"but it will be removed in a future release. "
"If you already set job_directives_skip, header_skip is ignored and you can remove it."
Expand Down
6 changes: 3 additions & 3 deletions dask_jobqueue/jobqueue.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ jobqueue:
# PBS resource manager options
shebang: "#!/usr/bin/env bash"
queue: null
project: null
account: null
walltime: '00:30:00'
env-extra: null
job-script-prologue: []
Expand Down Expand Up @@ -110,7 +110,7 @@ jobqueue:
# SLURM resource manager options
shebang: "#!/usr/bin/env bash"
queue: null
project: null
account: null
walltime: '00:30:00'
env-extra: null
job-script-prologue: []
Expand Down Expand Up @@ -142,7 +142,7 @@ jobqueue:
# PBS resource manager options
shebang: "#!/usr/bin/env bash"
queue: null
project: null
account: null
walltime: '00:30:00'
env-extra: null
job-script-prologue: []
Expand Down
2 changes: 1 addition & 1 deletion dask_jobqueue/lsf.py
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ class LSFCluster(JobQueueCluster):
queue : str
Destination queue for each worker job. Passed to `#BSUB -q` option.
project : str
Accounting string associated with each worker job. Passed to
Project associated with each worker job. Passed to
`#BSUB -P` option.
{job}
ncpus : int
Expand Down
2 changes: 1 addition & 1 deletion dask_jobqueue/oar.py
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ class OARCluster(JobQueueCluster):
queue : str
Destination queue for each worker job. Passed to `#OAR -q` option.
project : str
Accounting string associated with each worker job. Passed to `#OAR -p` option.
Project associated with each worker job. Passed to `#OAR --project` option.
{job}
{cluster}
resource_spec : str
Expand Down
32 changes: 23 additions & 9 deletions dask_jobqueue/pbs.py
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
import logging
import math
import os
import warnings

import dask

Expand Down Expand Up @@ -45,6 +46,7 @@ def __init__(
name=None,
queue=None,
project=None,
account=None,
resource_spec=None,
walltime=None,
config_name=None,
Expand All @@ -63,22 +65,32 @@ def __init__(
if walltime is None:
walltime = dask.config.get("jobqueue.%s.walltime" % self.config_name)

if project is None:
project = dask.config.get(
"jobqueue.%s.project" % self.config_name
if account is None:
account = dask.config.get(
"jobqueue.%s.account" % self.config_name
) or os.environ.get("PBS_ACCOUNT")

# Try to find a project name from environment variable
project = project or os.environ.get("PBS_ACCOUNT")
if project is None:
project = dask.config.get("jobqueue.%s.project" % self.config_name, None)
if project is not None:
warn = (
"project has been renamed to account as this kwarg was used wit -A option. "
"You are still using it (please also check config files). "
"If you did not set account yet, project will be respected for now, "
"but it will be removed in a future release. "
"If you already set account, project is ignored and you can remove it."
)
warnings.warn(warn, FutureWarning)
if not account:
account = project

header_lines = []
# PBS header build
if self.job_name is not None:
header_lines.append("#PBS -N %s" % self.job_name)
if queue is not None:
header_lines.append("#PBS -q %s" % queue)
if project is not None:
header_lines.append("#PBS -A %s" % project)
if account is not None:
header_lines.append("#PBS -A %s" % account)
if resource_spec is None:
# Compute default resources specifications
resource_spec = "select=1:ncpus=%d" % self.worker_cores
Expand Down Expand Up @@ -121,6 +133,8 @@ class PBSCluster(JobQueueCluster):
queue : str
Destination queue for each worker job. Passed to `#PBS -q` option.
project : str
Deprecated: use ``account`` instead. This parameter will be removed in a future version.
account : str
Accounting string associated with each worker job. Passed to `#PBS -A` option.
{job}
{cluster}
Expand All @@ -136,7 +150,7 @@ class PBSCluster(JobQueueCluster):
Examples
--------
>>> from dask_jobqueue import PBSCluster
>>> cluster = PBSCluster(queue='regular', project="myproj", cores=24,
>>> cluster = PBSCluster(queue='regular', account="myaccountingstring", cores=24,
... memory="500 GB")
>>> cluster.scale(jobs=10) # ask for 10 jobs
Expand Down
2 changes: 1 addition & 1 deletion dask_jobqueue/sge.py
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ class SGECluster(JobQueueCluster):
queue : str
Destination queue for each worker job. Passed to `#$ -q` option.
project : str
Accounting string associated with each worker job. Passed to `#$ -A` option.
Project associated with each worker job. Passed to `#$ -P` option.
{job}
{cluster}
resource_spec : str
Expand Down
28 changes: 23 additions & 5 deletions dask_jobqueue/slurm.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
import logging
import math
import warnings

import dask

Expand All @@ -20,6 +21,7 @@ def __init__(
name=None,
queue=None,
project=None,
account=None,
walltime=None,
job_cpu=None,
job_mem=None,
Expand All @@ -32,8 +34,22 @@ def __init__(

if queue is None:
queue = dask.config.get("jobqueue.%s.queue" % self.config_name)
if account is None:
account = dask.config.get("jobqueue.%s.account" % self.config_name)
if project is None:
project = dask.config.get("jobqueue.%s.project" % self.config_name)
project = dask.config.get("jobqueue.%s.project" % self.config_name, None)
if project is not None:
warn = (
"project has been renamed to account as this kwarg was used wit -A option. "
"You are still using it (please also check config files). "
"If you did not set account yet, project will be respected for now, "
"but it will be removed in a future release. "
"If you already set account, project is ignored and you can remove it."
)
warnings.warn(warn, FutureWarning)
if not account:
account = project

if walltime is None:
walltime = dask.config.get("jobqueue.%s.walltime" % self.config_name)
if job_cpu is None:
Expand All @@ -56,8 +72,8 @@ def __init__(
)
if queue is not None:
header_lines.append("#SBATCH -p %s" % queue)
if project is not None:
header_lines.append("#SBATCH -A %s" % project)
if account is not None:
header_lines.append("#SBATCH -A %s" % account)

# Init resources, always 1 task,
# and then number of cpu is processes * threads if not set
Expand Down Expand Up @@ -123,7 +139,9 @@ class SLURMCluster(JobQueueCluster):
queue : str
Destination queue for each worker job. Passed to `#SBATCH -p` option.
project : str
Accounting string associated with each worker job. Passed to `#SBATCH -A` option.
Deprecated: use ``account`` instead. This parameter will be removed in a future version.
account : str
Accounting string associated with each worker job. Passed to `#PBS -A` option.
{job}
{cluster}
walltime : str
Expand All @@ -143,7 +161,7 @@ class SLURMCluster(JobQueueCluster):
>>> from dask_jobqueue import SLURMCluster
>>> cluster = SLURMCluster(
... queue='regular',
... project="myproj",
... account="myaccount",
... cores=24,
... memory="500 GB"
... )
Expand Down
70 changes: 67 additions & 3 deletions dask_jobqueue/tests/test_pbs.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ def test_header(Cluster):

with Cluster(
queue="regular",
project="DaskOnPBS",
account="DaskOnPBS",
processes=4,
cores=8,
resource_spec="select=1:ncpus=24:mem=100GB",
Expand Down Expand Up @@ -82,7 +82,7 @@ def test_job_script(Cluster):

with Cluster(
queue="regular",
project="DaskOnPBS",
account="DaskOnPBS",
processes=4,
cores=8,
resource_spec="select=1:ncpus=24:mem=100GB",
Expand Down Expand Up @@ -347,7 +347,7 @@ def test_config(loop):
def test_config_name_pbs_takes_custom_config():
conf = {
"queue": "myqueue",
"project": "myproject",
"account": "myaccount",
"ncpus": 1,
"cores": 1,
"memory": "2 GB",
Expand Down Expand Up @@ -390,3 +390,67 @@ def test_informative_errors():
async def test_adapt(loop):
async with PBSCluster(cores=1, memory="1 GB", asynchronous=True) as cluster:
cluster.adapt()


def test_deprecation_project():
import warnings

# test issuing of warning
warnings.simplefilter("always")

job_cls = PBSCluster.job_cls
with warnings.catch_warnings(record=True) as w:
# should give a warning
job = job_cls(cores=1, memory="1 GB", project=["project is used"])
assert len(w) == 1
assert issubclass(w[0].category, FutureWarning)
assert "project has been renamed" in str(w[0].message)
with warnings.catch_warnings(record=True) as w:
# should give a warning
job = job_cls(
cores=1,
memory="1 GB",
project=["project is used"],
account=["account is used"],
)
assert len(w) == 1
assert issubclass(w[0].category, FutureWarning)
assert "project has been renamed" in str(w[0].message)
with warnings.catch_warnings(record=True) as w:
# should not give a warning
job = job_cls(
cores=1,
memory="1 GB",
account=["account is used, project not"],
)
assert len(w) == 0

# the rest is not about the warning but about behaviour: if account is not
# set, project should still be used if provided
warnings.simplefilter("ignore")
job = job_cls(
cores=1,
memory="1 GB",
project=["project"],
account=["account"],
)
job_script = job.job_script()
assert "project" not in job_script
assert "account" in job_script

job = job_cls(
cores=1,
memory="1 GB",
project=["project"],
)
job_script = job.job_script()
assert "project" in job_script

job = job_cls(
cores=1,
memory="1 GB",
project=["project"],
account=[],
)
job_script = job.job_script()
assert "project" in job_script
68 changes: 66 additions & 2 deletions dask_jobqueue/tests/test_slurm.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ def test_header():

with SLURMCluster(
queue="regular",
project="DaskOnSlurm",
account="DaskOnSlurm",
processes=4,
cores=8,
memory="28GB",
Expand Down Expand Up @@ -179,7 +179,7 @@ def test_adaptive(loop):
def test_config_name_slurm_takes_custom_config():
conf = {
"queue": "myqueue",
"project": "myproject",
"account": "myaccount",
"ncpus": 1,
"cores": 1,
"memory": "2 GB",
Expand Down Expand Up @@ -249,3 +249,67 @@ def test_worker_name_uses_cluster_name(loop):
"test-my-env-variable-value-0",
"test-my-env-variable-value-1",
]


def test_deprecation_project():
import warnings

# test issuing of warning
warnings.simplefilter("always")

job_cls = SLURMCluster.job_cls
with warnings.catch_warnings(record=True) as w:
# should give a warning
job = job_cls(cores=1, memory="1 GB", project=["project is used"])
assert len(w) == 1
assert issubclass(w[0].category, FutureWarning)
assert "project has been renamed" in str(w[0].message)
with warnings.catch_warnings(record=True) as w:
# should give a warning
job = job_cls(
cores=1,
memory="1 GB",
project=["project is used"],
account=["account is used"],
)
assert len(w) == 1
assert issubclass(w[0].category, FutureWarning)
assert "project has been renamed" in str(w[0].message)
with warnings.catch_warnings(record=True) as w:
# should not give a warning
job = job_cls(
cores=1,
memory="1 GB",
account=["account is used, project not"],
)
assert len(w) == 0

# the rest is not about the warning but about behaviour: if account is not
# set, project should still be used if provided
warnings.simplefilter("ignore")
job = job_cls(
cores=1,
memory="1 GB",
project=["project"],
account=["account"],
)
job_script = job.job_script()
assert "project" not in job_script
assert "account" in job_script

job = job_cls(
cores=1,
memory="1 GB",
project=["project"],
)
job_script = job.job_script()
assert "project" in job_script

job = job_cls(
cores=1,
memory="1 GB",
project=["project"],
account=[],
)
job_script = job.job_script()
assert "project" in job_script

0 comments on commit 045d168

Please sign in to comment.