2020-04-30 01:18:09 -07:00
|
|
|
import logging
|
2020-04-10 00:56:08 -07:00
|
|
|
from typing import List, Tuple
|
|
|
|
import time
|
|
|
|
|
2021-12-21 08:39:05 +01:00
|
|
|
import ray
|
2020-04-10 00:56:08 -07:00
|
|
|
from ray.util.iter import from_actors, LocalIterator
|
|
|
|
from ray.util.iter_metrics import SharedMetrics
|
|
|
|
from ray.rllib.evaluation.rollout_worker import get_global_worker
|
|
|
|
from ray.rllib.evaluation.worker_set import WorkerSet
|
2021-03-18 20:27:41 +01:00
|
|
|
from ray.rllib.execution.common import AGENT_STEPS_SAMPLED_COUNTER, \
|
2021-09-30 16:39:05 +02:00
|
|
|
STEPS_SAMPLED_COUNTER, SAMPLE_TIMER, GRAD_WAIT_TIMER, \
|
2021-03-18 20:27:41 +01:00
|
|
|
_check_sample_batch_type, _get_shared_metrics
|
2020-04-30 01:18:09 -07:00
|
|
|
from ray.rllib.policy.sample_batch import SampleBatch, DEFAULT_POLICY_ID, \
|
|
|
|
MultiAgentBatch
|
2021-09-30 16:39:05 +02:00
|
|
|
from ray.rllib.utils.metrics.learner_info import LEARNER_INFO, \
|
|
|
|
LEARNER_STATS_KEY
|
2020-04-30 01:18:09 -07:00
|
|
|
from ray.rllib.utils.sgd import standardized
|
2020-08-15 13:24:22 +02:00
|
|
|
from ray.rllib.utils.typing import PolicyID, SampleBatchType, ModelGradients
|
2020-04-30 01:18:09 -07:00
|
|
|
|
|
|
|
logger = logging.getLogger(__name__)
|
2020-04-10 00:56:08 -07:00
|
|
|
|
|
|
|
|
2021-12-21 08:39:05 +01:00
|
|
|
def synchronous_parallel_sample(workers: WorkerSet) -> List[SampleBatch]:
|
|
|
|
# No remote workers in the set -> Use local worker for collecting
|
|
|
|
# samples.
|
|
|
|
if not workers.remote_workers():
|
|
|
|
return [workers.local_worker().sample()]
|
|
|
|
|
|
|
|
# Loop over remote workers' `sample()` method in parallel.
|
|
|
|
sample_batches = ray.get(
|
|
|
|
[r.sample.remote() for r in workers.remote_workers()])
|
|
|
|
|
|
|
|
return sample_batches
|
|
|
|
|
|
|
|
|
2020-05-05 01:38:10 -07:00
|
|
|
def ParallelRollouts(workers: WorkerSet, *, mode="bulk_sync",
|
|
|
|
num_async=1) -> LocalIterator[SampleBatch]:
|
2020-04-10 00:56:08 -07:00
|
|
|
"""Operator to collect experiences in parallel from rollout workers.
|
|
|
|
|
|
|
|
If there are no remote workers, experiences will be collected serially from
|
|
|
|
the local worker instance instead.
|
|
|
|
|
2020-09-20 11:27:02 +02:00
|
|
|
Args:
|
2020-04-10 00:56:08 -07:00
|
|
|
workers (WorkerSet): set of rollout workers to use.
|
2020-09-21 23:03:06 -07:00
|
|
|
mode (str): One of 'async', 'bulk_sync', 'raw'. In 'async' mode,
|
|
|
|
batches are returned as soon as they are computed by rollout
|
|
|
|
workers with no order guarantees. In 'bulk_sync' mode, we collect
|
|
|
|
one batch from each worker and concatenate them together into a
|
|
|
|
large batch to return. In 'raw' mode, the ParallelIterator object
|
|
|
|
is returned directly and the caller is responsible for implementing
|
|
|
|
gather and updating the timesteps counter.
|
2020-05-05 01:38:10 -07:00
|
|
|
num_async (int): In async mode, the max number of async
|
2020-04-10 00:56:08 -07:00
|
|
|
requests in flight per actor.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
A local iterator over experiences collected in parallel.
|
|
|
|
|
|
|
|
Examples:
|
|
|
|
>>> rollouts = ParallelRollouts(workers, mode="async")
|
|
|
|
>>> batch = next(rollouts)
|
|
|
|
>>> print(batch.count)
|
|
|
|
50 # config.rollout_fragment_length
|
|
|
|
|
|
|
|
>>> rollouts = ParallelRollouts(workers, mode="bulk_sync")
|
|
|
|
>>> batch = next(rollouts)
|
|
|
|
>>> print(batch.count)
|
|
|
|
200 # config.rollout_fragment_length * config.num_workers
|
|
|
|
|
|
|
|
Updates the STEPS_SAMPLED_COUNTER counter in the local iterator context.
|
|
|
|
"""
|
|
|
|
|
|
|
|
# Ensure workers are initially in sync.
|
|
|
|
workers.sync_weights()
|
|
|
|
|
|
|
|
def report_timesteps(batch):
|
2020-05-21 10:16:18 -07:00
|
|
|
metrics = _get_shared_metrics()
|
2020-04-10 00:56:08 -07:00
|
|
|
metrics.counters[STEPS_SAMPLED_COUNTER] += batch.count
|
2021-03-18 20:27:41 +01:00
|
|
|
if isinstance(batch, MultiAgentBatch):
|
|
|
|
metrics.counters[AGENT_STEPS_SAMPLED_COUNTER] += \
|
|
|
|
batch.agent_steps()
|
|
|
|
else:
|
|
|
|
metrics.counters[AGENT_STEPS_SAMPLED_COUNTER] += batch.count
|
2020-04-10 00:56:08 -07:00
|
|
|
return batch
|
|
|
|
|
|
|
|
if not workers.remote_workers():
|
2021-06-30 12:32:11 +02:00
|
|
|
# Handle the `num_workers=0` case, in which the local worker
|
|
|
|
# has to do sampling as well.
|
2020-04-10 00:56:08 -07:00
|
|
|
def sampler(_):
|
|
|
|
while True:
|
|
|
|
yield workers.local_worker().sample()
|
|
|
|
|
2021-08-17 13:01:41 +01:00
|
|
|
return (LocalIterator(sampler,
|
|
|
|
SharedMetrics()).for_each(report_timesteps))
|
2020-04-10 00:56:08 -07:00
|
|
|
|
|
|
|
# Create a parallel iterator over generated experiences.
|
|
|
|
rollouts = from_actors(workers.remote_workers())
|
|
|
|
|
|
|
|
if mode == "bulk_sync":
|
|
|
|
return rollouts \
|
|
|
|
.batch_across_shards() \
|
|
|
|
.for_each(lambda batches: SampleBatch.concat_samples(batches)) \
|
|
|
|
.for_each(report_timesteps)
|
|
|
|
elif mode == "async":
|
|
|
|
return rollouts.gather_async(
|
2020-05-05 01:38:10 -07:00
|
|
|
num_async=num_async).for_each(report_timesteps)
|
2020-04-30 01:18:09 -07:00
|
|
|
elif mode == "raw":
|
|
|
|
return rollouts
|
2020-04-10 00:56:08 -07:00
|
|
|
else:
|
2020-04-30 01:18:09 -07:00
|
|
|
raise ValueError("mode must be one of 'bulk_sync', 'async', 'raw', "
|
|
|
|
"got '{}'".format(mode))
|
2020-04-10 00:56:08 -07:00
|
|
|
|
|
|
|
|
|
|
|
def AsyncGradients(
|
2020-06-19 13:09:05 -07:00
|
|
|
workers: WorkerSet) -> LocalIterator[Tuple[ModelGradients, int]]:
|
2020-04-10 00:56:08 -07:00
|
|
|
"""Operator to compute gradients in parallel from rollout workers.
|
|
|
|
|
2020-09-20 11:27:02 +02:00
|
|
|
Args:
|
2020-04-10 00:56:08 -07:00
|
|
|
workers (WorkerSet): set of rollout workers to use.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
A local iterator over policy gradients computed on rollout workers.
|
|
|
|
|
|
|
|
Examples:
|
|
|
|
>>> grads_op = AsyncGradients(workers)
|
|
|
|
>>> print(next(grads_op))
|
|
|
|
{"var_0": ..., ...}, 50 # grads, batch count
|
|
|
|
|
|
|
|
Updates the STEPS_SAMPLED_COUNTER counter and LEARNER_INFO field in the
|
|
|
|
local iterator context.
|
|
|
|
"""
|
|
|
|
|
|
|
|
# Ensure workers are initially in sync.
|
|
|
|
workers.sync_weights()
|
|
|
|
|
|
|
|
# This function will be applied remotely on the workers.
|
|
|
|
def samples_to_grads(samples):
|
|
|
|
return get_global_worker().compute_gradients(samples), samples.count
|
|
|
|
|
|
|
|
# Record learner metrics and pass through (grads, count).
|
|
|
|
class record_metrics:
|
|
|
|
def _on_fetch_start(self):
|
|
|
|
self.fetch_start_time = time.perf_counter()
|
|
|
|
|
|
|
|
def __call__(self, item):
|
|
|
|
(grads, info), count = item
|
2020-05-21 10:16:18 -07:00
|
|
|
metrics = _get_shared_metrics()
|
2020-04-10 00:56:08 -07:00
|
|
|
metrics.counters[STEPS_SAMPLED_COUNTER] += count
|
2021-09-30 16:39:05 +02:00
|
|
|
metrics.info[LEARNER_INFO] = {
|
|
|
|
DEFAULT_POLICY_ID: info
|
|
|
|
} if LEARNER_STATS_KEY in info else info
|
2020-04-10 00:56:08 -07:00
|
|
|
metrics.timers[GRAD_WAIT_TIMER].push(time.perf_counter() -
|
|
|
|
self.fetch_start_time)
|
|
|
|
return grads, count
|
|
|
|
|
|
|
|
rollouts = from_actors(workers.remote_workers())
|
|
|
|
grads = rollouts.for_each(samples_to_grads)
|
|
|
|
return grads.gather_async().for_each(record_metrics())
|
|
|
|
|
|
|
|
|
|
|
|
class ConcatBatches:
|
|
|
|
"""Callable used to merge batches into larger batches for training.
|
|
|
|
|
|
|
|
This should be used with the .combine() operator.
|
|
|
|
|
|
|
|
Examples:
|
|
|
|
>>> rollouts = ParallelRollouts(...)
|
2020-12-09 01:41:45 +01:00
|
|
|
>>> rollouts = rollouts.combine(ConcatBatches(
|
|
|
|
... min_batch_size=10000, count_steps_by="env_steps"))
|
2020-04-10 00:56:08 -07:00
|
|
|
>>> print(next(rollouts).count)
|
|
|
|
10000
|
|
|
|
"""
|
|
|
|
|
2020-12-09 01:41:45 +01:00
|
|
|
def __init__(self, min_batch_size: int, count_steps_by: str = "env_steps"):
|
2020-04-10 00:56:08 -07:00
|
|
|
self.min_batch_size = min_batch_size
|
2020-12-09 01:41:45 +01:00
|
|
|
self.count_steps_by = count_steps_by
|
2020-04-10 00:56:08 -07:00
|
|
|
self.buffer = []
|
|
|
|
self.count = 0
|
2021-08-17 13:01:41 +01:00
|
|
|
self.last_batch_time = time.perf_counter()
|
2020-04-10 00:56:08 -07:00
|
|
|
|
|
|
|
def __call__(self, batch: SampleBatchType) -> List[SampleBatchType]:
|
|
|
|
_check_sample_batch_type(batch)
|
2020-12-09 01:41:45 +01:00
|
|
|
|
|
|
|
if self.count_steps_by == "env_steps":
|
2021-09-30 08:30:20 +02:00
|
|
|
size = batch.count
|
2020-12-09 01:41:45 +01:00
|
|
|
else:
|
|
|
|
assert isinstance(batch, MultiAgentBatch), \
|
|
|
|
"`count_steps_by=agent_steps` only allowed in multi-agent " \
|
|
|
|
"environments!"
|
2021-09-30 08:30:20 +02:00
|
|
|
size = batch.agent_steps()
|
|
|
|
|
|
|
|
# Incoming batch is an empty dummy batch -> Ignore.
|
|
|
|
# Possibly produced automatically by a PolicyServer to unblock
|
|
|
|
# an external env waiting for inputs from unresponsive/disconnected
|
|
|
|
# client(s).
|
|
|
|
if size == 0:
|
|
|
|
return []
|
|
|
|
|
|
|
|
self.count += size
|
|
|
|
self.buffer.append(batch)
|
2020-12-09 01:41:45 +01:00
|
|
|
|
2020-04-10 00:56:08 -07:00
|
|
|
if self.count >= self.min_batch_size:
|
2020-04-30 01:18:09 -07:00
|
|
|
if self.count > self.min_batch_size * 2:
|
|
|
|
logger.info("Collected more training samples than expected "
|
|
|
|
"(actual={}, expected={}). ".format(
|
|
|
|
self.count, self.min_batch_size) +
|
|
|
|
"This may be because you have many workers or "
|
|
|
|
"long episodes in 'complete_episodes' batch mode.")
|
2020-04-10 00:56:08 -07:00
|
|
|
out = SampleBatch.concat_samples(self.buffer)
|
2021-08-17 13:01:41 +01:00
|
|
|
|
|
|
|
perf_counter = time.perf_counter()
|
2020-05-21 10:16:18 -07:00
|
|
|
timer = _get_shared_metrics().timers[SAMPLE_TIMER]
|
2021-08-17 13:01:41 +01:00
|
|
|
timer.push(perf_counter - self.last_batch_time)
|
2020-04-10 00:56:08 -07:00
|
|
|
timer.push_units_processed(self.count)
|
2021-08-17 13:01:41 +01:00
|
|
|
|
|
|
|
self.last_batch_time = perf_counter
|
2020-04-10 00:56:08 -07:00
|
|
|
self.buffer = []
|
|
|
|
self.count = 0
|
|
|
|
return [out]
|
|
|
|
return []
|
2020-04-30 01:18:09 -07:00
|
|
|
|
|
|
|
|
|
|
|
class SelectExperiences:
|
|
|
|
"""Callable used to select experiences from a MultiAgentBatch.
|
|
|
|
|
|
|
|
This should be used with the .for_each() operator.
|
|
|
|
|
|
|
|
Examples:
|
|
|
|
>>> rollouts = ParallelRollouts(...)
|
|
|
|
>>> rollouts = rollouts.for_each(SelectExperiences(["pol1", "pol2"]))
|
|
|
|
>>> print(next(rollouts).policy_batches.keys())
|
|
|
|
{"pol1", "pol2"}
|
|
|
|
"""
|
|
|
|
|
2020-05-07 23:40:29 -07:00
|
|
|
def __init__(self, policy_ids: List[PolicyID]):
|
|
|
|
assert isinstance(policy_ids, list), policy_ids
|
2020-04-30 01:18:09 -07:00
|
|
|
self.policy_ids = policy_ids
|
|
|
|
|
|
|
|
def __call__(self, samples: SampleBatchType) -> SampleBatchType:
|
|
|
|
_check_sample_batch_type(samples)
|
|
|
|
|
|
|
|
if isinstance(samples, MultiAgentBatch):
|
|
|
|
samples = MultiAgentBatch({
|
|
|
|
k: v
|
|
|
|
for k, v in samples.policy_batches.items()
|
|
|
|
if k in self.policy_ids
|
|
|
|
}, samples.count)
|
|
|
|
|
|
|
|
return samples
|
|
|
|
|
|
|
|
|
|
|
|
class StandardizeFields:
|
|
|
|
"""Callable used to standardize fields of batches.
|
|
|
|
|
|
|
|
This should be used with the .for_each() operator. Note that the input
|
|
|
|
may be mutated by this operator for efficiency.
|
|
|
|
|
|
|
|
Examples:
|
|
|
|
>>> rollouts = ParallelRollouts(...)
|
|
|
|
>>> rollouts = rollouts.for_each(StandardizeFields(["advantages"]))
|
|
|
|
>>> print(np.std(next(rollouts)["advantages"]))
|
|
|
|
1.0
|
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, fields: List[str]):
|
|
|
|
self.fields = fields
|
|
|
|
|
|
|
|
def __call__(self, samples: SampleBatchType) -> SampleBatchType:
|
|
|
|
_check_sample_batch_type(samples)
|
|
|
|
wrapped = False
|
|
|
|
|
|
|
|
if isinstance(samples, SampleBatch):
|
2022-01-05 18:22:33 +01:00
|
|
|
samples = samples.as_multi_agent()
|
2020-04-30 01:18:09 -07:00
|
|
|
wrapped = True
|
|
|
|
|
|
|
|
for policy_id in samples.policy_batches:
|
|
|
|
batch = samples.policy_batches[policy_id]
|
|
|
|
for field in self.fields:
|
2022-01-11 19:50:03 +01:00
|
|
|
if field not in batch:
|
|
|
|
raise KeyError(
|
|
|
|
f"`{field}` not found in SampleBatch for policy "
|
|
|
|
f"`{policy_id}`! Maybe this policy fails to add "
|
|
|
|
f"{field} in its `postprocess_trajectory` method? Or "
|
|
|
|
"this policy is not meant to learn at all and you "
|
|
|
|
"forgot to add it to the list under `config."
|
|
|
|
"multiagent.policies_to_train`.")
|
2020-04-30 01:18:09 -07:00
|
|
|
batch[field] = standardized(batch[field])
|
|
|
|
|
|
|
|
if wrapped:
|
|
|
|
samples = samples.policy_batches[DEFAULT_POLICY_ID]
|
|
|
|
|
|
|
|
return samples
|