Skip to content

Commit

Permalink
[rllib] Allow envs to be auto-registered; add on_train_result callbac…
Browse files Browse the repository at this point in the history
…k with curriculum example (#3451)

* train step and docs

* debug

* doc

* doc

* fix examples

* fix code

* integration test

* fix

* ...

* space

* instance

* Update .travis.yml

* fix test
  • Loading branch information
ericl authored Dec 4, 2018
1 parent be6567e commit ce355d1
Show file tree
Hide file tree
Showing 13 changed files with 191 additions and 242 deletions.
30 changes: 15 additions & 15 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,21 @@ script:
# module is only found if the test directory is in the PYTHONPATH.
- export PYTHONPATH="$PYTHONPATH:./test/"

# ray tune tests
- python python/ray/tune/test/dependency_test.py
- python -m pytest -v python/ray/tune/test/trial_runner_test.py
- python -m pytest -v python/ray/tune/test/trial_scheduler_test.py
- python -m pytest -v python/ray/tune/test/experiment_test.py
- python -m pytest -v python/ray/tune/test/tune_server_test.py
- python -m pytest -v python/ray/tune/test/ray_trial_executor_test.py
- python -m pytest -v python/ray/tune/test/automl_searcher_test.py

# ray rllib tests
- python -m pytest -v python/ray/rllib/test/test_catalog.py
- python -m pytest -v python/ray/rllib/test/test_filters.py
- python -m pytest -v python/ray/rllib/test/test_optimizers.py
- python -m pytest -v python/ray/rllib/test/test_evaluators.py

- python -m pytest -v python/ray/test/test_global_state.py
- python -m pytest -v python/ray/test/test_queue.py
- python -m pytest -v python/ray/test/test_ray_init.py
Expand All @@ -153,21 +168,6 @@ script:
- python -m pytest -v test/credis_test.py
- python -m pytest -v test/node_manager_test.py

# ray tune tests
- python python/ray/tune/test/dependency_test.py
- python -m pytest -v python/ray/tune/test/trial_runner_test.py
- python -m pytest -v python/ray/tune/test/trial_scheduler_test.py
- python -m pytest -v python/ray/tune/test/experiment_test.py
- python -m pytest -v python/ray/tune/test/tune_server_test.py
- python -m pytest -v python/ray/tune/test/ray_trial_executor_test.py
- python -m pytest -v python/ray/tune/test/automl_searcher_test.py

# ray rllib tests
- python -m pytest -v python/ray/rllib/test/test_catalog.py
- python -m pytest -v python/ray/rllib/test/test_filters.py
- python -m pytest -v python/ray/rllib/test/test_optimizers.py
- python -m pytest -v python/ray/rllib/test/test_evaluators.py

# ray temp file tests
- python -m pytest -v test/tempfile_test.py

Expand Down
28 changes: 20 additions & 8 deletions doc/source/rllib-env.rst
Original file line number Diff line number Diff line change
Expand Up @@ -24,27 +24,39 @@ ARS **Yes** **Yes** No No

.. _`+parametric`: rllib-models.html#variable-length-parametric-action-spaces

In the high-level agent APIs, environments are identified with string names. By default, the string will be interpreted as a gym `environment name <https://gym.openai.com/envs>`__, however you can also register custom environments by name:
You can pass either a string name or a Python class to specify an environment. By default, strings will be interpreted as a gym `environment name <https://gym.openai.com/envs>`__. Custom env classes must take a single ``env_config`` parameter in their constructor:

.. code-block:: python
import ray
from ray.tune.registry import register_env
from ray.rllib.agents import ppo
def env_creator(env_config):
import gym
return gym.make("CartPole-v0") # or return your own custom env
class MyEnv(gym.Env):
def __init__(self, env_config):
self.action_space = ...
self.observation_space = ...
...
register_env("my_env", env_creator)
ray.init()
trainer = ppo.PPOAgent(env="my_env", config={
"env_config": {}, # config to pass to env creator
trainer = ppo.PPOAgent(env=MyEnv, config={
"env_config": {}, # config to pass to env class
})
while True:
print(trainer.train())
You can also register a custom env creator function with a string name. This function must take a single ``env_config`` parameter and return an env instance:

.. code-block:: python
from ray.tune.registry import register_env
def env_creator(env_config):
return MyEnv(...) # return an env instance
register_env("my_env", env_creator)
trainer = ppo.PPOAgent(env="my_env")
Configuring Environments
------------------------

Expand Down
141 changes: 112 additions & 29 deletions doc/source/rllib-training.rst
Original file line number Diff line number Diff line change
Expand Up @@ -224,35 +224,6 @@ Sometimes, it is necessary to coordinate between pieces of code that live in dif
Ray actors provide high levels of performance, so in more complex cases they can be used implement communication patterns such as parameter servers and allreduce.

Debugging
---------

Gym Monitor
~~~~~~~~~~~

The ``"monitor": true`` config can be used to save Gym episode videos to the result dir. For example:

.. code-block:: bash
python ray/python/ray/rllib/train.py --env=PongDeterministic-v4 \
--run=A2C --config '{"num_workers": 2, "monitor": true}'
# videos will be saved in the ~/ray_results/<experiment> dir, for example
openaigym.video.0.31401.video000000.meta.json
openaigym.video.0.31401.video000000.mp4
openaigym.video.0.31403.video000000.meta.json
openaigym.video.0.31403.video000000.mp4
Log Verbosity
~~~~~~~~~~~~~

You can control the agent log level via the ``"log_level"`` flag. Valid values are "INFO" (default), "DEBUG", "WARN", and "ERROR". This can be used to increase or decrease the verbosity of internal logging. For example:

.. code-block:: bash
python ray/python/ray/rllib/train.py --env=PongDeterministic-v4 \
--run=A2C --config '{"num_workers": 2, "log_level": "DEBUG"}'
Callbacks and Custom Metrics
~~~~~~~~~~~~~~~~~~~~~~~~~~~~

Expand All @@ -278,6 +249,10 @@ You can provide callback functions to be called at points during policy evaluati
episode.episode_id, episode.length, mean_pole_angle))
episode.custom_metrics["mean_pole_angle"] = mean_pole_angle
def on_train_result(info):
print("agent.train() result: {} -> {} episodes".format(
info["agent"].__name__, info["result"]["episodes_this_iter"]))
ray.init()
trials = tune.run_experiments({
"test": {
Expand All @@ -288,6 +263,7 @@ You can provide callback functions to be called at points during policy evaluati
"on_episode_start": tune.function(on_episode_start),
"on_episode_step": tune.function(on_episode_step),
"on_episode_end": tune.function(on_episode_end),
"on_train_result": tune.function(on_train_result),
},
},
}
Expand All @@ -297,6 +273,113 @@ Custom metrics can be accessed and visualized like any other training result:

.. image:: custom_metric.png

Example: Curriculum Learning
~~~~~~~~~~~~~~~~~~~~~~~~~~~~

Let's look at two ways to use the above APIs to implement `curriculum learning <https://bair.berkeley.edu/blog/2017/12/20/reverse-curriculum/>`__. In curriculum learning, the agent task is adjusted over time to improve the learning process. Suppose that we have an environment class with a ``set_phase()`` method that we can call to adjust the task difficulty over time:

Approach 1: Use the Agent API and update the environment between calls to ``train()``. This example shows the agent being run inside a Tune function:

.. code-block:: python
import ray
from ray import tune
from ray.rllib.agents.ppo import PPOAgent
def train(config, reporter):
agent = PPOAgent(config=config, env=YourEnv)
while True:
result = agent.train()
reporter(**result)
if result["episode_reward_mean"] > 200:
phase = 2
elif result["episode_reward_mean"] > 100:
phase = 1
else:
phase = 0
agent.optimizer.foreach_evaluator(lambda ev: ev.env.set_phase(phase))
ray.init()
tune.run_experiments({
"curriculum": {
"run": train,
"config": {
"num_gpus": 0,
"num_workers": 2,
},
"trial_resources": {
"cpu": 1,
"gpu": lambda spec: spec.config.num_gpus,
"extra_cpu": lambda spec: spec.config.num_workers,
},
},
})
Approach 2: Use the callbacks API to update the environment on new training results:

.. code-block:: python
import ray
from ray import tune
def on_train_result(info):
result = info["result"]
if result["episode_reward_mean"] > 200:
phase = 2
elif result["episode_reward_mean"] > 100:
phase = 1
else:
phase = 0
agent = info["agent"]
agent.optimizer.foreach_evaluator(lambda ev: ev.env.set_phase(phase))
ray.init()
tune.run_experiments({
"curriculum": {
"run": "PPO",
"env": YourEnv,
"config": {
"callbacks": {
"on_train_result": tune.function(on_train_result),
},
},
},
})
Debugging
---------

Gym Monitor
~~~~~~~~~~~

The ``"monitor": true`` config can be used to save Gym episode videos to the result dir. For example:

.. code-block:: bash
python ray/python/ray/rllib/train.py --env=PongDeterministic-v4 \
--run=A2C --config '{"num_workers": 2, "monitor": true}'
# videos will be saved in the ~/ray_results/<experiment> dir, for example
openaigym.video.0.31401.video000000.meta.json
openaigym.video.0.31401.video000000.mp4
openaigym.video.0.31403.video000000.meta.json
openaigym.video.0.31403.video000000.mp4
Log Verbosity
~~~~~~~~~~~~~

You can control the agent log level via the ``"log_level"`` flag. Valid values are "INFO" (default), "DEBUG", "WARN", and "ERROR". This can be used to increase or decrease the verbosity of internal logging. For example:

.. code-block:: bash
python ray/python/ray/rllib/train.py --env=PongDeterministic-v4 \
--run=A2C --config '{"num_workers": 2, "log_level": "DEBUG"}'
Stack Traces
~~~~~~~~~~~~

You can use the ``ray stack`` command to dump the stack traces of all the Python workers on a single node. This can be useful for debugging unexpected hangs or performance issues.

REST API
--------

Expand Down
27 changes: 22 additions & 5 deletions python/ray/rllib/agents/agent.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,20 +2,21 @@
from __future__ import division
from __future__ import print_function

from datetime import datetime
import copy
import os
import logging
import os
import pickle
import six
import tempfile
from datetime import datetime
import tensorflow as tf

import ray
from ray.rllib.models import MODEL_DEFAULTS
from ray.rllib.evaluation.policy_evaluator import PolicyEvaluator
from ray.rllib.optimizers.policy_optimizer import PolicyOptimizer
from ray.rllib.utils import FilterManager, deep_update, merge_dicts
from ray.tune.registry import ENV_CREATOR, _global_registry
from ray.tune.registry import ENV_CREATOR, register_env, _global_registry
from ray.tune.trainable import Trainable
from ray.tune.trial import Resources
from ray.tune.logger import UnifiedLogger
Expand All @@ -40,6 +41,7 @@
"on_episode_step": None, # arg: {"env": .., "episode": ...}
"on_episode_end": None, # arg: {"env": .., "episode": ...}
"on_sample_end": None, # arg: {"samples": .., "evaluator": ...}
"on_train_result": None, # arg: {"agent": ..., "result": ...}
},

