toil.batchSystems.abstractBatchSystem

Module Contents

Classes

BatchJobExitReason

Enum where members are also (and must be) ints

UpdatedBatchJobInfo

WorkerCleanupInfo

AbstractBatchSystem

An abstract base class to represent the interface the batch system must provide to Toil.

BatchSystemSupport

Partial implementation of AbstractBatchSystem, support methods.

NodeInfo

The coresUsed attribute is a floating point value between 0 (all cores idle) and 1 (all cores

AbstractScalableBatchSystem

A batch system that supports a variable number of worker nodes.

ResourcePool

Represents an integral amount of a resource (such as memory bytes).

ResourceSet

Represents a collection of distinct resources (such as accelerators).

Attributes

logger

EXIT_STATUS_UNAVAILABLE_VALUE

toil.batchSystems.abstractBatchSystem.logger
toil.batchSystems.abstractBatchSystem.EXIT_STATUS_UNAVAILABLE_VALUE = 255
class toil.batchSystems.abstractBatchSystem.BatchJobExitReason[source]

Bases: enum.IntEnum

Enum where members are also (and must be) ints

FINISHED: int = 1

Successfully finished.

FAILED: int = 2

Job finished, but failed.

LOST: int = 3

Preemptable failure (job’s executing host went away).

KILLED: int = 4

Job killed before finishing.

ERROR: int = 5

Internal error.

MEMLIMIT: int = 6

Job hit batch system imposed memory limit.

classmethod to_string(value)[source]

Convert to human-readable string.

Given an int that may be or may be equal to a value from the enum, produce the string value of its matching enum entry, or a stringified int.

Parameters:

value (int)

Return type:

str

class toil.batchSystems.abstractBatchSystem.UpdatedBatchJobInfo[source]

Bases: NamedTuple

jobID: int
exitStatus: int

The exit status (integer value) of the job. 0 implies successful.

EXIT_STATUS_UNAVAILABLE_VALUE is used when the exit status is not available (e.g. job is lost, or otherwise died but actual exit code was not reported).

exitReason: BatchJobExitReason | None
wallTime: float | int | None
class toil.batchSystems.abstractBatchSystem.WorkerCleanupInfo[source]

Bases: NamedTuple

work_dir: str | None

Work directory path (where the cache would go) if specified by user

coordination_dir: str | None

Coordination directory path (where lock files would go) if specified by user

workflow_id: str

Used to identify files specific to this workflow

clean_work_dir: str

When to clean up the work and coordination directories for a job (‘always’, ‘onSuccess’, ‘onError’, ‘never’)

class toil.batchSystems.abstractBatchSystem.AbstractBatchSystem[source]

Bases: abc.ABC

An abstract base class to represent the interface the batch system must provide to Toil.

abstract classmethod supportsAutoDeployment()[source]

Whether this batch system supports auto-deployment of the user script itself.

If it does, the setUserScript() can be invoked to set the resource object representing the user script.

Note to implementors: If your implementation returns True here, it should also override

Return type:

bool

abstract classmethod supportsWorkerCleanup()[source]

Whether this batch system supports worker cleanup.

Indicates whether this batch system invokes BatchSystemSupport.workerCleanup() after the last job for a particular workflow invocation finishes. Note that the term worker refers to an entire node, not just a worker process. A worker process may run more than one job sequentially, and more than one concurrent worker process may exist on a worker node, for the same workflow. The batch system is said to shut down after the last worker process terminates.

Return type:

bool

abstract setUserScript(userScript)[source]

Set the user script for this workflow.

This method must be called before the first job is issued to this batch system, and only if supportsAutoDeployment() returns True, otherwise it will raise an exception.

Parameters:

userScript (toil.resource.Resource) – the resource object representing the user script or module and the modules it depends on.

Return type:

None

set_message_bus(message_bus)[source]

Give the batch system an opportunity to connect directly to the message bus, so that it can send informational messages about the jobs it is running to other Toil components.

Parameters:

message_bus (toil.bus.MessageBus)

Return type:

None

abstract issueBatchJob(jobDesc, job_environment=None)[source]

Issues a job with the specified command to the batch system and returns a unique jobID.

Parameters:
  • jobDesc (toil.job.JobDescription) – a toil.job.JobDescription

  • job_environment (Optional[Dict[str, str]]) – a collection of job-specific environment variables to be set on the worker.

Returns:

a unique jobID that can be used to reference the newly issued job

Return type:

int

abstract killBatchJobs(jobIDs)[source]

Kills the given job IDs. After returning, the killed jobs will not appear in the results of getRunningBatchJobIDs. The killed job will not be returned from getUpdatedBatchJob.

Parameters:

jobIDs (List[int]) – list of IDs of jobs to kill

Return type:

None

abstract getIssuedBatchJobIDs()[source]

Gets all currently issued jobs

Returns:

A list of jobs (as jobIDs) currently issued (may be running, or may be waiting to be run). Despite the result being a list, the ordering should not be depended upon.

Return type:

List[int]

abstract getRunningBatchJobIDs()[source]

Gets a map of jobs as jobIDs that are currently running (not just waiting) and how long they have been running, in seconds.

Returns:

dictionary with currently running jobID keys and how many seconds they have been running as the value

Return type:

Dict[int, float]

abstract getUpdatedBatchJob(maxWait)[source]

Returns information about job that has updated its status (i.e. ceased running, either successfully or with an error). Each such job will be returned exactly once.

Does not return info for jobs killed by killBatchJobs, although they may cause None to be returned earlier than maxWait.

Parameters:

maxWait (int) – the number of seconds to block, waiting for a result

Returns:

If a result is available, returns UpdatedBatchJobInfo. Otherwise it returns None. wallTime is the number of seconds (a strictly positive float) in wall-clock time the job ran for, or None if this batch system does not support tracking wall time.

Return type:

Optional[UpdatedBatchJobInfo]

getSchedulingStatusMessage()[source]

Get a log message fragment for the user about anything that might be going wrong in the batch system, if available.

If no useful message is available, return None.

This can be used to report what resource is the limiting factor when scheduling jobs, for example. If the leader thinks the workflow is stuck, the message can be displayed to the user to help them diagnose why it might be stuck.

Returns:

User-directed message about scheduling state.

Return type:

Optional[str]

abstract shutdown()[source]

Called at the completion of a toil invocation. Should cleanly terminate all worker threads.

Return type:

None

abstract setEnv(name, value=None)[source]

Set an environment variable for the worker process before it is launched.

The worker process will typically inherit the environment of the machine it is running on but this method makes it possible to override specific variables in that inherited environment before the worker is launched. Note that this mechanism is different to the one used by the worker internally to set up the environment of a job. A call to this method affects all jobs issued after this method returns. Note to implementors: This means that you would typically need to copy the variables before enqueuing a job.

If no value is provided it will be looked up from the current environment.

Parameters:
  • name (str)

  • value (Optional[str])

Return type:

None

classmethod add_options(parser)[source]

If this batch system provides any command line options, add them to the given parser.

Parameters:

parser (Union[argparse.ArgumentParser, argparse._ArgumentGroup])

Return type:

None

classmethod setOptions(setOption)[source]

Process command line or configuration options relevant to this batch system.

Parameters:

setOption (toil.batchSystems.options.OptionSetter) – A function with signature setOption(option_name, parsing_function=None, check_function=None, default=None, env=None) returning nothing, used to update run configuration as a side effect.

Return type:

None

getWorkerContexts()[source]

Get a list of picklable context manager objects to wrap worker work in, in order.

Can be used to ask the Toil worker to do things in-process (such as configuring environment variables, hot-deploying user scripts, or cleaning up a node) that would otherwise require a wrapping “executor” process.

Return type:

List[ContextManager[Any]]

class toil.batchSystems.abstractBatchSystem.BatchSystemSupport(config, maxCores, maxMemory, maxDisk)[source]

Bases: AbstractBatchSystem

Partial implementation of AbstractBatchSystem, support methods.

Parameters:
check_resource_request(requirer)[source]

Check resource request is not greater than that available or allowed.

Parameters:
  • requirer (toil.job.Requirer) – Object whose requirements are being checked

  • job_name (str) – Name of the job being checked, for generating a useful error report.

  • detail (str) – Batch-system-specific message to include in the error.

Raises:

InsufficientSystemResources – raised when a resource is requested in an amount greater than allowed

Return type:

None

setEnv(name, value=None)[source]

Set an environment variable for the worker process before it is launched. The worker process will typically inherit the environment of the machine it is running on but this method makes it possible to override specific variables in that inherited environment before the worker is launched. Note that this mechanism is different to the one used by the worker internally to set up the environment of a job. A call to this method affects all jobs issued after this method returns. Note to implementors: This means that you would typically need to copy the variables before enqueuing a job.

If no value is provided it will be looked up from the current environment.

Parameters:
  • name (str) – the environment variable to be set on the worker.

  • value (Optional[str]) – if given, the environment variable given by name will be set to this value. If None, the variable’s current value will be used as the value on the worker

Raises:

RuntimeError – if value is None and the name cannot be found in the environment

Return type:

None

set_message_bus(message_bus)[source]

Give the batch system an opportunity to connect directly to the message bus, so that it can send informational messages about the jobs it is running to other Toil components.

Parameters:

message_bus (toil.bus.MessageBus)

Return type:

None

get_batch_logs_dir()[source]

Get the directory where the backing batch system should save its logs.

Only really makes sense if the backing batch system actually saves logs to a filesystem; Kubernetes for example does not. Ought to be a directory shared between the leader and the workers, if the backing batch system writes logs onto the worker’s view of the filesystem, like many HPC schedulers do.

Return type:

str

format_std_out_err_path(toil_job_id, cluster_job_id, std)[source]

Format path for batch system standard output/error and other files generated by the batch system itself.

Files will be written to the batch logs directory (–batchLogsDir, defaulting to the Toil work directory) with names containing both the Toil and batch system job IDs, for ease of debugging job failures.

Param:

int toil_job_id : The unique id that Toil gives a job.

Param:

cluster_job_id : What the cluster, for example, GridEngine, uses as its internal job id.

Param:

string std : The provenance of the stream (for example: ‘err’ for ‘stderr’ or ‘out’ for ‘stdout’)

Return type:

string : Formatted filename; however if self.config.noStdOutErr is true, returns ‘/dev/null’ or equivalent.

Parameters:
  • toil_job_id (int)

  • cluster_job_id (str)

  • std (str)

format_std_out_err_glob(toil_job_id)[source]

Get a glob string that will match all file paths generated by format_std_out_err_path for a job.

Parameters:

toil_job_id (int)

Return type:

str

static workerCleanup(info)[source]

Cleans up the worker node on batch system shutdown.

Also see supportsWorkerCleanup().

Parameters:

info (WorkerCleanupInfo) – A named tuple consisting of all the relevant information for cleaning up the worker.

Return type:

None

class toil.batchSystems.abstractBatchSystem.NodeInfo(coresUsed, memoryUsed, coresTotal, memoryTotal, requestedCores, requestedMemory, workers)[source]

The coresUsed attribute is a floating point value between 0 (all cores idle) and 1 (all cores busy), reflecting the CPU load of the node.

The memoryUsed attribute is a floating point value between 0 (no memory used) and 1 (all memory used), reflecting the memory pressure on the node.

The coresTotal and memoryTotal attributes are the node’s resources, not just the used resources

The requestedCores and requestedMemory attributes are all the resources that Toil Jobs have reserved on the node, regardless of whether the resources are actually being used by the Jobs.

The workers attribute is an integer reflecting the number of workers currently active workers on the node.

Parameters:
class toil.batchSystems.abstractBatchSystem.AbstractScalableBatchSystem[source]

Bases: AbstractBatchSystem

A batch system that supports a variable number of worker nodes.

Used by toil.provisioners.clusterScaler.ClusterScaler to scale the number of worker nodes in the cluster up or down depending on overall load.

abstract getNodes(preemptible=None, timeout=600)[source]

Returns a dictionary mapping node identifiers of preemptible or non-preemptible nodes to NodeInfo objects, one for each node.

Parameters:
  • preemptible (Optional[bool]) – If True (False) only (non-)preemptible nodes will be returned. If None, all nodes will be returned.

  • timeout (int)

Return type:

Dict[str, NodeInfo]

abstract nodeInUse(nodeIP)[source]

Can be used to determine if a worker node is running any tasks. If the node is doesn’t exist, this function should simply return False.

Parameters:

nodeIP (str) – The worker nodes private IP address

Returns:

True if the worker node has been issued any tasks, else False

Return type:

bool

abstract ignoreNode(nodeAddress)[source]

Stop sending jobs to this node. Used in autoscaling when the autoscaler is ready to terminate a node, but jobs are still running. This allows the node to be terminated after the current jobs have finished.

Parameters:

nodeAddress (str) – IP address of node to ignore.

Return type:

None

abstract unignoreNode(nodeAddress)[source]

Stop ignoring this address, presumably after a node with this address has been terminated. This allows for the possibility of a new node having the same address as a terminated one.

Parameters:

nodeAddress (str)

Return type:

None

exception toil.batchSystems.abstractBatchSystem.InsufficientSystemResources(requirer, resource, available=None, batch_system=None, source=None, details=[])[source]

Bases: Exception

Common base class for all non-exit exceptions.

Parameters:
  • requirer (toil.job.Requirer)

  • resource (str)

  • available (Optional[toil.job.ParsedRequirement])

  • batch_system (Optional[str])

  • source (Optional[str])

  • details (List[str])

__str__()[source]

Explain the exception.

Return type:

str

exception toil.batchSystems.abstractBatchSystem.AcquisitionTimeoutException(resource, requested, available)[source]

Bases: Exception

To be raised when a resource request times out.

Parameters:
class toil.batchSystems.abstractBatchSystem.ResourcePool(initial_value, resource_type, timeout=5)[source]

Represents an integral amount of a resource (such as memory bytes). Amounts can be acquired immediately or with a timeout, and released. Provides a context manager to do something with an amount of resource acquired.

Parameters:
  • initial_value (int)

  • resource_type (str)

  • timeout (float)

acquireNow(amount)[source]

Reserve the given amount of the given resource. Returns True if successful and False if this is not possible immediately.

Parameters:

amount (int)

Return type:

bool

acquire(amount)[source]

Reserve the given amount of the given resource. Raises AcquisitionTimeoutException if this is not possible in under self.timeout time.

Parameters:

amount (int)

Return type:

None

release(amount)[source]
Parameters:

amount (int)

Return type:

None

__str__()[source]

Return str(self).

Return type:

str

__repr__()[source]

Return repr(self).

Return type:

str

acquisitionOf(amount)[source]
Parameters:

amount (int)

Return type:

Iterator[None]

class toil.batchSystems.abstractBatchSystem.ResourceSet(initial_value, resource_type, timeout=5)[source]

Represents a collection of distinct resources (such as accelerators). Subsets can be acquired immediately or with a timeout, and released. Provides a context manager to do something with a set of of resources acquired.

Parameters:
  • initial_value (Set[int])

  • resource_type (str)

  • timeout (float)

acquireNow(subset)[source]

Reserve the given amount of the given resource. Returns True if successful and False if this is not possible immediately.

Parameters:

subset (Set[int])

Return type:

bool

acquire(subset)[source]

Reserve the given amount of the given resource. Raises AcquisitionTimeoutException if this is not possible in under self.timeout time.

Parameters:

subset (Set[int])

Return type:

None

release(subset)[source]
Parameters:

subset (Set[int])

Return type:

None

get_free_snapshot()[source]

Get a snapshot of what items are free right now. May be stale as soon as you get it, but you will need some kind of hint to try and do an acquire.

Return type:

Set[int]

__str__()[source]

Return str(self).

Return type:

str

__repr__()[source]

Return repr(self).

Return type:

str

acquisitionOf(subset)[source]
Parameters:

subset (Set[int])

Return type:

Iterator[None]