-
-
Notifications
You must be signed in to change notification settings - Fork 138
/
pbs.py
140 lines (118 loc) · 4.52 KB
/
pbs.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
import logging
import math
import os
import dask
from .core import Job, JobQueueCluster, job_parameters, cluster_parameters
logger = logging.getLogger(__name__)
def pbs_format_bytes_ceil(n):
""" Format bytes as text.
PBS expects KiB, MiB or Gib, but names it KB, MB, GB whereas Dask makes the difference between KB and KiB.
>>> pbs_format_bytes_ceil(1)
'1B'
>>> pbs_format_bytes_ceil(1234)
'1234B'
>>> pbs_format_bytes_ceil(12345678)
'13MB'
>>> pbs_format_bytes_ceil(1234567890)
'1177MB'
>>> pbs_format_bytes_ceil(15000000000)
'14GB'
"""
if n >= 10 * (1024 ** 3):
return "%dGB" % math.ceil(n / (1024 ** 3))
if n >= 10 * (1024 ** 2):
return "%dMB" % math.ceil(n / (1024 ** 2))
if n >= 10 * 1024:
return "%dkB" % math.ceil(n / 1024)
return "%dB" % n
class PBSJob(Job):
submit_command = "qsub"
cancel_command = "qdel"
config_name = "pbs"
def __init__(
self,
*args,
queue=None,
project=None,
resource_spec=None,
walltime=None,
job_extra=None,
config_name="pbs",
**kwargs
):
if queue is None:
queue = dask.config.get("jobqueue.%s.queue" % config_name)
if resource_spec is None:
resource_spec = dask.config.get("jobqueue.%s.resource-spec" % config_name)
if walltime is None:
walltime = dask.config.get("jobqueue.%s.walltime" % config_name)
if job_extra is None:
job_extra = dask.config.get("jobqueue.%s.job-extra" % config_name)
if project is None:
project = dask.config.get(
"jobqueue.%s.project" % config_name
) or os.environ.get("PBS_ACCOUNT")
# Instantiate args and parameters from parent abstract class
super().__init__(*args, config_name=config_name, **kwargs)
# Try to find a project name from environment variable
project = project or os.environ.get("PBS_ACCOUNT")
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 resource_spec is None:
# Compute default resources specifications
resource_spec = "select=1:ncpus=%d" % self.worker_cores
memory_string = pbs_format_bytes_ceil(self.worker_memory)
resource_spec += ":mem=" + memory_string
logger.info(
"Resource specification for PBS not set, initializing it to %s"
% resource_spec
)
if resource_spec is not None:
header_lines.append("#PBS -l %s" % resource_spec)
if walltime is not None:
header_lines.append("#PBS -l walltime=%s" % walltime)
if self.log_directory is not None:
header_lines.append("#PBS -e %s/" % self.log_directory)
header_lines.append("#PBS -o %s/" % self.log_directory)
header_lines.extend(["#PBS %s" % arg for arg in job_extra])
header_lines.append("JOB_ID=${PBS_JOBID%%.*}")
# Declare class attribute that shall be overridden
self.job_header = "\n".join(header_lines)
logger.debug("Job script: \n %s" % self.job_script())
class PBSCluster(JobQueueCluster):
__doc__ = """ Launch Dask on a PBS cluster
Parameters
----------
queue : str
Destination queue for each worker job. Passed to `#PBS -q` option.
project : str
Accounting string associated with each worker job. Passed to `#PBS -A` option.
{job}
{cluster}
resource_spec : str
Request resources and specify job placement. Passed to `#PBS -l` option.
walltime : str
Walltime for each worker job.
job_extra : list
List of other PBS options. Each option will be prepended with the #PBS prefix.
Examples
--------
>>> from dask_jobqueue import PBSCluster
>>> cluster = PBSCluster(queue='regular', project="myproj", cores=24,
... memory="500 GB")
>>> cluster.scale(jobs=10) # ask for 10 jobs
>>> 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(maximum_jobs=20)
""".format(
job=job_parameters, cluster=cluster_parameters
)
job_cls = PBSJob
config_name = "pbs"