# === Policy ===
Expand Down Expand Up @@ -277,7 +279,7 @@ def __init__(self, config=None, env=None, logger_creator=None):
self.global_vars = {"timestep": 0}

# Agents allow env ids to be passed directly to the constructor.
self._env_id = env or config.get("env")
self._env_id = _register_if_needed(env or config.get("env"))

# Create a default logger creator if no logger_creator is specified
if logger_creator is None:
Expand Down Expand Up @@ -319,7 +321,13 @@ def train(self):
logger.debug("synchronized filters: {}".format(
self.local_evaluator.filters))

return Trainable.train(self)
result = Trainable.train(self)
if self.config["callbacks"].get("on_train_result"):
self.config["callbacks"]["on_train_result"]({
"agent": self,
"result": result,
})
return result

def _setup(self, config):
env = self._env_id
Expand Down Expand Up @@ -447,6 +455,15 @@ def _restore(self, checkpoint_path):
self.__setstate__(extra_data)


def _register_if_needed(env_object):
if isinstance(env_object, six.string_types):
return env_object
elif isinstance(env_object, type):
name = env_object.__name__
register_env(name, lambda config: env_object(config))
return name


def get_agent_class(alg):
"""Returns the class of a known agent given its name."""

Expand Down
51 changes: 0 additions & 51 deletions python/ray/rllib/examples/carla/a3c_lane_keep.py

This file was deleted.

Loading

0 comments on commit ce355d1

Please sign in to comment.