2020-04-06 12:16:35 -07:00
.. _trainable-docs:
2020-05-16 12:55:08 -07:00
Training (tune.Trainable, tune.report)
======================================
2020-03-23 12:23:21 -07:00
2020-06-15 10:42:54 -07:00
Training can be done with either a **Class API** (`` tune.Trainable `` ) or **function API** (`` tune.report `` ).
2020-04-06 12:16:35 -07:00
2020-04-27 18:01:00 -07:00
For the sake of example, let's maximize this objective function:
.. code-block :: python
def objective(x, a, b):
return a * (x * * 0.5) + b
.. _tune-function-api:
2020-06-15 10:42:54 -07:00
Function API
------------
2020-10-10 00:54:31 -07:00
With the Function API, you can report intermediate metrics by simply calling `` tune.report `` within the provided function.
2020-04-06 12:16:35 -07:00
.. code-block :: python
def trainable(config):
2020-04-27 18:01:00 -07:00
# config (dict): A dict of hyperparameters.
for x in range(20):
2020-06-15 10:42:54 -07:00
intermediate_score = objective(x, config["a"], config["b"])
2020-04-06 12:16:35 -07:00
2020-09-18 04:18:35 +01:00
tune.report(score=intermediate_score) # This sends the score to Tune.
2020-04-27 18:01:00 -07:00
analysis = tune.run(
trainable,
2020-06-15 10:42:54 -07:00
config={"a": 2, "b": 4}
)
2020-04-27 18:01:00 -07:00
print("best config: ", analysis.get_best_config(metric="score", mode="max"))
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
.. tip :: Do not use `` tune.report `` within a `` Trainable `` class.
Tune will run this function on a separate thread in a Ray actor process.
2020-10-10 00:54:31 -07:00
You'll notice that Ray Tune will output extra values in addition to the user reported metrics, such as `` iterations_since_restore `` . See :ref: `tune-autofilled-metrics` for an explanation/glossary of these values.
2020-09-18 04:18:35 +01:00
.. tip :: If you want to leverage multi-node data parallel training with PyTorch while using parallel hyperparameter tuning, check out our :ref: `PyTorch <tune-pytorch-cifar>` user guide and Tune's :ref: `distributed pytorch integrations <tune-integration-torch>` .
Function API return and yield values
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Instead of using `` tune.report() `` , you can also use Python's `` yield ``
statement to report metrics to Ray Tune:
.. code-block :: python
def trainable(config):
# config (dict): A dict of hyperparameters.
for x in range(20):
intermediate_score = objective(x, config["a"], config["b"])
yield {"score": intermediate_score} # This sends the score to Tune.
analysis = tune.run(
trainable,
config={"a": 2, "b": 4}
)
print("best config: ", analysis.get_best_config(metric="score", mode="max"))
If you yield a dictionary object, this will work just as `` tune.report() `` .
If you yield a number, if will be reported to Ray Tune with the key `` _metric `` , i.e.
as if you had called `` tune.report(_metric=value) `` .
Ray Tune supports the same functionality for return values if you only
report metrics at the end of each run:
.. code-block :: python
def trainable(config):
# config (dict): A dict of hyperparameters.
final_score = 0
for x in range(20):
final_score = objective(x, config["a"], config["b"])
return {"score": final_score} # This sends the score to Tune.
analysis = tune.run(
trainable,
config={"a": 2, "b": 4}
)
print("best config: ", analysis.get_best_config(metric="score", mode="max"))
2020-07-30 09:46:37 -07:00
.. _tune-function-checkpointing:
2020-06-15 10:42:54 -07:00
Function API Checkpointing
~~~~~~~~~~~~~~~~~~~~~~~~~~
2020-07-30 09:46:37 -07:00
Many Tune features rely on checkpointing, including the usage of certain Trial Schedulers and fault tolerance. To use Tune's checkpointing features, you must expose a `` checkpoint_dir `` argument in the function signature, and call `` tune.checkpoint_dir `` :
2020-06-15 10:42:54 -07:00
.. code-block :: python
import time
from ray import tune
2020-07-30 09:46:37 -07:00
def train_func(config, checkpoint_dir=None):
2020-06-15 10:42:54 -07:00
start = 0
2020-07-30 09:46:37 -07:00
if checkpoint_dir:
with open(os.path.join(checkpoint_dir, "checkpoint")) as f:
2020-06-15 10:42:54 -07:00
state = json.loads(f.read())
start = state["step"] + 1
for iter in range(start, 100):
time.sleep(1)
2020-10-26 12:19:03 -07:00
with tune.checkpoint_dir(step=step) as checkpoint_dir:
2020-07-30 09:46:37 -07:00
path = os.path.join(checkpoint_dir, "checkpoint")
with open(path, "w") as f:
f.write(json.dumps({"step": start}))
2020-06-15 10:42:54 -07:00
tune.report(hello="world", ray="tune")
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
tune.run(train_func)
2020-04-06 12:16:35 -07:00
2020-07-30 09:46:37 -07:00
.. note :: `` checkpoint_freq `` and `` checkpoint_at_end `` will not work with Function API checkpointing.
2020-06-15 10:42:54 -07:00
In this example, checkpoints will be saved by training iteration to `` local_dir/exp_name/trial_name/checkpoint_<step> `` . You can restore a single trial checkpoint by using `` tune.run(restore=<checkpoint_dir>) `` :
.. code-block :: python
analysis = tune.run(
train,
config={
"max_iter": 5
},
).trials
last_ckpt = trial.checkpoint.value
analysis = tune.run(train, config={"max_iter": 10}, restore=last_ckpt)
Tune also may copy or move checkpoints during the course of tuning. For this purpose, it is important not to depend on absolute paths in the implementation of `` save `` .
2020-04-27 18:01:00 -07:00
.. _tune-class-api:
2020-04-06 12:16:35 -07:00
2020-04-27 18:01:00 -07:00
Trainable Class API
-------------------
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
.. caution :: Do not use `` tune.report `` within a `` Trainable `` class.
2020-04-06 12:16:35 -07:00
The Trainable **class API** will require users to subclass `` ray.tune.Trainable `` . Here's a naive example of this API:
.. code-block :: python
from ray import tune
2020-04-27 18:01:00 -07:00
class Trainable(tune.Trainable):
2020-07-01 11:00:00 -07:00
def setup(self, config):
2020-04-27 18:01:00 -07:00
# config (dict): A dict of hyperparameters
self.x = 0
self.a = config["a"]
self.b = config["b"]
2020-04-06 12:16:35 -07:00
2020-07-01 11:00:00 -07:00
def step(self): # This is called iteratively.
2020-04-27 18:01:00 -07:00
score = objective(self.x, self.a, self.b)
self.x += 1
return {"score": score}
2020-04-06 12:16:35 -07:00
analysis = tune.run(
2020-04-27 18:01:00 -07:00
Trainable,
stop={"training_iteration": 20},
2020-04-06 12:16:35 -07:00
config={
2020-04-27 18:01:00 -07:00
"a": 2,
"b": 4
2020-04-06 12:16:35 -07:00
})
2020-04-27 18:01:00 -07:00
print('best config: ', analysis.get_best_config(metric="score", mode="max"))
2020-04-06 12:16:35 -07:00
2020-04-27 18:01:00 -07:00
As a subclass of `` tune.Trainable `` , Tune will create a `` Trainable `` object on a separate process (using the :ref: `Ray Actor API <actor-guide>` ).
2020-04-06 12:16:35 -07:00
2020-07-01 11:00:00 -07:00
1. `` setup `` function is invoked once training starts.
2. `` step `` is invoked **multiple times** . Each time, the Trainable object executes one logical iteration of training in the tuning process, which may include one or more iterations of actual training.
3. `` cleanup `` is invoked when training is finished.
2020-04-06 12:16:35 -07:00
2020-07-01 11:00:00 -07:00
.. tip :: As a rule of thumb, the execution time of `` step `` should be large enough to avoid overheads (i.e. more than a few seconds), but short enough to report progress periodically (i.e. at most a few minutes).
2020-04-06 12:16:35 -07:00
2020-10-10 00:54:31 -07:00
You'll notice that Ray Tune will output extra values in addition to the user reported metrics, such as `` iterations_since_restore `` . See :ref: `tune-autofilled-metrics` for an explanation/glossary of these values.
2020-04-06 12:16:35 -07:00
2020-04-27 18:01:00 -07:00
.. _tune-trainable-save-restore:
2020-06-15 10:42:54 -07:00
Class API Checkpointing
~~~~~~~~~~~~~~~~~~~~~~~
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
You can also implement checkpoint/restore using the Trainable Class API:
2020-04-06 12:16:35 -07:00
.. code-block :: python
class MyTrainableClass(Trainable):
2020-07-01 11:00:00 -07:00
def save_checkpoint(self, tmp_checkpoint_dir):
2020-04-06 12:16:35 -07:00
checkpoint_path = os.path.join(tmp_checkpoint_dir, "model.pth")
torch.save(self.model.state_dict(), checkpoint_path)
return tmp_checkpoint_dir
2020-07-01 11:00:00 -07:00
def load_checkpoint(self, tmp_checkpoint_dir):
2020-04-06 12:16:35 -07:00
checkpoint_path = os.path.join(tmp_checkpoint_dir, "model.pth")
self.model.load_state_dict(torch.load(checkpoint_path))
2020-06-15 10:42:54 -07:00
tune.run(MyTrainableClass, checkpoint_freq=2)
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
You can checkpoint with three different mechanisms: manually, periodically, and at termination.
2020-04-06 12:16:35 -07:00
2020-07-01 11:00:00 -07:00
**Manual Checkpointing** : A custom Trainable can manually trigger checkpointing by returning `` should_checkpoint: True `` (or `` tune.result.SHOULD_CHECKPOINT: True `` ) in the result dictionary of `step` . This can be especially helpful in spot instances:
2020-04-06 12:16:35 -07:00
.. code-block :: python
2020-07-01 11:00:00 -07:00
def step(self):
2020-06-15 10:42:54 -07:00
# training code
result = {"mean_accuracy": accuracy}
if detect_instance_preemption():
result.update(should_checkpoint=True)
return result
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
**Periodic Checkpointing** : periodic checkpointing can be used to provide fault-tolerance for experiments. This can be enabled by setting `` checkpoint_freq=<int> `` and `` max_failures=<int> `` to checkpoint trials every *N* iterations and recover from up to *M* crashes per trial, e.g.:
2020-04-25 18:25:56 -07:00
2020-06-15 10:42:54 -07:00
.. code-block :: python
2020-04-25 18:25:56 -07:00
2020-06-15 10:42:54 -07:00
tune.run(
my_trainable,
checkpoint_freq=10,
max_failures=5,
)
**Checkpointing at Termination** : The checkpoint_freq may not coincide with the exact end of an experiment. If you want a checkpoint to be created at the end
of a trial, you can additionally set the `` checkpoint_at_end=True `` :
2020-04-25 18:25:56 -07:00
.. code-block :: python
2020-06-15 10:42:54 -07:00
:emphasize-lines: 5
2020-04-25 18:25:56 -07:00
tune.run(
my_trainable,
2020-06-15 10:42:54 -07:00
checkpoint_freq=10,
checkpoint_at_end=True,
max_failures=5,
2020-04-25 18:25:56 -07:00
)
2020-07-01 11:00:00 -07:00
Use `` validate_save_restore `` to catch `` save_checkpoint `` /`` load_checkpoint `` errors before execution.
2020-06-15 10:42:54 -07:00
.. code-block :: python
from ray.tune.utils import validate_save_restore
# both of these should return
validate_save_restore(MyTrainableClass)
validate_save_restore(MyTrainableClass, use_object_store=True)
2020-04-25 18:25:56 -07:00
2020-04-06 12:16:35 -07:00
Advanced: Reusing Actors
~~~~~~~~~~~~~~~~~~~~~~~~
2020-06-15 10:42:54 -07:00
.. note :: This feature is only for the Trainable Class API.
2020-04-06 12:16:35 -07:00
Your Trainable can often take a long time to start. To avoid this, you can do `` tune.run(reuse_actors=True) `` to reuse the same Trainable Python process and object for multiple hyperparameters.
This requires you to implement `` Trainable.reset_config `` , which provides a new set of hyperparameters. It is up to the user to correctly update the hyperparameters of your trainable.
.. code-block :: python
class PytorchTrainble(tune.Trainable):
"""Train a Pytorch ConvNet."""
2020-07-01 11:00:00 -07:00
def setup(self, config):
2020-04-06 12:16:35 -07:00
self.train_loader, self.test_loader = get_data_loaders()
self.model = ConvNet()
self.optimizer = optim.SGD(
self.model.parameters(),
lr=config.get("lr", 0.01),
momentum=config.get("momentum", 0.9))
def reset_config(self, new_config):
for param_group in self.optimizer.param_groups:
if "lr" in new_config:
param_group["lr"] = new_config["lr"]
if "momentum" in new_config:
param_group["momentum"] = new_config["momentum"]
self.model = ConvNet()
self.config = new_config
return True
2020-03-23 12:23:21 -07:00
2020-06-15 10:42:54 -07:00
Advanced Resource Allocation
----------------------------
2020-04-06 12:16:35 -07:00
2021-02-23 18:46:02 +01:00
Trainables can themselves be distributed. If your trainable function / class creates further Ray actors or tasks that also consume CPU / GPU resources, you will want to add more bundles to the :class: `PlacementGroupFactory` to reserve extra resource slots. For example, if a trainable class requires 1 GPU itself, but also launches 4 actors, each using another GPU, then you should use this:
2020-03-23 12:23:21 -07:00
2020-06-15 10:42:54 -07:00
.. code-block :: python
2021-02-23 18:46:02 +01:00
:emphasize-lines: 4-10
2020-06-15 10:42:54 -07:00
tune.run(
my_trainable,
name="my_trainable",
2021-02-23 18:46:02 +01:00
resources_per_trial=tune.PlacementGroupFactory([
{"CPU": 1, "GPU": 1},
{"GPU": 1},
{"GPU": 1},
{"GPU": 1},
{"GPU": 1}
])
2020-06-15 10:42:54 -07:00
)
The `` Trainable `` also provides the `` default_resource_requests `` interface to automatically declare the `` resources_per_trial `` based on the given configuration.
2020-03-23 12:23:21 -07:00
2020-07-05 01:16:20 -07:00
.. _tune-function-docstring:
2020-03-23 12:23:21 -07:00
2020-06-15 10:42:54 -07:00
tune.report / tune.checkpoint (Function API)
--------------------------------------------
2020-03-23 12:23:21 -07:00
2020-06-15 10:42:54 -07:00
.. autofunction :: ray.tune.report
2020-07-30 09:46:37 -07:00
.. autofunction :: ray.tune.checkpoint_dir
2020-06-15 10:42:54 -07:00
.. autofunction :: ray.tune.get_trial_dir
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
.. autofunction :: ray.tune.get_trial_name
2020-04-06 12:16:35 -07:00
2020-06-15 10:42:54 -07:00
.. autofunction :: ray.tune.get_trial_id
tune.Trainable (Class API)
--------------------------
.. autoclass :: ray.tune.Trainable
:member-order: groupwise
:private-members:
2020-04-06 12:16:35 -07:00
:members:
2020-10-10 00:54:31 -07:00
.. _tune-util-ref:
Utilities
---------
.. autofunction :: ray.tune.utils.wait_for_gpu
.. autofunction :: ray.tune.utils.diagnose_serialization
.. autofunction :: ray.tune.utils.validate_save_restore
2020-07-30 09:46:37 -07:00
.. _tune-ddp-doc:
Distributed Torch
-----------------
2020-11-10 14:59:08 -08:00
Ray offers lightweight integrations to distribute your PyTorch training on Ray Tune.
2020-07-30 09:46:37 -07:00
.. autofunction :: ray.tune.integration.torch.DistributedTrainableCreator
2020-08-31 23:30:48 +01:00
:noindex:
2020-07-30 09:46:37 -07:00
.. autofunction :: ray.tune.integration.torch.distributed_checkpoint_dir
2020-08-31 23:30:48 +01:00
:noindex:
2020-07-30 09:46:37 -07:00
.. autofunction :: ray.tune.integration.torch.is_distributed_trainable
2020-08-31 23:30:48 +01:00
:noindex:
2020-07-30 09:46:37 -07:00
2020-11-10 14:59:08 -08:00
.. _tune-dist-tf-doc:
Distributed TensorFlow
----------------------
Ray also offers lightweight integrations to distribute your TensorFlow training on Ray Tune.
.. autofunction :: ray.tune.integration.tensorflow.DistributedTrainableCreator
:noindex:
2021-02-26 13:59:28 +01:00
.. _tune-durable-trainable:
2020-06-15 10:42:54 -07:00
tune.DurableTrainable
---------------------
2021-02-26 13:59:28 +01:00
Tune provides a :func: `ray.tune.durable` wrapper that can be used to convert any kind of trainable
to a `` DurableTrainable `` , including pre-registered RLLib trainers and :ref: `function trainables <tune-function-api>` .
The :class: `DurableTrainable <ray.tune.DurableTrainable>` syncs trial logs and checkpoints to cloud storage (via the `upload_dir` ). This is especially
useful when training a large number of distributed trials, as logs and checkpoints are otherwise synchronized
via SSH, which quickly can become a performance bottleneck. The :class: `DurableTrainable <ray.tune.DurableTrainable>` class inherits from
:class: `Trainable <ray.tune.Trainable>` and thus can be extended like the base class.
2020-06-15 10:42:54 -07:00
.. autoclass :: ray.tune.DurableTrainable
2021-02-26 13:59:28 +01:00
.. autofunction :: ray.tune.durable
2020-03-23 12:23:21 -07:00
StatusReporter
2020-04-06 12:16:35 -07:00
--------------
2020-03-23 12:23:21 -07:00
.. autoclass :: ray.tune.function_runner.StatusReporter
:members: __call__, logdir