From fcbc6dea478074d4c00b185bc392d6dc55b501a2 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 24 Nov 2018 20:00:27 -0800 Subject: [PATCH 01/40] Tests and better recovery handling --- python/ray/tune/ray_trial_executor.py | 28 +++++-- .../ray/tune/test/ray_trial_executor_test.py | 12 ++- python/ray/tune/test/trial_runner_test.py | 73 ++++++++++++++++--- python/ray/tune/trial.py | 8 +- python/ray/tune/trial_executor.py | 29 +------- python/ray/tune/trial_runner.py | 70 +++++++++++++++--- python/ray/tune/tune.py | 3 +- 7 files changed, 162 insertions(+), 61 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 7b725e05f342..dbee770b44cd 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -110,25 +110,39 @@ def _stop_trial(self, trial, error=False, error_msg=None, if stop_logger: trial.close_logger() - def start_trial(self, trial, checkpoint_obj=None): - """Starts the trial.""" + def start_trial(self, trial, checkpoint=None, raise_on_failure=False): + """Starts the trial. + + Will not return resources if trial repeatedly fails on start. + + Args: + trial (Trial): Trial to be started. + checkpoint(Checkpoint): A Python object or path storing the state + of trial. + raise_on_failure (bool): To raise exception on failure in starting. + + Raises: + Exception after 1 retries if `raise_on_failure` is True. + """ self._commit_resources(trial.resources) try: - self._start_trial(trial, checkpoint_obj) + self._start_trial(trial, checkpoint) except Exception: logger.exception("Error stopping runner - retrying...") error_msg = traceback.format_exc() time.sleep(2) self._stop_trial(trial, error=True, error_msg=error_msg) try: - self._start_trial(trial) - except Exception: + self._start_trial(trial, checkpoint) + except Exception as exc: logger.exception("Error starting runner, aborting!") error_msg = traceback.format_exc() self._stop_trial(trial, error=True, error_msg=error_msg) # note that we don't return the resources, since they may # have been lost + if raise_on_failure: + raise exc def _find_item(self, dictionary, item): out = [rid for rid, t in dictionary.items() if t is item] @@ -140,11 +154,13 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): self._stop_trial( trial, error=error, error_msg=error_msg, stop_logger=stop_logger) if prior_status == Trial.RUNNING: - self._return_resources(trial.resources) out = self._find_item(self._running, trial) for result_id in out: self._running.pop(result_id) + logger.debug("Returning resources for this trial.") + self._return_resources(trial.resources) + def continue_training(self, trial): """Continues the training of this trial.""" diff --git a/python/ray/tune/test/ray_trial_executor_test.py b/python/ray/tune/test/ray_trial_executor_test.py index 35c413e717bb..8e6ef765cee3 100644 --- a/python/ray/tune/test/ray_trial_executor_test.py +++ b/python/ray/tune/test/ray_trial_executor_test.py @@ -9,8 +9,9 @@ from ray.rllib import _register_all from ray.tune import Trainable from ray.tune.ray_trial_executor import RayTrialExecutor +from ray.tune.registry import _global_registry, TRAINABLE_CLASS from ray.tune.suggest import BasicVariantGenerator -from ray.tune.trial import Trial, Checkpoint +from ray.tune.trial import Trial, Checkpoint, Resources class RayTrialExecutorTest(unittest.TestCase): @@ -50,6 +51,15 @@ def testPauseResume(self): self.trial_executor.stop_trial(trial) self.assertEqual(Trial.TERMINATED, trial.status) + def testStartFailure(self): + _global_registry.register(TRAINABLE_CLASS, "asdf", None) + trial = Trial("asdf", resources=Resources(1, 0)) + self.trial_executor.start_trial(trial) + self.assertEqual(Trial.ERROR, trial.status) + self.assertRaises( + Exception, lambda: self.trial_executor.start_trial( + trial, raise_on_error=True)) + def testPauseResume2(self): """Tests that pausing works for trials being processed.""" trial = Trial("__fake") diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 6b142d354ec7..6af30e87fd3a 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -5,6 +5,7 @@ import os import time import unittest +from unittest import mock import ray from ray.rllib import _register_all @@ -845,6 +846,23 @@ def testMaxConcurrentSuggestions(self): self.assertEqual(len(searcher.next_trials()), 0) +def create_mock_components(): + + class _MockScheduler(FIFOScheduler): + errored_trials = [] + def on_trial_error(self, trial_runner, trial): + self.errored_trials += [trial] + + class _MockSearchAlg(BasicVariantGenerator): + errored_trials = [] + def on_trial_complete(self, trial_id, error=False, **kwargs): + if error: + self.errored_trials += [trial_id] + + searchalg = _MockSearchAlg() + scheduler = _MockScheduler() + return searchalg, scheduler + class TrialRunnerTest(unittest.TestCase): def tearDown(self): ray.shutdown() @@ -889,16 +907,6 @@ def train(config, reporter): self.assertLessEqual(len(trial.logdir), 200) trial_executor.stop_trial(trial) - def testTrialErrorOnStart(self): - ray.init() - trial_executor = RayTrialExecutor() - _global_registry.register(TRAINABLE_CLASS, "asdf", None) - trial = Trial("asdf", resources=Resources(1, 0)) - try: - trial_executor.start_trial(trial) - except Exception as e: - self.assertIn("a class", str(e)) - def testExtraResources(self): ray.init(num_cpus=4, num_gpus=2) runner = TrialRunner(BasicVariantGenerator()) @@ -1055,7 +1063,9 @@ def testThrowOnOverstep(self): def testFailureRecoveryDisabled(self): ray.init(num_cpus=1, num_gpus=1) - runner = TrialRunner(BasicVariantGenerator()) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) kwargs = { "resources": Resources(cpu=1, gpu=1), "checkpoint_freq": 1, @@ -1074,10 +1084,15 @@ def testFailureRecoveryDisabled(self): runner.step() self.assertEqual(trials[0].status, Trial.ERROR) self.assertEqual(trials[0].num_failures, 1) + self.assertEqual(len(searchalg.errored_trials), 1) + self.assertEqual(len(scheduler.errored_trials), 1) def testFailureRecoveryEnabled(self): ray.init(num_cpus=1, num_gpus=1) - runner = TrialRunner(BasicVariantGenerator()) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) + kwargs = { "resources": Resources(cpu=1, gpu=1), "checkpoint_freq": 1, @@ -1098,6 +1113,40 @@ def testFailureRecoveryEnabled(self): self.assertEqual(trials[0].num_failures, 1) runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) + self.assertEqual(len(searchalg.errored_trials), 0) + self.assertEqual(len(scheduler.errored_trials), 0) + + def testFailureRecoveryNodeRemoval(self): + ray.init(num_cpus=1, num_gpus=1) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) + + kwargs = { + "resources": Resources(cpu=1, gpu=1), + "checkpoint_freq": 1, + "max_failures": 1, + "config": { + "mock_error": True, + }, + } + runner.add_trial(Trial("__fake", **kwargs)) + trials = runner.get_trials() + + with mock.patch('ray.global_state.cluster_resources') as res_mock: + res_mock.return_value = {"CPU": 1, "GPU": 1} + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + + # Mimic a node failure + res_mock.return_value = {"CPU": 0, "GPU": 0} + runner.step() + self.assertEqual(trials[0].status, Trial.PENDING) + self.assertEqual(trials[0].num_failures, 1) + self.assertEqual(len(searchalg.errored_trials), 0) + self.assertEqual(len(scheduler.errored_trials), 1) def testFailureRecoveryMaxFailures(self): ray.init(num_cpus=1, num_gpus=1) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 65683eeb53c7..83da6e0eff39 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -281,10 +281,12 @@ def has_checkpoint(self): def should_recover(self): """Returns whether the trial qualifies for restoring. - This is if a checkpoint frequency is set, which includes settings - where there may not yet be a checkpoint. + This is if a checkpoint frequency is set and has not failed more than + max_failures. This may return true even when there may not yet + be a checkpoint. """ - return self.checkpoint_freq > 0 + return (self.checkpoint_freq > 0 + and self.num_failures < self.max_failures) def update_last_result(self, result, terminate=False): if terminate: diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index e0b541218bf1..d67e7705a1b1 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -31,15 +31,14 @@ def has_resources(self, resources): raise NotImplementedError("Subclasses of TrialExecutor must provide " "has_resources() method") - def start_trial(self, trial, checkpoint=None): - """Starts the trial restoring from checkpoint if checkpoint != None. - - If an error is encountered when starting the trial, an exception will - be thrown. + def start_trial(self, trial, checkpoint=None, raise_on_failure=False): + """Starts the trial restoring from checkpoint if checkpoint is provided. Args: + trial (Trial): Trial to be started. checkpoint(Checkpoint): A Python object or path storing the state of trial. + raise_on_failure (bool): To raise exception on failure in starting. """ raise NotImplementedError("Subclasses of TrialExecutor must provide " "start_trial() method") @@ -59,26 +58,6 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): raise NotImplementedError("Subclasses of TrialExecutor must provide " "stop_trial() method") - def restart_trial(self, trial, error_msg=None): - """Restarts or requeues the trial. - - The state of the trial should restore from the last checkpoint. Trial - is requeued if the cluster no longer has resources to accomodate it. - - Args: - error_msg (str): Optional error message. - """ - self.stop_trial( - trial, - error=error_msg is not None, - error_msg=error_msg, - stop_logger=False) - trial.result_logger.flush() - if self.has_resources(trial.resources): - self.start_trial(trial) - else: - trial.status = Trial.PENDING - def continue_training(self, trial): """Continues the training of this trial.""" pass diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 98bbbcb71c64..53107cbe86ff 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -11,8 +11,8 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor -from ray.tune.result import TIME_THIS_ITER_S -from ray.tune.trial import Trial +from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR +from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -116,6 +116,11 @@ def step(self): self.trial_executor.start_trial(next_trial) elif self.trial_executor.get_running_trials(): self._process_events() + if self._checkpoint_freq: + if self._iteration % self._checkpoint_freq == 0: + self.save() + + self._iteration += 1 else: for trial in self._trials: if trial.status == Trial.PENDING: @@ -297,24 +302,65 @@ def _process_events(self): logger.exception("Error processing event.") error_msg = traceback.format_exc() if trial.status == Trial.RUNNING: - if trial.should_recover() and \ - trial.num_failures < trial.max_failures: - self._try_recover(trial, error_msg) + if trial.should_recover(): + self.try_recover(trial, error_msg) else: + self.trial_executor.stop_trial( + trial, + error=error_msg is not None, + error_msg=error_msg) self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( trial.trial_id, error=True) - self.trial_executor.stop_trial(trial, True, error_msg) - def _try_recover(self, trial, error_msg): + def _checkpoint_if_needed(self, trial): + """Checkpoints trial based off trial.last_result.""" + if trial.should_checkpoint(): + + # Save trial runtime if possible + if hasattr(trial, "runner") and trial.runner: + self.trial_executor.save(trial, storage=Checkpoint.DISK) + + try: + self._trial_checkpoints[trial] = pickle.dumps(trial) + except ValueError: + logger.exception("Error checkpointing full trial state.") + + def try_recover(self, trial, error_msg): + """Tries to recover trial. + + Notifies SearchAlgorithm and Scheduler if failure to recover. + """ try: - logger.info("Attempting to recover" - " trial state from last checkpoint.") - self.trial_executor.restart_trial(trial, error_msg) + self.trial_executor.stop_trial( + trial, + error=error_msg is not None, + error_msg=error_msg, + stop_logger=False) + trial.result_logger.flush() + if self.trial_executor.has_resources(trial.resources): + logger.info("Attempting to recover" + " trial state from last checkpoint.") + self.trial_executor.start_trial(trial, raise_on_failure=True) + else: + logger.debug("Notifying Scheduler and requeueing trial.") + self._requeue_trial(trial) except Exception: error_msg = traceback.format_exc() - logger.warning("Error recovering trial from checkpoint, abort.") - self.trial_executor.stop_trial(trial, True, error_msg=error_msg) + logger.exception("Error recovering trial from checkpoint, abort.") + self._scheduler_alg.on_trial_error(self, trial) + self._search_alg.on_trial_complete( + trial.trial_id, error=True) + + def _requeue_trial(self, trial): + """Notification to TrialScheduler and requeue trial. + + This does not notify the SearchAlgorithm because + the function evaluation is still in progress. + """ + self._scheduler_alg.on_trial_error(self, trial) + trial.status = Trial.PENDING + self._scheduler_alg.on_trial_add(self, trial) def _update_trial_queue(self, blocking=False, timeout=600): """Adds next trials to queue if possible. diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 335660ecb836..c7aa6e560edc 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -8,6 +8,7 @@ from ray.tune.error import TuneError from ray.tune.suggest import BasicVariantGenerator from ray.tune.trial import Trial, DEBUG_PRINT_INTERVAL +from ray.tune.result import DEFAULT_RESULTS_DIR from ray.tune.log_sync import wait_for_log_sync from ray.tune.trial_runner import TrialRunner from ray.tune.schedulers import (HyperBandScheduler, AsyncHyperBandScheduler, @@ -90,8 +91,6 @@ def run_experiments(experiments=None, if search_alg is None: search_alg = BasicVariantGenerator() - search_alg.add_configurations(experiments) - runner = TrialRunner( search_alg, scheduler=scheduler, From 5d8e414af9fd70edaa8faef89aafd4a7d043a516 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 18:01:19 -0800 Subject: [PATCH 02/40] py2mock --- .travis/install-dependencies.sh | 4 ++-- python/ray/tune/test/trial_runner_test.py | 13 +++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index 293c1b8b6b04..3f1ea4922bc8 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -25,7 +25,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update sudo apt-get install -y cmake pkg-config python-dev python-numpy build-essential autoconf curl libtool unzip @@ -51,7 +51,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed which -s brew diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 6af30e87fd3a..2d40e8eeb90f 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -3,9 +3,9 @@ from __future__ import print_function import os +import sys import time import unittest -from unittest import mock import ray from ray.rllib import _register_all @@ -26,6 +26,11 @@ SuggestionAlgorithm) from ray.tune.suggest.variant_generator import RecursiveDependencyError +if sys.version_info >= (3, 3): + from unittest.mock import patch +else: + from mock import patch + class TrainableFunctionApiTest(unittest.TestCase): def setUp(self): @@ -1133,15 +1138,15 @@ def testFailureRecoveryNodeRemoval(self): runner.add_trial(Trial("__fake", **kwargs)) trials = runner.get_trials() - with mock.patch('ray.global_state.cluster_resources') as res_mock: - res_mock.return_value = {"CPU": 1, "GPU": 1} + with patch('ray.global_state.cluster_resources') as resource_mock: + resource_mock.return_value = {"CPU": 1, "GPU": 1} runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) # Mimic a node failure - res_mock.return_value = {"CPU": 0, "GPU": 0} + resource_mock.return_value = {"CPU": 0, "GPU": 0} runner.step() self.assertEqual(trials[0].status, Trial.PENDING) self.assertEqual(trials[0].num_failures, 1) From 9137de0d7cc58215a283c24fe9ad47e37afec33c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 18:44:27 -0800 Subject: [PATCH 03/40] nit --- python/ray/tune/ray_trial_executor.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index dbee770b44cd..ec360852391d 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -154,13 +154,12 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): self._stop_trial( trial, error=error, error_msg=error_msg, stop_logger=stop_logger) if prior_status == Trial.RUNNING: + logger.debug("Returning resources for this trial.") + self._return_resources(trial.resources) out = self._find_item(self._running, trial) for result_id in out: self._running.pop(result_id) - logger.debug("Returning resources for this trial.") - self._return_resources(trial.resources) - def continue_training(self, trial): """Continues the training of this trial.""" From 445372493b58446cb2018ed23834b803213cc5be Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 17:26:06 -0800 Subject: [PATCH 04/40] Fix counting resources test --- python/ray/tune/test/cluster_tests.py | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index f9425cc3e301..d95c884a1d44 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -86,24 +86,24 @@ def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster - assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] - nodes += [cluster.add_node(resources=dict(CPU=1))] - assert cluster.wait_for_nodes() - assert ray.global_state.cluster_resources()["CPU"] == 2 - + assert ray.global_state.cluster_resources()["CPU"] == 1 runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) runner.step() # run 1 + nodes += [cluster.add_node(resources=dict(CPU=1))] + assert cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 2 cluster.remove_node(nodes.pop()) assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 runner.step() # run 2 + assert sum(t.status == Trial.RUNNING for t in runner.get_trials()) == 1 for i in range(5): nodes += [cluster.add_node(resources=dict(CPU=1))] @@ -111,12 +111,7 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 6 runner.step() # 1 result - - for i in range(5): - node = nodes.pop() - cluster.remove_node(node) - assert cluster.wait_for_nodes() - assert ray.global_state.cluster_resources()["CPU"] == 1 + assert sum(t.status == Trial.RUNNING for t in runner.get_trials()) == 2 @pytest.mark.skip("Add this test once reconstruction is fixed") From 5a2449970880a640386c4fff8329642509754ae5 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 18:54:17 -0800 Subject: [PATCH 05/40] Remove extraneous changes --- python/ray/tune/trial.py | 10 ++++++---- python/ray/tune/trial_runner.py | 17 ++--------------- 2 files changed, 8 insertions(+), 19 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 83da6e0eff39..b6bdbd1d755b 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -216,17 +216,19 @@ def should_stop(self, result): return False - def should_checkpoint(self, result): + def should_checkpoint(self): """Whether this trial is due for checkpointing.""" + result = self.last_result or {} if result.get(DONE) and self.checkpoint_at_end: return True - if not self.checkpoint_freq: + if self.checkpoint_freq: + return result.get( + TRAINING_ITERATION, 0) % self.checkpoint_freq == 0 + else: return False - return self.last_result[TRAINING_ITERATION] % self.checkpoint_freq == 0 - def progress_string(self): """Returns a progress message for printing out to the console.""" diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 53107cbe86ff..d77da2b85b8c 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -116,11 +116,6 @@ def step(self): self.trial_executor.start_trial(next_trial) elif self.trial_executor.get_running_trials(): self._process_events() - if self._checkpoint_freq: - if self._iteration % self._checkpoint_freq == 0: - self.save() - - self._iteration += 1 else: for trial in self._trials: if trial.status == Trial.PENDING: @@ -283,17 +278,14 @@ def _process_events(self): result, terminate=(decision == TrialScheduler.STOP)) if decision == TrialScheduler.CONTINUE: - if trial.should_checkpoint(result): - # TODO(rliaw): This is a blocking call - self.trial_executor.save(trial) + self._checkpoint_if_needed(trial) self.trial_executor.continue_training(trial) elif decision == TrialScheduler.PAUSE: self.trial_executor.pause_trial(trial) elif decision == TrialScheduler.STOP: # Checkpoint before ending the trial # if checkpoint_at_end experiment option is set to True - if trial.should_checkpoint(result): - self.trial_executor.save(trial) + self._checkpoint_if_needed(trial) self.trial_executor.stop_trial(trial) else: assert False, "Invalid scheduling decision: {}".format( @@ -321,11 +313,6 @@ def _checkpoint_if_needed(self, trial): if hasattr(trial, "runner") and trial.runner: self.trial_executor.save(trial, storage=Checkpoint.DISK) - try: - self._trial_checkpoints[trial] = pickle.dumps(trial) - except ValueError: - logger.exception("Error checkpointing full trial state.") - def try_recover(self, trial, error_msg): """Tries to recover trial. From b750d4efdfada19704951756f96433d470d667b6 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:04:11 -0800 Subject: [PATCH 06/40] docs --- python/ray/tune/ray_trial_executor.py | 2 +- python/ray/tune/trial_runner.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index ec360852391d..3e460e80dc0a 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -117,7 +117,7 @@ def start_trial(self, trial, checkpoint=None, raise_on_failure=False): Args: trial (Trial): Trial to be started. - checkpoint(Checkpoint): A Python object or path storing the state + checkpoint (Checkpoint): A Python object or path storing the state of trial. raise_on_failure (bool): To raise exception on failure in starting. diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index d77da2b85b8c..a3aa25bf0024 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -317,6 +317,10 @@ def try_recover(self, trial, error_msg): """Tries to recover trial. Notifies SearchAlgorithm and Scheduler if failure to recover. + + Args: + trial (Trial): Trial to recover. + error_msg (str): Error message from prior to invoking this method. """ try: self.trial_executor.stop_trial( From 14da6ec51f50f9205a70b79bd5c20d478e97be7c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 00:50:01 -0800 Subject: [PATCH 07/40] yapf --- python/ray/tune/trial.py | 4 ++-- python/ray/tune/trial_runner.py | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index b6bdbd1d755b..d3a4e8145103 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -224,8 +224,8 @@ def should_checkpoint(self): return True if self.checkpoint_freq: - return result.get( - TRAINING_ITERATION, 0) % self.checkpoint_freq == 0 + return result.get(TRAINING_ITERATION, + 0) % self.checkpoint_freq == 0 else: return False diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index a3aa25bf0024..20a2bac2d213 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -340,8 +340,7 @@ def try_recover(self, trial, error_msg): error_msg = traceback.format_exc() logger.exception("Error recovering trial from checkpoint, abort.") self._scheduler_alg.on_trial_error(self, trial) - self._search_alg.on_trial_complete( - trial.trial_id, error=True) + self._search_alg.on_trial_complete(trial.trial_id, error=True) def _requeue_trial(self, trial): """Notification to TrialScheduler and requeue trial. From 394c0e941931eb8781b34631f967ef094342c77e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:22:22 -0800 Subject: [PATCH 08/40] Lint and small changes to tests --- python/ray/tune/test/cluster_tests.py | 48 +++++++++++++---------- python/ray/tune/test/trial_runner_test.py | 4 +- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index d95c884a1d44..348de35f7b09 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -10,23 +10,28 @@ except ImportError: pytest_timeout = None -from ray.test.cluster_utils import Cluster import ray from ray import tune +from ray.rllib import _register_all +from ray.test.cluster_utils import Cluster from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner from ray.tune.suggest import BasicVariantGenerator -def register_test_trainable(): - class _Train(tune.Trainable): +def register_fail_trainable(): + class _Fail(tune.Trainable): + """Fails on the 4th iteration.""" + def _setup(self, config): - self.state = {"hi": 1} + self.state = {"hi": 0} def _train(self): self.state["hi"] += 1 time.sleep(0.5) + if self.state["hi"] >= 4: + assert False return {} def _save(self, path): @@ -35,13 +40,10 @@ def _save(self, path): def _restore(self, state): self.state = state - tune.register_trainable("test", _Train) + tune.register_trainable("test", _Fail) -@pytest.fixture -def start_connected_cluster(): - # Start the Ray processes. - +def _start_new_cluster(): cluster = Cluster( initialize_head=True, connect=True, @@ -51,7 +53,15 @@ def start_connected_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() + # Pytest doesn't play nicely with imports + _register_all() + return cluster + + +@pytest.fixture +def start_connected_cluster(): + # Start the Ray processes. + cluster = _start_new_cluster() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -71,17 +81,14 @@ def start_connected_emptyhead_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() + # Pytest doesn't play nicely with imports + _register_all() yield cluster # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() -@pytest.mark.skipif( - pytest_timeout is None, - reason="Timeout package not installed; skipping test.") -@pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" @@ -128,7 +135,7 @@ def test_remove_node_before_result(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 3}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) @@ -174,7 +181,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): } # Test recovery of trial that hasn't been checkpointed - t = Trial("test", **kwargs) + t = Trial("__fake", **kwargs) runner.add_trial(t) runner.step() # start runner.step() # 1 result @@ -194,7 +201,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t.status == Trial.TERMINATED # Test recovery of trial that has been checkpointed - t2 = Trial("test", **kwargs) + t2 = Trial("__fake", **kwargs) runner.add_trial(t2) runner.step() # start runner.step() # 1 result @@ -211,7 +218,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t2.status == Trial.TERMINATED # Test recovery of trial that won't be checkpointed - t3 = Trial("test", **{"stopping_criterion": {"training_iteration": 3}}) + t3 = Trial("__fake", **{"stopping_criterion": {"training_iteration": 3}}) runner.add_trial(t3) runner.step() # start runner.step() # 1 result @@ -233,6 +240,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): """Removing a node in full cluster causes Trial to be requeued.""" cluster = start_connected_emptyhead_cluster node = cluster.add_node(resources=dict(CPU=1)) + assert cluster.wait_for_nodes() runner = TrialRunner(BasicVariantGenerator()) kwargs = { @@ -243,7 +251,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): "max_failures": 1 } - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 2d40e8eeb90f..8e4aa2cea148 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -852,14 +852,15 @@ def testMaxConcurrentSuggestions(self): def create_mock_components(): - class _MockScheduler(FIFOScheduler): errored_trials = [] + def on_trial_error(self, trial_runner, trial): self.errored_trials += [trial] class _MockSearchAlg(BasicVariantGenerator): errored_trials = [] + def on_trial_complete(self, trial_id, error=False, **kwargs): if error: self.errored_trials += [trial_id] @@ -868,6 +869,7 @@ def on_trial_complete(self, trial_id, error=False, **kwargs): scheduler = _MockScheduler() return searchalg, scheduler + class TrialRunnerTest(unittest.TestCase): def tearDown(self): ray.shutdown() From 48fd3c3fcbf0e6fbd1fd559852095d12ab157a54 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:23:45 -0800 Subject: [PATCH 09/40] lint --- python/ray/tune/trial_runner.py | 2 +- python/ray/tune/tune.py | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 20a2bac2d213..d36bc1dec907 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -11,7 +11,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor -from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR +from ray.tune.result import TIME_THIS_ITER_S from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index c7aa6e560edc..7840fa4fbdc2 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -8,7 +8,6 @@ from ray.tune.error import TuneError from ray.tune.suggest import BasicVariantGenerator from ray.tune.trial import Trial, DEBUG_PRINT_INTERVAL -from ray.tune.result import DEFAULT_RESULTS_DIR from ray.tune.log_sync import wait_for_log_sync from ray.tune.trial_runner import TrialRunner from ray.tune.schedulers import (HyperBandScheduler, AsyncHyperBandScheduler, From bcf40513722b34a60206f00513a1bb074fbf6896 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:27:23 -0800 Subject: [PATCH 10/40] nit --- python/ray/tune/tune.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 7840fa4fbdc2..335660ecb836 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -90,6 +90,8 @@ def run_experiments(experiments=None, if search_alg is None: search_alg = BasicVariantGenerator() + search_alg.add_configurations(experiments) + runner = TrialRunner( search_alg, scheduler=scheduler, From 0f67265a9fd6c1c39525a5c092f5eb29f946db1d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 20:52:16 -0800 Subject: [PATCH 11/40] small extraneous removals --- python/ray/tune/test/cluster_tests.py | 25 ------------------------- python/ray/tune/trial_runner.py | 7 ++----- 2 files changed, 2 insertions(+), 30 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 348de35f7b09..59f12181b8ff 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -3,7 +3,6 @@ from __future__ import print_function import json -import time import pytest try: import pytest_timeout @@ -11,7 +10,6 @@ pytest_timeout = None import ray -from ray import tune from ray.rllib import _register_all from ray.test.cluster_utils import Cluster from ray.tune.error import TuneError @@ -20,29 +18,6 @@ from ray.tune.suggest import BasicVariantGenerator -def register_fail_trainable(): - class _Fail(tune.Trainable): - """Fails on the 4th iteration.""" - - def _setup(self, config): - self.state = {"hi": 0} - - def _train(self): - self.state["hi"] += 1 - time.sleep(0.5) - if self.state["hi"] >= 4: - assert False - return {} - - def _save(self, path): - return self.state - - def _restore(self, state): - self.state = state - - tune.register_trainable("test", _Fail) - - def _start_new_cluster(): cluster = Cluster( initialize_head=True, diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index d36bc1dec907..b3ac14b98bc1 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -297,18 +297,15 @@ def _process_events(self): if trial.should_recover(): self.try_recover(trial, error_msg) else: - self.trial_executor.stop_trial( - trial, - error=error_msg is not None, - error_msg=error_msg) self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( trial.trial_id, error=True) + self.trial_executor.stop_trial( + trial, error=True, error_msg=error_msg) def _checkpoint_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" if trial.should_checkpoint(): - # Save trial runtime if possible if hasattr(trial, "runner") and trial.runner: self.trial_executor.save(trial, storage=Checkpoint.DISK) From 74b6a939d47432d36306642218f3e14142860822 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 20:58:55 -0800 Subject: [PATCH 12/40] fix some merge? --- python/ray/tune/trial_runner.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 548c68c51cc0..f64092cf2cf3 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -360,13 +360,11 @@ def _process_events(self): if trial.should_recover(): self.try_recover(trial, error_msg) else: - self.trial_executor.stop_trial( - trial, - error=error_msg is not None, - error_msg=error_msg) self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( trial.trial_id, error=True) + self.trial_executor.stop_trial( + trial, error=True, error_msg=error_msg) def _checkpoint_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" From e3c088fa1ebf8baa4b3b08763ac4234863e9b7f8 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 27 Nov 2018 01:14:10 -0800 Subject: [PATCH 13/40] [rllib] PPO doesn't work with fractional num gpus (#3396) * frac ppo * gpu test --- python/ray/rllib/optimizers/multi_gpu_optimizer.py | 5 ++++- test/jenkins_tests/run_multi_node_tests.sh | 8 ++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/python/ray/rllib/optimizers/multi_gpu_optimizer.py b/python/ray/rllib/optimizers/multi_gpu_optimizer.py index 7e01ee9041dc..771acb5ac72c 100644 --- a/python/ray/rllib/optimizers/multi_gpu_optimizer.py +++ b/python/ray/rllib/optimizers/multi_gpu_optimizer.py @@ -3,6 +3,7 @@ from __future__ import print_function import logging +import math import numpy as np from collections import defaultdict import tensorflow as tf @@ -44,7 +45,9 @@ def _init(self, if not num_gpus: self.devices = ["/cpu:0"] else: - self.devices = ["/gpu:{}".format(i) for i in range(num_gpus)] + self.devices = [ + "/gpu:{}".format(i) for i in range(int(math.ceil(num_gpus))) + ] self.batch_size = int(sgd_batch_size / len(self.devices)) * len( self.devices) assert self.batch_size % len(self.devices) == 0 diff --git a/test/jenkins_tests/run_multi_node_tests.sh b/test/jenkins_tests/run_multi_node_tests.sh index 93f00a0eed72..40e9635d9d8a 100755 --- a/test/jenkins_tests/run_multi_node_tests.sh +++ b/test/jenkins_tests/run_multi_node_tests.sh @@ -53,6 +53,14 @@ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ --stop '{"training_iteration": 2}' \ --config '{"simple_optimizer": true, "num_sgd_iter": 2, "model": {"use_lstm": true}}' +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/train.py \ + --env CartPole-v1 \ + --run PPO \ + --stop '{"training_iteration": 2}' \ + --config '{"num_gpus": 0.1}' \ + --ray-num-gpus 1 + docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/rllib/train.py \ --env CartPole-v1 \ From 20b8b1d891e0b27412fdf80a628cb48cfca7f273 Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Tue, 27 Nov 2018 04:28:02 -0800 Subject: [PATCH 14/40] Add script for running stress tests. (#3378) * Add script for running stress tests. * Add an actor tree test where actors die with some probability * Improve test. * Small fix * Update tests. * Minor change --- test/stress_tests/run_stress_tests.sh | 19 +++ test/stress_tests/stress_testing_config.yaml | 115 ++++++++++++++++++ test/stress_tests/test_dead_actors.py | 72 +++++++++++ .../test_many_tasks_and_transfers.py | 84 +++++++++++++ 4 files changed, 290 insertions(+) create mode 100755 test/stress_tests/run_stress_tests.sh create mode 100644 test/stress_tests/stress_testing_config.yaml create mode 100644 test/stress_tests/test_dead_actors.py create mode 100644 test/stress_tests/test_many_tasks_and_transfers.py diff --git a/test/stress_tests/run_stress_tests.sh b/test/stress_tests/run_stress_tests.sh new file mode 100755 index 000000000000..ba0886037c9e --- /dev/null +++ b/test/stress_tests/run_stress_tests.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +# Cause the script to exit if a single command fails. +set -e + +# Show explicitly which commands are currently running. +set -x + +ROOT_DIR=$(cd "$(dirname "${BASH_SOURCE:-$0}")"; pwd) + +# Start a large cluster using the autoscaler. +ray up -y $ROOT_DIR/stress_testing_config.yaml + +# Run a bunch of stress tests. +ray submit $ROOT_DIR/stress_testing_config.yaml test_many_tasks_and_transfers.py +ray submit $ROOT_DIR/stress_testing_config.yaml test_dead_actors.py + +# Tear down the cluster. +ray down -y $ROOT_DIR/stress_testing_config.yaml diff --git a/test/stress_tests/stress_testing_config.yaml b/test/stress_tests/stress_testing_config.yaml new file mode 100644 index 000000000000..7126366b53fd --- /dev/null +++ b/test/stress_tests/stress_testing_config.yaml @@ -0,0 +1,115 @@ +# An unique identifier for the head node and workers of this cluster. +cluster_name: stress-testing + +# The minimum number of workers nodes to launch in addition to the head +# node. This number should be >= 0. +min_workers: 100 + +# The maximum number of workers nodes to launch in addition to the head +# node. This takes precedence over min_workers. +max_workers: 100 + +# The autoscaler will scale up the cluster to this target fraction of resource +# usage. For example, if a cluster of 10 nodes is 100% busy and +# target_utilization is 0.8, it would resize the cluster to 13. This fraction +# can be decreased to increase the aggressiveness of upscaling. +# This value must be less than 1.0 for scaling to happen. +target_utilization_fraction: 0.8 + +# If a node is idle for this many minutes, it will be removed. +idle_timeout_minutes: 5 + +# Cloud-provider specific configuration. +provider: + type: aws + region: us-west-2 + availability_zone: us-west-2a + +# How Ray will authenticate with newly launched nodes. +auth: + ssh_user: ubuntu +# By default Ray creates a new private keypair, but you can also use your own. +# If you do so, make sure to also set "KeyName" in the head and worker node +# configurations below. +# ssh_private_key: /path/to/your/key.pem + +# Provider-specific config for the head node, e.g. instance type. By default +# Ray will auto-configure unspecified fields such as SubnetId and KeyName. +# For more documentation on available fields, see: +# http://boto3.readthedocs.io/en/latest/reference/services/ec2.html#EC2.ServiceResource.create_instances +head_node: + InstanceType: m5.12xlarge + ImageId: ami-0def3275 # Default Ubuntu 16.04 AMI. + + # Set primary volume to 25 GiB + BlockDeviceMappings: + - DeviceName: /dev/sda1 + Ebs: + VolumeSize: 50 + + # Additional options in the boto docs. + +# Provider-specific config for worker nodes, e.g. instance type. By default +# Ray will auto-configure unspecified fields such as SubnetId and KeyName. +# For more documentation on available fields, see: +# http://boto3.readthedocs.io/en/latest/reference/services/ec2.html#EC2.ServiceResource.create_instances +worker_nodes: + InstanceType: m5.large + ImageId: ami-0def3275 # Default Ubuntu 16.04 AMI. + + # Set primary volume to 25 GiB + BlockDeviceMappings: + - DeviceName: /dev/sda1 + Ebs: + VolumeSize: 50 + + # Run workers on spot by default. Comment this out to use on-demand. + InstanceMarketOptions: + MarketType: spot + # Additional options can be found in the boto docs, e.g. + # SpotOptions: + # MaxPrice: MAX_HOURLY_PRICE + + # Additional options in the boto docs. + +# Files or directories to copy to the head and worker nodes. The format is a +# dictionary from REMOTE_PATH: LOCAL_PATH, e.g. +file_mounts: { +# "/path1/on/remote/machine": "/path1/on/local/machine", +# "/path2/on/remote/machine": "/path2/on/local/machine", +} + +# List of shell commands to run to set up nodes. +setup_commands: + # Consider uncommenting these if you run into dpkg locking issues + # - sudo pkill -9 apt-get || true + # - sudo pkill -9 dpkg || true + # - sudo dpkg --configure -a + # Install basics. + - sudo apt-get update + - sudo apt-get install -y cmake pkg-config build-essential autoconf curl libtool unzip flex bison python + # Install Anaconda. + - wget https://repo.continuum.io/archive/Anaconda3-5.0.1-Linux-x86_64.sh || true + - bash Anaconda3-5.0.1-Linux-x86_64.sh -b -p $HOME/anaconda3 || true + - echo 'export PATH="$HOME/anaconda3/bin:$PATH"' >> ~/.bashrc + # # Build Ray. + # - git clone https://github.com/ray-project/ray || true + - pip install boto3==1.4.8 cython==0.27.3 + # - cd ray/python; git checkout master; git pull; pip install -e . --verbose + - pip install https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp36-cp36m-manylinux1_x86_64.whl + +# Custom commands that will be run on the head node after common setup. +head_setup_commands: [] + +# Custom commands that will be run on worker nodes after common setup. +worker_setup_commands: [] + +# Command to start ray on the head node. You don't need to change this. +head_start_ray_commands: + - ray stop + - ulimit -n 65536; ray start --head --num-redis-shards=5 --redis-port=6379 --autoscaling-config=~/ray_bootstrap_config.yaml + +# Command to start ray on worker nodes. You don't need to change this. +worker_start_ray_commands: + - ray stop + - ulimit -n 65536; ray start --redis-address=$RAY_HEAD_IP:6379 --num-gpus=100 diff --git a/test/stress_tests/test_dead_actors.py b/test/stress_tests/test_dead_actors.py new file mode 100644 index 000000000000..72b801142635 --- /dev/null +++ b/test/stress_tests/test_dead_actors.py @@ -0,0 +1,72 @@ +#!/usr/bin/env python + +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import logging +import numpy as np +import sys + +import ray + +logger = logging.getLogger(__name__) + +ray.init(redis_address="localhost:6379") + + +@ray.remote +class Child(object): + def __init__(self, death_probability): + self.death_probability = death_probability + + def ping(self): + # Exit process with some probability. + exit_chance = np.random.rand() + if exit_chance > self.death_probability: + sys.exit(-1) + + +@ray.remote +class Parent(object): + def __init__(self, num_children, death_probability): + self.death_probability = death_probability + self.children = [ + Child.remote(death_probability) for _ in range(num_children) + ] + + def ping(self, num_pings): + children_outputs = [] + for _ in range(num_pings): + children_outputs += [ + child.ping.remote() for child in self.children + ] + try: + ray.get(children_outputs) + except Exception: + # Replace the children if one of them died. + self.__init__(len(self.children), self.death_probability) + + def kill(self): + # Clean up children. + ray.get([child.__ray_terminate__.remote() for child in self.children]) + + +num_parents = 10 +num_children = 10 +death_probability = 0.95 + +parents = [ + Parent.remote(num_children, death_probability) for _ in range(num_parents) +] +for i in range(100): + ray.get([parent.ping.remote(10) for parent in parents]) + + # Kill a parent actor with some probability. + exit_chance = np.random.rand() + if exit_chance > death_probability: + parent_index = np.random.randint(len(parents)) + parents[parent_index].kill.remote() + parents[parent_index] = Parent.remote(num_children, death_probability) + + logger.info("Finished trial", i) diff --git a/test/stress_tests/test_many_tasks_and_transfers.py b/test/stress_tests/test_many_tasks_and_transfers.py new file mode 100644 index 000000000000..87b8239a08fe --- /dev/null +++ b/test/stress_tests/test_many_tasks_and_transfers.py @@ -0,0 +1,84 @@ +#!/usr/bin/env python + +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import numpy as np +import logging +import time + +import ray + +logger = logging.getLogger(__name__) + +ray.init(redis_address="localhost:6379") + +# These numbers need to match the values in the autoscaler config file. +num_remote_nodes = 100 +head_node_cpus = 2 +num_remote_cpus = num_remote_nodes * head_node_cpus + +# Wait until the expected number of nodes have joined the cluster. +while True: + if len(ray.global_state.client_table()) >= num_remote_nodes + 1: + break +logger.info("Nodes have all joined. There are {} resources." + .format(ray.global_state.cluster_resources())) + + +# Require 1 GPU to force the tasks to be on remote machines. +@ray.remote(num_gpus=1) +def f(size, *xs): + return np.ones(size, dtype=np.uint8) + + +# Require 1 GPU to force the actors to be on remote machines. +@ray.remote(num_cpus=1, num_gpus=1) +class Actor(object): + def method(self, size, *xs): + return np.ones(size, dtype=np.uint8) + + +# Launch a bunch of tasks. +start_time = time.time() +logger.info("Submitting many tasks.") +for i in range(10): + logger.info("Iteration {}".format(i)) + ray.get([f.remote(0) for _ in range(100000)]) +logger.info("Finished after {} seconds.".format(time.time() - start_time)) + +# Launch a bunch of tasks, each with a bunch of dependencies. +start_time = time.time() +logger.info("Submitting tasks with many dependencies.") +x_ids = [] +for i in range(5): + logger.info("Iteration {}".format(i)) + x_ids = [f.remote(0, *x_ids) for _ in range(10000)] +ray.get(x_ids) +logger.info("Finished after {} seconds.".format(time.time() - start_time)) + +# Create a bunch of actors. +start_time = time.time() +logger.info("Creating {} actors.".format(num_remote_cpus)) +actors = [Actor.remote() for _ in range(num_remote_cpus)] +logger.info("Finished after {} seconds.".format(time.time() - start_time)) + +# Submit a bunch of small tasks to each actor. +start_time = time.time() +logger.info("Submitting many small actor tasks.") +x_ids = [] +for _ in range(100000): + x_ids = [a.method.remote(0) for a in actors] +ray.get(x_ids) +logger.info("Finished after {} seconds.".format(time.time() - start_time)) + +# Submit a bunch of actor tasks with all-to-all communication. +start_time = time.time() +logger.info("Submitting actor tasks with all-to-all communication.") +x_ids = [] +for _ in range(50): + for size_exponent in [0, 1, 2, 3, 4, 5, 6]: + x_ids = [a.method.remote(10**size_exponent, *x_ids) for a in actors] +ray.get(x_ids) +logger.info("Finished after {} seconds.".format(time.time() - start_time)) From 0d56fc10ccf0802ae7b335532698b4b1fc573fe4 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 27 Nov 2018 09:50:59 -0800 Subject: [PATCH 15/40] Move setproctitle to ray[debug] package (#3415) --- doc/source/installation.rst | 2 +- doc/source/rllib.rst | 2 +- doc/source/tune.rst | 2 +- python/ray/memory_monitor.py | 5 +++-- python/ray/scripts/scripts.py | 2 +- python/ray/tune/trial_runner.py | 3 ++- python/ray/worker.py | 19 +++++++++++++++---- python/setup.py | 6 ++++-- 8 files changed, 28 insertions(+), 13 deletions(-) diff --git a/doc/source/installation.rst b/doc/source/installation.rst index 20d49edaea7d..f3ad649d683d 100644 --- a/doc/source/installation.rst +++ b/doc/source/installation.rst @@ -10,7 +10,7 @@ You can install the latest stable version of Ray as follows. .. code-block:: bash - pip install -U ray + pip install -U ray # also recommended: ray[debug] Trying snapshots from master ---------------------------- diff --git a/doc/source/rllib.rst b/doc/source/rllib.rst index 23c69506e008..2de444b52965 100644 --- a/doc/source/rllib.rst +++ b/doc/source/rllib.rst @@ -15,7 +15,7 @@ RLlib has extra dependencies on top of ``ray``. First, you'll need to install ei .. code-block:: bash pip install tensorflow # or tensorflow-gpu - pip install ray[rllib] + pip install ray[rllib] # also recommended: ray[debug] You might also want to clone the `Ray repo `__ for convenient access to RLlib helper scripts: diff --git a/doc/source/tune.rst b/doc/source/tune.rst index 87f28531bee8..14c95fb0edcb 100644 --- a/doc/source/tune.rst +++ b/doc/source/tune.rst @@ -45,7 +45,7 @@ You'll need to first `install ray `__ to import Tune. .. code-block:: bash - pip install ray + pip install ray # also recommended: ray[debug] Quick Start diff --git a/python/ray/memory_monitor.py b/python/ray/memory_monitor.py index 23d6c12b5f9c..00cf86816dbf 100644 --- a/python/ray/memory_monitor.py +++ b/python/ray/memory_monitor.py @@ -56,8 +56,9 @@ def __init__(self, error_threshold=0.95, check_interval=1): if not psutil: logger.warning( "WARNING: Not monitoring node memory since `psutil` is not " - "installed. Install this with `pip install psutil` to enable " - "debugging of memory-related crashes.") + "installed. Install this with `pip install psutil` " + "(or ray[debug]) to enable debugging of memory-related " + "crashes.") def raise_if_low_memory(self): if not psutil: diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index d78f2f9c1dd1..929ee3785906 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -651,7 +651,7 @@ def stack(): COMMAND = """ pyspy=`which py-spy` if [ ! -e "$pyspy" ]; then - echo "ERROR: Please 'pip install py-spy' first" + echo "ERROR: Please 'pip install py-spy' (or ray[debug]) first" exit 1 fi # Set IFS to iterate over lines instead of over words. diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 98bbbcb71c64..d89b3dda7ee1 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -236,7 +236,8 @@ def _memory_debug_string(self): return "Memory usage on this node: {}/{} GB{}".format( round(used_gb, 1), round(total_gb, 1), warn) except ImportError: - return "Unknown memory usage (`pip install psutil` to resolve)" + return ("Unknown memory usage. Please run `pip install psutil` " + "(or ray[debug]) to resolve)") def has_resources(self, resources): """Returns whether this runner has at least the specified resources.""" diff --git a/python/ray/worker.py b/python/ray/worker.py index f68bb42886f0..c3c01f4859fc 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -12,7 +12,6 @@ import numpy as np import os import redis -import setproctitle import signal import sys import threading @@ -73,6 +72,15 @@ # using logging.basicConfig in its entry/init points. logger = logging.getLogger(__name__) +try: + import setproctitle +except ImportError: + setproctitle = None + logger.warning( + "WARNING: Not updating worker name since `setproctitle` is not " + "installed. Install this with `pip install setproctitle` " + "(or ray[debug]) to enable monitoring of worker processes.") + class RayTaskError(Exception): """An object used internally to represent a task that threw an exception. @@ -1916,7 +1924,8 @@ def connect(info, # Initialize some fields. if mode is WORKER_MODE: worker.worker_id = random_string() - setproctitle.setproctitle("ray_worker") + if setproctitle: + setproctitle.setproctitle("ray_worker") else: # This is the code path of driver mode. if driver_id is None: @@ -2163,9 +2172,11 @@ def disconnect(worker=global_worker): @contextmanager def _changeproctitle(title, next_title): - setproctitle.setproctitle(title) + if setproctitle: + setproctitle.setproctitle(title) yield - setproctitle.setproctitle(next_title) + if setproctitle: + setproctitle.setproctitle(next_title) def _try_to_compute_deterministic_class_id(cls, depth=5): diff --git a/python/setup.py b/python/setup.py index 7198de2329bf..c92ffa65b481 100644 --- a/python/setup.py +++ b/python/setup.py @@ -64,7 +64,10 @@ optional_ray_files += ray_autoscaler_files -extras = {"rllib": ["pyyaml", "gym[atari]", "opencv-python", "lz4", "scipy"]} +extras = { + "rllib": ["pyyaml", "gym[atari]", "opencv-python", "lz4", "scipy"], + "debug": ["psutil", "setproctitle", "py-spy"], +} class build_ext(_build_ext.build_ext): @@ -139,7 +142,6 @@ def find_version(*filepath): "pytest", "pyyaml", "redis", - "setproctitle", # The six module is required by pyarrow. "six >= 1.0.0", "flatbuffers", From c2108ca64fd3d9f3eefb4450d181595b651eb16c Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 27 Nov 2018 16:48:12 -0800 Subject: [PATCH 16/40] Don't put entire actor registry in debug string since it's too long (#3395) --- src/ray/raylet/actor_registration.cc | 12 +----------- src/ray/raylet/actor_registration.h | 8 +++----- src/ray/raylet/node_manager.cc | 21 +++++++++++++++++---- 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/ray/raylet/actor_registration.cc b/src/ray/raylet/actor_registration.cc index 3f4a67c1d21c..7ea95e656642 100644 --- a/src/ray/raylet/actor_registration.cc +++ b/src/ray/raylet/actor_registration.cc @@ -10,7 +10,6 @@ namespace raylet { ActorRegistration::ActorRegistration(const ActorTableDataT &actor_table_data) : actor_table_data_(actor_table_data), - alive_(true), execution_dependency_(ObjectID::nil()), frontier_() {} @@ -44,16 +43,7 @@ bool ActorRegistration::IsAlive() const { return actor_table_data_.state == ActorState::ALIVE; } -std::string ActorRegistration::DebugString() const { - std::stringstream result; - if (alive_) { - result << "alive"; - } else { - result << "dead"; - } - result << ", num handles: " << frontier_.size(); - return result.str(); -} +int ActorRegistration::NumHandles() const { return frontier_.size(); } } // namespace raylet diff --git a/src/ray/raylet/actor_registration.h b/src/ray/raylet/actor_registration.h index faa05eb2686c..4cf9b110afe1 100644 --- a/src/ray/raylet/actor_registration.h +++ b/src/ray/raylet/actor_registration.h @@ -94,17 +94,15 @@ class ActorRegistration { /// \return True if the local actor is alive and false if it is dead. bool IsAlive() const; - /// Returns debug string for class. + /// Returns num handles to this actor entry. /// - /// \return string. - std::string DebugString() const; + /// \return int. + int NumHandles() const; private: /// Information from the global actor table about this actor, including the /// node manager location. ActorTableDataT actor_table_data_; - /// True if the actor is alive and false otherwise. - bool alive_; /// The object representing the state following the actor's most recently /// executed task. The next task to execute on the actor should be marked as /// execution-dependent on this object. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index a0b7526e8713..457fa4bba834 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1764,14 +1764,27 @@ std::string NodeManager::DebugString() const { result << "\n" << reconstruction_policy_.DebugString(); result << "\n" << task_dependency_manager_.DebugString(); result << "\n" << lineage_cache_.DebugString(); + result << "\nActorRegistry:"; + int live_actors = 0; + int dead_actors = 0; + int max_num_handles = 0; + for (auto &pair : actor_registry_) { + if (pair.second.IsAlive()) { + live_actors += 1; + } else { + dead_actors += 1; + } + if (pair.second.NumHandles() > max_num_handles) { + max_num_handles = pair.second.NumHandles(); + } + } + result << "\n- num live actors: " << live_actors; + result << "\n- num dead actors: " << dead_actors; + result << "\n- max num handles: " << max_num_handles; result << "\nRemoteConnections:"; for (auto &pair : remote_server_connections_) { result << "\n" << pair.first.hex() << ": " << pair.second->DebugString(); } - result << "\nActorRegistry:"; - for (auto &pair : actor_registry_) { - result << "\n" << pair.first.hex() << ": " << pair.second.DebugString(); - } result << "\nDebugString() time ms: " << (current_time_ms() - now_ms); return result.str(); } From f0df97db6f29d2e4e16135fde9156b7e756f6a63 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 27 Nov 2018 23:35:19 -0800 Subject: [PATCH 17/40] [rllib] example and docs on how to use parametric actions with DQN / PG algorithms (#3384) --- doc/source/rllib-algorithms.rst | 6 +- doc/source/rllib-env.rst | 30 +-- doc/source/rllib-models.rst | 84 ++++++++ doc/source/rllib-training.rst | 4 +- doc/source/rllib.rst | 3 +- .../ray/rllib/agents/dqn/dqn_policy_graph.py | 57 +++-- python/ray/rllib/agents/ppo/ppo.py | 5 + .../examples/parametric_action_cartpole.py | 196 ++++++++++++++++++ python/ray/rllib/models/catalog.py | 2 +- python/ray/rllib/models/preprocessors.py | 3 + .../rllib/tuned_examples/atari-dist-dqn.yaml | 2 +- .../ray/rllib/tuned_examples/atari-dqn.yaml | 4 +- .../rllib/tuned_examples/atari-duel-ddqn.yaml | 4 +- .../tuned_examples/pong-impala-fast.yaml | 2 +- test/jenkins_tests/run_multi_node_tests.sh | 9 + 15 files changed, 366 insertions(+), 45 deletions(-) create mode 100644 python/ray/rllib/examples/parametric_action_cartpole.py diff --git a/doc/source/rllib-algorithms.rst b/doc/source/rllib-algorithms.rst index 66bf08a6c399..1d0501215745 100644 --- a/doc/source/rllib-algorithms.rst +++ b/doc/source/rllib-algorithms.rst @@ -133,10 +133,10 @@ Tuned examples: `Pendulum-v0 `__ `[implementation] `__ -RLlib DQN is implemented using the SyncReplayOptimizer. The algorithm can be scaled by increasing the number of workers, using the AsyncGradientsOptimizer for async DQN, or using Ape-X. Memory usage is reduced by compressing samples in the replay buffer with LZ4. All of the DQN improvements evaluated in `Rainbow `__ are available, though not all are enabled by default. +RLlib DQN is implemented using the SyncReplayOptimizer. The algorithm can be scaled by increasing the number of workers, using the AsyncGradientsOptimizer for async DQN, or using Ape-X. Memory usage is reduced by compressing samples in the replay buffer with LZ4. All of the DQN improvements evaluated in `Rainbow `__ are available, though not all are enabled by default. See also how to use `parametric-actions in DQN `__. Tuned examples: `PongDeterministic-v4 `__, `Rainbow configuration `__, `{BeamRider,Breakout,Qbert,SpaceInvaders}NoFrameskip-v4 `__, `with Dueling and Double-Q `__, `with Distributional DQN `__. diff --git a/doc/source/rllib-env.rst b/doc/source/rllib-env.rst index c1381f561cd4..ca36186e1a5f 100644 --- a/doc/source/rllib-env.rst +++ b/doc/source/rllib-env.rst @@ -7,20 +7,22 @@ RLlib works with several different types of environments, including `OpenAI Gym **Compatibility matrix**: -============= ================ ================== =========== ================== -Algorithm Discrete Actions Continuous Actions Multi-Agent Recurrent Policies -============= ================ ================== =========== ================== -A2C, A3C **Yes** **Yes** **Yes** **Yes** -PPO **Yes** **Yes** **Yes** **Yes** -PG **Yes** **Yes** **Yes** **Yes** -IMPALA **Yes** No **Yes** **Yes** -DQN, Rainbow **Yes** No **Yes** No -DDPG, TD3 No **Yes** **Yes** No -APEX-DQN **Yes** No **Yes** No -APEX-DDPG No **Yes** **Yes** No -ES **Yes** **Yes** No No -ARS **Yes** **Yes** No No -============= ================ ================== =========== ================== +============= ======================= ================== =========== ================== +Algorithm Discrete Actions Continuous Actions Multi-Agent Recurrent Policies +============= ======================= ================== =========== ================== +A2C, A3C **Yes** `+parametric`_ **Yes** **Yes** **Yes** +PPO **Yes** `+parametric`_ **Yes** **Yes** **Yes** +PG **Yes** `+parametric`_ **Yes** **Yes** **Yes** +IMPALA **Yes** `+parametric`_ No **Yes** **Yes** +DQN, Rainbow **Yes** `+parametric`_ No **Yes** No +DDPG, TD3 No **Yes** **Yes** No +APEX-DQN **Yes** `+parametric`_ No **Yes** No +APEX-DDPG No **Yes** **Yes** No +ES **Yes** **Yes** No No +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 `__, however you can also register custom environments by name: diff --git a/doc/source/rllib-models.rst b/doc/source/rllib-models.rst index 5fde37f53087..883b0a6bb526 100644 --- a/doc/source/rllib-models.rst +++ b/doc/source/rllib-models.rst @@ -110,6 +110,43 @@ Custom models should subclass the common RLlib `model class `__ and associated `training scripts `__. You can also reference the `unit tests `__ for Tuple and Dict spaces, which show how to access nested observation fields. +Custom Recurrent Models +~~~~~~~~~~~~~~~~~~~~~~~ + +Instead of using the ``use_lstm: True`` option, it can be preferable use a custom recurrent model. This provides more control over postprocessing of the LSTM output and can also allow the use of multiple LSTM cells to process different portions of the input. The only difference from a normal custom model is that you have to define ``self.state_init``, ``self.state_in``, and ``self.state_out``. You can refer to the existing `lstm.py `__ model as an example to implement your own model: + + +.. code-block:: python + + class MyCustomLSTM(Model): + def _build_layers_v2(self, input_dict, num_outputs, options): + # Some initial layers to process inputs, shape [BATCH, OBS...]. + features = some_hidden_layers(input_dict["obs"]) + + # Add back the nested time dimension for tf.dynamic_rnn, new shape + # will be [BATCH, MAX_SEQ_LEN, OBS...]. + last_layer = add_time_dimension(features, self.seq_lens) + + # Setup the LSTM cell (see lstm.py for an example) + lstm = rnn.BasicLSTMCell(256, state_is_tuple=True) + self.state_init = ... + self.state_in = ... + lstm_out, lstm_state = tf.nn.dynamic_rnn( + lstm, + last_layer, + initial_state=..., + sequence_length=self.seq_lens, + time_major=False, + dtype=tf.float32) + self.state_out = list(lstm_state) + + # Drop the time dimension again so back to shape [BATCH, OBS...]. + # Note that we retain the zero padding (see issue #2992). + last_layer = tf.reshape(lstm_out, [-1, cell_size]) + logits = linear(last_layer, num_outputs, "action", + normc_initializer(0.01)) + return logits, last_layer + Custom Preprocessors -------------------- @@ -188,6 +225,53 @@ Then, you can create an agent with your custom policy graph by: In this example we overrode existing methods of the existing DDPG policy graph, i.e., `_build_q_network`, `_build_p_network`, `_build_action_network`, `_build_actor_critic_loss`, but you can also replace the entire graph class entirely. +Variable-length / Parametric Action Spaces +------------------------------------------ + +Custom models can be used to work with environments where (1) the set of valid actions varies per step, and/or (2) the number of valid actions is very large, as in `OpenAI Five `__ and `Horizon `__. The general idea is that the meaning of actions can be completely conditioned on the observation, that is, the ``a`` in ``Q(s, a)`` is just a token in ``[0, MAX_AVAIL_ACTIONS)`` that only has meaning in the context of ``s``. This works with algorithms in the `DQN and policy-gradient families `__ and can be implemented as follows: + +1. The environment should return a mask and/or list of valid action embeddings as part of the observation for each step. To enable batching, the number of actions can be allowed to vary from 1 to some max number: + +.. code-block:: python + + class MyParamActionEnv(gym.Env): + def __init__(self, max_avail_actions): + self.action_space = Discrete(max_avail_actions) + self.observation_space = Dict({ + "action_mask": Box(0, 1, shape=(max_avail_actions, )), + "avail_actions": Box(-1, 1, shape=(max_avail_actions, action_embedding_sz)), + "real_obs": ..., + }) + +2. A custom model can be defined that can interpret the ``action_mask`` and ``avail_actions`` portions of the observation. Here the model computes the action logits via the dot product of some network output and each action embedding. Invalid actions can be masked out of the softmax by scaling the probability to zero: + +.. code-block:: python + + class MyParamActionModel(Model): + def _build_layers_v2(self, input_dict, num_outputs, options): + avail_actions = input_dict["obs"]["avail_actions"] + action_mask = input_dict["obs"]["action_mask"] + + output = FullyConnectedNetwork( + input_dict["obs"]["real_obs"], num_outputs=action_embedding_sz) + + # Expand the model output to [BATCH, 1, EMBED_SIZE]. Note that the + # avail actions tensor is of shape [BATCH, MAX_ACTIONS, EMBED_SIZE]. + intent_vector = tf.expand_dims(output, 1) + + # Shape of logits is [BATCH, MAX_ACTIONS]. + action_logits = tf.reduce_sum(avail_actions * intent_vector, axis=2) + + # Mask out invalid actions (use tf.float32.min for stability) + inf_mask = tf.maximum(tf.log(action_mask), tf.float32.min) + masked_logits = inf_mask + action_logits + + return masked_logits, last_layer + + +Depending on your use case it may make sense to use just the masking, just action embeddings, or both. For a runnable example of this in code, check out `parametric_action_cartpole.py `__. Note that since masking introduces ``tf.float32.min`` values into the model output, this technique might not work with all algorithm options. For example, algorithms might crash if they incorrectly process the ``tf.float32.min`` values. The cartpole example has working configurations for DQN and several policy gradient algorithms. + + Model-Based Rollouts -------------------- diff --git a/doc/source/rllib-training.rst b/doc/source/rllib-training.rst index 6b1366f4ee08..dc37d22943ba 100644 --- a/doc/source/rllib-training.rst +++ b/doc/source/rllib-training.rst @@ -73,13 +73,13 @@ In an example below, we train A2C by specifying 8 workers through the config fla python ray/python/ray/rllib/train.py --env=PongDeterministic-v4 \ --run=A2C --config '{"num_workers": 8}' -.. image:: rllib-config.svg - Specifying Resources ~~~~~~~~~~~~~~~~~~~~ You can control the degree of parallelism used by setting the ``num_workers`` hyperparameter for most agents. The number of GPUs the driver should use can be set via the ``num_gpus`` option. Similarly, the resource allocation to workers can be controlled via ``num_cpus_per_worker``, ``num_gpus_per_worker``, and ``custom_resources_per_worker``. The number of GPUs can be a fractional quantity to allocate only a fraction of a GPU. For example, with DQN you can pack five agents onto one GPU by setting ``num_gpus: 0.2``. Note that in Ray < 0.6.0 fractional GPU support requires setting the environment variable ``RAY_USE_XRAY=1``. +.. image:: rllib-config.svg + Common Parameters ~~~~~~~~~~~~~~~~~ diff --git a/doc/source/rllib.rst b/doc/source/rllib.rst index 2de444b52965..e96bd6fccbcb 100644 --- a/doc/source/rllib.rst +++ b/doc/source/rllib.rst @@ -56,7 +56,7 @@ Algorithms - `Deep Deterministic Policy Gradients (DDPG, TD3) `__ - - `Deep Q Networks (DQN, Rainbow) `__ + - `Deep Q Networks (DQN, Rainbow, Parametric DQN) `__ - `Policy Gradients `__ @@ -75,6 +75,7 @@ Models and Preprocessors * `Custom Models `__ * `Custom Preprocessors `__ * `Customizing Policy Graphs `__ +* `Variable-length / Parametric Action Spaces `__ * `Model-Based Rollouts `__ RLlib Concepts diff --git a/python/ray/rllib/agents/dqn/dqn_policy_graph.py b/python/ray/rllib/agents/dqn/dqn_policy_graph.py index 6125cd9d387b..2bbff99246f3 100644 --- a/python/ray/rllib/agents/dqn/dqn_policy_graph.py +++ b/python/ray/rllib/agents/dqn/dqn_policy_graph.py @@ -30,16 +30,21 @@ def __init__(self, sigma0=0.5): self.model = model with tf.variable_scope("action_value"): - action_out = model.last_layer - for i in range(len(hiddens)): - if use_noisy: - action_out = self.noisy_layer("hidden_%d" % i, action_out, - hiddens[i], sigma0) - else: - action_out = layers.fully_connected( - action_out, - num_outputs=hiddens[i], - activation_fn=tf.nn.relu) + if hiddens: + action_out = model.last_layer + for i in range(len(hiddens)): + if use_noisy: + action_out = self.noisy_layer( + "hidden_%d" % i, action_out, hiddens[i], sigma0) + else: + action_out = layers.fully_connected( + action_out, + num_outputs=hiddens[i], + activation_fn=tf.nn.relu) + else: + # Avoid postprocessing the outputs. This enables custom models + # to be used for parametric action DQN. + action_out = model.outputs if use_noisy: action_scores = self.noisy_layer( "output", @@ -47,11 +52,13 @@ def __init__(self, num_actions * num_atoms, sigma0, non_linear=False) - else: + elif hiddens: action_scores = layers.fully_connected( action_out, num_outputs=num_actions * num_atoms, activation_fn=None) + else: + action_scores = model.outputs if num_atoms > 1: # Distributional Q-learning uses a discrete support z # to represent the action value distribution @@ -107,7 +114,7 @@ def __init__(self, self.logits = support_logits_per_action self.dist = support_prob_per_action else: - action_scores_mean = tf.reduce_mean(action_scores, 1) + action_scores_mean = _reduce_mean_ignore_inf(action_scores, 1) action_scores_centered = action_scores - tf.expand_dims( action_scores_mean, 1) self.value = state_score + action_scores_centered @@ -176,11 +183,15 @@ class QValuePolicy(object): def __init__(self, q_values, observations, num_actions, stochastic, eps): deterministic_actions = tf.argmax(q_values, axis=1) batch_size = tf.shape(observations)[0] - random_actions = tf.random_uniform( - tf.stack([batch_size]), - minval=0, - maxval=num_actions, - dtype=tf.int64) + + # Special case masked out actions (q_value ~= -inf) so that we don't + # even consider them for exploration. + random_valid_action_logits = tf.where( + tf.equal(q_values, tf.float32.min), + tf.ones_like(q_values) * tf.float32.min, tf.ones_like(q_values)) + random_actions = tf.squeeze( + tf.multinomial(random_valid_action_logits, 1), axis=1) + chose_random = tf.random_uniform( tf.stack([batch_size]), minval=0, maxval=1, dtype=tf.float32) < eps stochastic_actions = tf.where(chose_random, random_actions, @@ -368,8 +379,8 @@ def _build_q_network(self, obs, space): qnet = QNetwork( ModelCatalog.get_model({ "obs": obs - }, space, 1, self.config["model"]), self.num_actions, - self.config["dueling"], self.config["hiddens"], + }, space, self.num_actions, self.config["model"]), + self.num_actions, self.config["dueling"], self.config["hiddens"], self.config["noisy"], self.config["num_atoms"], self.config["v_min"], self.config["v_max"], self.config["sigma0"]) return qnet.value, qnet.logits, qnet.dist, qnet.model @@ -507,6 +518,14 @@ def _postprocess_dqn(policy_graph, sample_batch): return batch +def _reduce_mean_ignore_inf(x, axis): + """Same as tf.reduce_mean() but ignores -inf values.""" + mask = tf.not_equal(x, tf.float32.min) + x_zeroed = tf.where(mask, x, tf.zeros_like(x)) + return (tf.reduce_sum(x_zeroed, axis) / tf.reduce_sum( + tf.cast(mask, tf.float32), axis)) + + def _huber_loss(x, delta=1.0): """Reference: https://en.wikipedia.org/wiki/Huber_loss""" return tf.where( diff --git a/python/ray/rllib/agents/ppo/ppo.py b/python/ray/rllib/agents/ppo/ppo.py index eb556877c5a7..722f9263d816 100644 --- a/python/ray/rllib/agents/ppo/ppo.py +++ b/python/ray/rllib/agents/ppo/ppo.py @@ -110,6 +110,11 @@ def _validate_config(self): and not self.config["simple_optimizer"]): logger.warn("forcing simple_optimizer=True in multi-agent mode") self.config["simple_optimizer"] = True + if self.config["observation_filter"] != "NoFilter": + # TODO(ekl): consider setting the default to be NoFilter + logger.warn( + "By default, observations will be normalized with {}".format( + self.config["observation_filter"])) def _train(self): prev_steps = self.optimizer.num_steps_sampled diff --git a/python/ray/rllib/examples/parametric_action_cartpole.py b/python/ray/rllib/examples/parametric_action_cartpole.py new file mode 100644 index 000000000000..a1438f0a2412 --- /dev/null +++ b/python/ray/rllib/examples/parametric_action_cartpole.py @@ -0,0 +1,196 @@ +"""Example of handling variable length and/or parametric action spaces. + +This is a toy example of the action-embedding based approach for handling large +discrete action spaces (potentially infinite in size), similar to how +OpenAI Five works: + + https://neuro.cs.ut.ee/the-use-of-embeddings-in-openai-five/ + +This currently works with RLlib's policy gradient style algorithms +(e.g., PG, PPO, IMPALA, A2C) and also DQN. + +Note that since the model outputs now include "-inf" tf.float32.min +values, not all algorithm options are supported at the moment. For example, +algorithms might crash if they don't properly ignore the -inf action scores. +Working configurations are given below. +""" + +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import argparse +import random +import numpy as np +import gym +from gym.spaces import Box, Discrete, Dict +import tensorflow as tf +import tensorflow.contrib.slim as slim + +import ray +from ray.rllib.models import Model, ModelCatalog +from ray.rllib.models.misc import normc_initializer +from ray.tune import run_experiments +from ray.tune.registry import register_env + +parser = argparse.ArgumentParser() +parser.add_argument("--stop", type=int, default=200) +parser.add_argument("--run", type=str, default="PPO") + + +class ParametricActionCartpole(gym.Env): + """Parametric action version of CartPole. + + In this env there are only ever two valid actions, but we pretend there are + actually up to `max_avail_actions` actions that can be taken, and the two + valid actions are randomly hidden among this set. + + At each step, we emit a dict of: + - the actual cart observation + - a mask of valid actions (e.g., [0, 0, 1, 0, 0, 1] for 6 max avail) + - the list of action embeddings (w/ zeroes for invalid actions) (e.g., + [[0, 0], + [0, 0], + [-0.2322, -0.2569], + [0, 0], + [0, 0], + [0.7878, 1.2297]] for max_avail_actions=6) + + In a real environment, the actions embeddings would be larger than two + units of course, and also there would be a variable number of valid actions + per step instead of always [LEFT, RIGHT]. + """ + + def __init__(self, max_avail_actions): + # Use simple random 2-unit action embeddings for [LEFT, RIGHT] + self.left_action_embed = np.random.randn(2) + self.right_action_embed = np.random.randn(2) + self.action_space = Discrete(max_avail_actions) + self.wrapped = gym.make("CartPole-v0") + self.observation_space = Dict({ + "action_mask": Box(0, 1, shape=(max_avail_actions, )), + "avail_actions": Box(-1, 1, shape=(max_avail_actions, 2)), + "cart": self.wrapped.observation_space, + }) + + def update_avail_actions(self): + self.action_assignments = [[0, 0]] * self.action_space.n + self.action_mask = [0] * self.action_space.n + self.left_idx, self.right_idx = random.sample( + range(self.action_space.n), 2) + self.action_assignments[self.left_idx] = self.left_action_embed + self.action_assignments[self.right_idx] = self.right_action_embed + self.action_mask[self.left_idx] = 1 + self.action_mask[self.right_idx] = 1 + + def reset(self): + self.update_avail_actions() + return { + "action_mask": self.action_mask, + "avail_actions": self.action_assignments, + "cart": self.wrapped.reset(), + } + + def step(self, action): + if action == self.left_idx: + actual_action = 0 + elif action == self.right_idx: + actual_action = 1 + else: + raise ValueError( + "Chosen action was not one of the non-zero action embeddings", + action, self.action_assignments, self.action_mask, + self.left_idx, self.right_idx) + orig_obs, rew, done, info = self.wrapped.step(actual_action) + self.update_avail_actions() + obs = { + "action_mask": self.action_mask, + "avail_actions": self.action_assignments, + "cart": orig_obs, + } + return obs, rew, done, info + + +class ParametricActionsModel(Model): + """Parametric action model that handles the dot product and masking. + + This assumes the outputs are logits for a single Categorical action dist. + Getting this to work with a more complex output (e.g., if the action space + is a tuple of several distributions) is also possible but left as an + exercise to the reader. + """ + + def _build_layers_v2(self, input_dict, num_outputs, options): + # Extract the available actions tensor from the observation. + avail_actions = input_dict["obs"]["avail_actions"] + action_mask = input_dict["obs"]["action_mask"] + action_embed_size = avail_actions.shape[2].value + if num_outputs != avail_actions.shape[1].value: + raise ValueError( + "This model assumes num outputs is equal to max avail actions", + num_outputs, avail_actions) + + # Standard FC net component. + last_layer = input_dict["obs"]["cart"] + hiddens = [256, 256] + for i, size in enumerate(hiddens): + label = "fc{}".format(i) + last_layer = slim.fully_connected( + last_layer, + size, + weights_initializer=normc_initializer(1.0), + activation_fn=tf.nn.tanh, + scope=label) + output = slim.fully_connected( + last_layer, + action_embed_size, + weights_initializer=normc_initializer(0.01), + activation_fn=None, + scope="fc_out") + + # Expand the model output to [BATCH, 1, EMBED_SIZE]. Note that the + # avail actions tensor is of shape [BATCH, MAX_ACTIONS, EMBED_SIZE]. + intent_vector = tf.expand_dims(output, 1) + + # Batch dot product => shape of logits is [BATCH, MAX_ACTIONS]. + action_logits = tf.reduce_sum(avail_actions * intent_vector, axis=2) + + # Mask out invalid actions (use tf.float32.min for stability) + inf_mask = tf.maximum(tf.log(action_mask), tf.float32.min) + masked_logits = inf_mask + action_logits + + return masked_logits, last_layer + + +if __name__ == "__main__": + args = parser.parse_args() + ray.init() + + ModelCatalog.register_custom_model("pa_model", ParametricActionsModel) + register_env("pa_cartpole", lambda _: ParametricActionCartpole(10)) + if args.run == "PPO": + cfg = { + "observation_filter": "NoFilter", # don't filter the action list + "vf_share_layers": True, # don't create duplicate value model + } + elif args.run == "DQN": + cfg = { + "hiddens": [], # don't postprocess the action scores + } + else: + cfg = {} + run_experiments({ + "parametric_cartpole": { + "run": args.run, + "env": "pa_cartpole", + "stop": { + "episode_reward_mean": args.stop, + }, + "config": dict({ + "model": { + "custom_model": "pa_model", + }, + "num_workers": 0, + }, **cfg), + }, + }) diff --git a/python/ray/rllib/models/catalog.py b/python/ray/rllib/models/catalog.py index 8f0b8ac82540..63a7e73890cc 100644 --- a/python/ray/rllib/models/catalog.py +++ b/python/ray/rllib/models/catalog.py @@ -217,7 +217,7 @@ def _get_model(input_dict, obs_space, num_outputs, options, state_in, seq_lens): if options.get("custom_model"): model = options["custom_model"] - logger.info("Using custom model {}".format(model)) + logger.debug("Using custom model {}".format(model)) return _global_registry.get(RLLIB_MODEL, model)( input_dict, obs_space, diff --git a/python/ray/rllib/models/preprocessors.py b/python/ray/rllib/models/preprocessors.py index 074fda29b96a..a4af708b7915 100644 --- a/python/ray/rllib/models/preprocessors.py +++ b/python/ray/rllib/models/preprocessors.py @@ -2,6 +2,7 @@ from __future__ import division from __future__ import print_function +from collections import OrderedDict import cv2 import logging import numpy as np @@ -164,6 +165,8 @@ def _init_shape(self, obs_space, options): return (size, ) def transform(self, observation): + if not isinstance(observation, OrderedDict): + observation = OrderedDict(sorted(list(observation.items()))) assert len(observation) == len(self.preprocessors), \ (len(observation), len(self.preprocessors)) return np.concatenate([ diff --git a/python/ray/rllib/tuned_examples/atari-dist-dqn.yaml b/python/ray/rllib/tuned_examples/atari-dist-dqn.yaml index 57cd5635d78b..d351e403f2e2 100644 --- a/python/ray/rllib/tuned_examples/atari-dist-dqn.yaml +++ b/python/ray/rllib/tuned_examples/atari-dist-dqn.yaml @@ -27,5 +27,5 @@ basic-dqn: prioritized_replay_alpha: 0.5 beta_annealing_fraction: 1.0 final_prioritized_replay_beta: 1.0 - num_gpus: 1 + num_gpus: 0.2 timesteps_per_iteration: 10000 diff --git a/python/ray/rllib/tuned_examples/atari-dqn.yaml b/python/ray/rllib/tuned_examples/atari-dqn.yaml index 264ddfd27b41..b8731bb054ef 100644 --- a/python/ray/rllib/tuned_examples/atari-dqn.yaml +++ b/python/ray/rllib/tuned_examples/atari-dqn.yaml @@ -1,4 +1,4 @@ -# Runs on a single g3.16xl node +# Runs on a single g3.4xl node # See https://github.com/ray-project/rl-experiments for results atari-basic-dqn: env: @@ -29,5 +29,5 @@ atari-basic-dqn: prioritized_replay_alpha: 0.5 beta_annealing_fraction: 1.0 final_prioritized_replay_beta: 1.0 - num_gpus: 1 + num_gpus: 0.2 timesteps_per_iteration: 10000 diff --git a/python/ray/rllib/tuned_examples/atari-duel-ddqn.yaml b/python/ray/rllib/tuned_examples/atari-duel-ddqn.yaml index be59d15ba807..b5a13162b61e 100644 --- a/python/ray/rllib/tuned_examples/atari-duel-ddqn.yaml +++ b/python/ray/rllib/tuned_examples/atari-duel-ddqn.yaml @@ -1,3 +1,5 @@ +# Runs on a single g3.4xl node +# See https://github.com/ray-project/rl-experiments for results dueling-ddqn: env: grid_search: @@ -27,5 +29,5 @@ dueling-ddqn: prioritized_replay_alpha: 0.5 beta_annealing_fraction: 1.0 final_prioritized_replay_beta: 1.0 - num_gpus: 1 + num_gpus: 0.2 timesteps_per_iteration: 10000 diff --git a/python/ray/rllib/tuned_examples/pong-impala-fast.yaml b/python/ray/rllib/tuned_examples/pong-impala-fast.yaml index 3466b63ea1c4..3c29f4e0c08e 100644 --- a/python/ray/rllib/tuned_examples/pong-impala-fast.yaml +++ b/python/ray/rllib/tuned_examples/pong-impala-fast.yaml @@ -9,7 +9,7 @@ pong-impala-fast: config: sample_batch_size: 50 train_batch_size: 1000 - num_workers: 256 + num_workers: 128 num_envs_per_worker: 5 broadcast_interval: 5 max_sample_requests_in_flight_per_worker: 1 diff --git a/test/jenkins_tests/run_multi_node_tests.sh b/test/jenkins_tests/run_multi_node_tests.sh index 40e9635d9d8a..86fd98af21d0 100755 --- a/test/jenkins_tests/run_multi_node_tests.sh +++ b/test/jenkins_tests/run_multi_node_tests.sh @@ -257,6 +257,15 @@ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/rllib/test/test_external_env.py +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/parametric_action_cartpole.py --run=PG --stop=50 + +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/parametric_action_cartpole.py --run=PPO --stop=50 + +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/parametric_action_cartpole.py --run=DQN --stop=50 + docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/rllib/test/test_lstm.py From 82863b525110d242e781341524c73e239dd3ad13 Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Tue, 27 Nov 2018 23:46:27 -0800 Subject: [PATCH 18/40] [autoscaler] Update autoscaler to use heartbeat batches. (#3409) --- doc/source/conf.py | 1 + python/ray/gcs_utils.py | 7 ++-- python/ray/monitor.py | 77 +++++++++++++++++++---------------------- test/failure_test.py | 6 ++-- 4 files changed, 45 insertions(+), 46 deletions(-) diff --git a/doc/source/conf.py b/doc/source/conf.py index e362f73309a3..2a2b1a37c207 100644 --- a/doc/source/conf.py +++ b/doc/source/conf.py @@ -42,6 +42,7 @@ "ray.core.generated.ClientTableData", "ray.core.generated.GcsTableEntry", "ray.core.generated.HeartbeatTableData", + "ray.core.generated.HeartbeatBatchTableData", "ray.core.generated.DriverTableData", "ray.core.generated.ErrorTableData", "ray.core.generated.ProfileTableData", diff --git a/python/ray/gcs_utils.py b/python/ray/gcs_utils.py index bbdbe04cf7fd..347f7ab9f806 100644 --- a/python/ray/gcs_utils.py +++ b/python/ray/gcs_utils.py @@ -11,6 +11,7 @@ from ray.core.generated.ErrorTableData import ErrorTableData from ray.core.generated.ProfileTableData import ProfileTableData from ray.core.generated.HeartbeatTableData import HeartbeatTableData +from ray.core.generated.HeartbeatBatchTableData import HeartbeatBatchTableData from ray.core.generated.DriverTableData import DriverTableData from ray.core.generated.ObjectTableData import ObjectTableData from ray.core.generated.ray.protocol.Task import Task @@ -20,14 +21,16 @@ __all__ = [ "GcsTableEntry", "ClientTableData", "ErrorTableData", "HeartbeatTableData", - "DriverTableData", "ProfileTableData", "ObjectTableData", "Task", - "TablePrefix", "TablePubsub", "construct_error_message" + "HeartbeatBatchTableData", "DriverTableData", "ProfileTableData", + "ObjectTableData", "Task", "TablePrefix", "TablePubsub", + "construct_error_message" ] FUNCTION_PREFIX = "RemoteFunction:" # xray heartbeats XRAY_HEARTBEAT_CHANNEL = str(TablePubsub.HEARTBEAT).encode("ascii") +XRAY_HEARTBEAT_BATCH_CHANNEL = str(TablePubsub.HEARTBEAT_BATCH).encode("ascii") # xray driver updates XRAY_DRIVER_CHANNEL = str(TablePubsub.DRIVER).encode("ascii") diff --git a/python/ray/monitor.py b/python/ray/monitor.py index 625641790de9..a37f75de7cf1 100644 --- a/python/ray/monitor.py +++ b/python/ray/monitor.py @@ -50,11 +50,6 @@ def __init__(self, # Setup subscriptions to the primary Redis server and the Redis shards. self.primary_subscribe_client = self.redis.pubsub( ignore_subscribe_messages=True) - self.shard_subscribe_clients = [] - for redis_client in self.state.redis_clients: - subscribe_client = redis_client.pubsub( - ignore_subscribe_messages=True) - self.shard_subscribe_clients.append(subscribe_client) # Keep a mapping from local scheduler client ID to IP address to use # for updating the load metrics. self.local_scheduler_id_to_ip_map = {} @@ -90,49 +85,50 @@ def __init__(self, str(e))) self.issue_gcs_flushes = False - def subscribe(self, channel, primary=True): - """Subscribe to the given channel. + def subscribe(self, channel): + """Subscribe to the given channel on the primary Redis shard. Args: channel (str): The channel to subscribe to. - primary: If True, then we only subscribe to the primary Redis - shard. Otherwise we subscribe to all of the other shards but - not the primary. Raises: Exception: An exception is raised if the subscription fails. """ - if primary: - self.primary_subscribe_client.subscribe(channel) - else: - for subscribe_client in self.shard_subscribe_clients: - subscribe_client.subscribe(channel) + self.primary_subscribe_client.subscribe(channel) - def xray_heartbeat_handler(self, unused_channel, data): - """Handle an xray heartbeat message from Redis.""" + def xray_heartbeat_batch_handler(self, unused_channel, data): + """Handle an xray heartbeat batch message from Redis.""" gcs_entries = ray.gcs_utils.GcsTableEntry.GetRootAsGcsTableEntry( data, 0) heartbeat_data = gcs_entries.Entries(0) - message = ray.gcs_utils.HeartbeatTableData.GetRootAsHeartbeatTableData( - heartbeat_data, 0) - num_resources = message.ResourcesAvailableLabelLength() - static_resources = {} - dynamic_resources = {} - for i in range(num_resources): - dyn = message.ResourcesAvailableLabel(i) - static = message.ResourcesTotalLabel(i) - dynamic_resources[dyn] = message.ResourcesAvailableCapacity(i) - static_resources[static] = message.ResourcesTotalCapacity(i) - - # Update the load metrics for this local scheduler. - client_id = ray.utils.binary_to_hex(message.ClientId()) - ip = self.local_scheduler_id_to_ip_map.get(client_id) - if ip: - self.load_metrics.update(ip, static_resources, dynamic_resources) - else: - print("Warning: could not find ip for client {} in {}.".format( - client_id, self.local_scheduler_id_to_ip_map)) + + message = (ray.gcs_utils.HeartbeatBatchTableData. + GetRootAsHeartbeatBatchTableData(heartbeat_data, 0)) + + for j in range(message.BatchLength()): + heartbeat_message = message.Batch(j) + + num_resources = heartbeat_message.ResourcesAvailableLabelLength() + static_resources = {} + dynamic_resources = {} + for i in range(num_resources): + dyn = heartbeat_message.ResourcesAvailableLabel(i) + static = heartbeat_message.ResourcesTotalLabel(i) + dynamic_resources[dyn] = ( + heartbeat_message.ResourcesAvailableCapacity(i)) + static_resources[static] = ( + heartbeat_message.ResourcesTotalCapacity(i)) + + # Update the load metrics for this local scheduler. + client_id = ray.utils.binary_to_hex(heartbeat_message.ClientId()) + ip = self.local_scheduler_id_to_ip_map.get(client_id) + if ip: + self.load_metrics.update(ip, static_resources, + dynamic_resources) + else: + print("Warning: could not find ip for client {} in {}.".format( + client_id, self.local_scheduler_id_to_ip_map)) def _xray_clean_up_entries_for_driver(self, driver_id): """Remove this driver's object/task entries from redis. @@ -222,8 +218,7 @@ def process_messages(self, max_messages=10000): max_messages: The maximum number of messages to process before returning. """ - subscribe_clients = ( - [self.primary_subscribe_client] + self.shard_subscribe_clients) + subscribe_clients = [self.primary_subscribe_client] for subscribe_client in subscribe_clients: for _ in range(max_messages): message = subscribe_client.get_message() @@ -237,9 +232,9 @@ def process_messages(self, max_messages=10000): # Determine the appropriate message handler. message_handler = None - if channel == ray.gcs_utils.XRAY_HEARTBEAT_CHANNEL: + if channel == ray.gcs_utils.XRAY_HEARTBEAT_BATCH_CHANNEL: # Similar functionality as local scheduler info channel - message_handler = self.xray_heartbeat_handler + message_handler = self.xray_heartbeat_batch_handler elif channel == ray.gcs_utils.XRAY_DRIVER_CHANNEL: # Handles driver death. message_handler = self.xray_driver_removed_handler @@ -299,7 +294,7 @@ def run(self): clients and cleaning up state accordingly. """ # Initialize the subscription channel. - self.subscribe(ray.gcs_utils.XRAY_HEARTBEAT_CHANNEL, primary=False) + self.subscribe(ray.gcs_utils.XRAY_HEARTBEAT_BATCH_CHANNEL) self.subscribe(ray.gcs_utils.XRAY_DRIVER_CHANNEL) # TODO(rkn): If there were any dead clients at startup, we should clean diff --git a/test/failure_test.py b/test/failure_test.py index 7895cbf1ac96..027ed38d6411 100644 --- a/test/failure_test.py +++ b/test/failure_test.py @@ -500,10 +500,10 @@ def test_warning_monitor_died(shutdown_only): # addition to the monitor. fake_id = 20 * b"\x00" malformed_message = "asdf" - redis_client = ray.worker.global_state.redis_clients[0] + redis_client = ray.worker.global_worker.redis_client redis_client.execute_command( - "RAY.TABLE_ADD", ray.gcs_utils.TablePrefix.HEARTBEAT, - ray.gcs_utils.TablePubsub.HEARTBEAT, fake_id, malformed_message) + "RAY.TABLE_ADD", ray.gcs_utils.TablePrefix.HEARTBEAT_BATCH, + ray.gcs_utils.TablePubsub.HEARTBEAT_BATCH, fake_id, malformed_message) wait_for_errors(ray_constants.MONITOR_DIED_ERROR, 1) From 139fbf7884445457cb076093c283ad0d19125017 Mon Sep 17 00:00:00 2001 From: Tianming Xu Date: Wed, 28 Nov 2018 15:51:18 +0800 Subject: [PATCH 19/40] Initialize client_id_ in ObjectManager constructor that takes user-defined ObjectDirectory (#3403) --- src/ray/object_manager/object_directory.cc | 4 ++++ src/ray/object_manager/object_directory.h | 7 +++++++ src/ray/object_manager/object_manager.cc | 2 +- src/ray/raylet/reconstruction_policy_test.cc | 1 + 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index fce8b11349cd..ab20d27b66c6 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -192,6 +192,10 @@ ray::Status ObjectDirectory::LookupLocations(const ObjectID &object_id, return status; } +ray::ClientID ObjectDirectory::GetLocalClientID() { + return gcs_client_->client_table().GetLocalClientId(); +} + std::string ObjectDirectory::DebugString() const { std::stringstream result; result << "ObjectDirectory:"; diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 7d6f1171079d..e36c4c41604e 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -104,6 +104,11 @@ class ObjectDirectoryInterface { virtual ray::Status ReportObjectRemoved(const ObjectID &object_id, const ClientID &client_id) = 0; + /// Get local client id + /// + /// \return ClientID + virtual ray::ClientID GetLocalClientID() = 0; + /// Returns debug string for class. /// /// \return string. @@ -145,6 +150,8 @@ class ObjectDirectory : public ObjectDirectoryInterface { ray::Status ReportObjectRemoved(const ObjectID &object_id, const ClientID &client_id) override; + ray::ClientID GetLocalClientID() override; + std::string DebugString() const override; /// ObjectDirectory should not be copied. diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index e9edf5f842d7..959235c95794 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -52,7 +52,7 @@ ObjectManager::ObjectManager(asio::io_service &main_service, gen_(std::chrono::high_resolution_clock::now().time_since_epoch().count()) { RAY_CHECK(config_.max_sends > 0); RAY_CHECK(config_.max_receives > 0); - // TODO(hme) Client ID is never set with this constructor. + client_id_ = object_directory_->GetLocalClientID(); main_service_ = &main_service; store_notification_.SubscribeObjAdded( [this](const object_manager::protocol::ObjectInfoT &object_info) { diff --git a/src/ray/raylet/reconstruction_policy_test.cc b/src/ray/raylet/reconstruction_policy_test.cc index 4062511ae3dd..9f1499c31664 100644 --- a/src/ray/raylet/reconstruction_policy_test.cc +++ b/src/ray/raylet/reconstruction_policy_test.cc @@ -39,6 +39,7 @@ class MockObjectDirectory : public ObjectDirectoryInterface { std::string DebugString() const { return ""; } MOCK_METHOD0(RegisterBackend, void(void)); + MOCK_METHOD0(GetLocalClientID, ray::ClientID()); MOCK_CONST_METHOD1(LookupRemoteConnectionInfo, void(RemoteConnectionInfo &)); MOCK_CONST_METHOD0(LookupAllRemoteConnections, std::vector()); MOCK_METHOD3(SubscribeObjectLocations, From c46ea2ff4b2d45f4205050d78048300bd2d12c4c Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 28 Nov 2018 14:59:58 -0800 Subject: [PATCH 20/40] Click 0.7 changes the naming convention for commands; fix this --- python/ray/scripts/scripts.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 929ee3785906..13fa63efbd40 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -674,16 +674,15 @@ def stack(): cli.add_command(start) cli.add_command(stop) -cli.add_command(create_or_update) cli.add_command(create_or_update, name="up") cli.add_command(attach) cli.add_command(exec_cmd, name="exec") -cli.add_command(rsync_down) -cli.add_command(rsync_up) +cli.add_command(rsync_down, name="rsync_down") +cli.add_command(rsync_up, name="rsync_up") cli.add_command(submit) cli.add_command(teardown) cli.add_command(teardown, name="down") -cli.add_command(get_head_ip) +cli.add_command(get_head_ip, name="get_head_ip") cli.add_command(stack) From 7e319dbf0ce36ce4260ed9425e5b49bb394cc87a Mon Sep 17 00:00:00 2001 From: Kristian Hartikainen Date: Thu, 29 Nov 2018 00:15:50 -0800 Subject: [PATCH 21/40] Automatically indent tune logger params (#3399) --- python/ray/tune/logger.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/logger.py b/python/ray/tune/logger.py index f73aa4a1ef8c..d2c79e6d871a 100644 --- a/python/ray/tune/logger.py +++ b/python/ray/tune/logger.py @@ -97,7 +97,12 @@ class _JsonLogger(Logger): def _init(self): config_out = os.path.join(self.logdir, "params.json") with open(config_out, "w") as f: - json.dump(self.config, f, sort_keys=True, cls=_SafeFallbackEncoder) + json.dump( + self.config, + f, + indent=2, + sort_keys=True, + cls=_SafeFallbackEncoder) local_file = os.path.join(self.logdir, "result.json") self.local_out = open(local_file, "w") From fd7e4943448ebdc057b799aa5f573fcfe63658f9 Mon Sep 17 00:00:00 2001 From: Chunyang Wen Date: Fri, 30 Nov 2018 02:21:46 +0800 Subject: [PATCH 22/40] Remove: duplicate feed_dict constructing (#3431) --- python/ray/experimental/sgd/sgd_worker.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/python/ray/experimental/sgd/sgd_worker.py b/python/ray/experimental/sgd/sgd_worker.py index 3dd0eefca86d..0d4b45c7c8bc 100644 --- a/python/ray/experimental/sgd/sgd_worker.py +++ b/python/ray/experimental/sgd/sgd_worker.py @@ -205,9 +205,6 @@ def for_model(self, fn): def compute_gradients(self): start = time.time() feed_dict = self._grad_feed_dict() - # Aggregate feed dicts for each model on this worker. - for model in self.models: - feed_dict.update(model.get_feed_dict()) # We only need to fetch the first per_device_grad, since they are # averaged across all devices by allreduce. fetches = self.sess.run( From 48a59352248dda9cd081a720f36750c48f5992e1 Mon Sep 17 00:00:00 2001 From: Stephanie Wang Date: Thu, 29 Nov 2018 10:48:35 -0800 Subject: [PATCH 23/40] Fault tolerance for actor creation (#3422) * Add regression test * Request actor creation if no actor location found * Comments * Address comments * Increase test timeout * Trigger test --- src/ray/raylet/node_manager.cc | 23 ++++++++++- test/component_failures_test.py | 68 +++++++++++++++++++++++++++++++++ 2 files changed, 90 insertions(+), 1 deletion(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 457fa4bba834..9110f0c87881 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -541,6 +541,13 @@ void NodeManager::HandleActorStateTransition(const ActorID &actor_id, << " already removed from the lineage cache. This is most " "likely due to reconstruction."; } + // Maintain the invariant that if a task is in the + // MethodsWaitingForActorCreation queue, then it is subscribed to its + // respective actor creation task and that task only. Since the actor + // location is now known, we can remove the task from the queue and + // forget its dependency on the actor creation task. + RAY_CHECK(task_dependency_manager_.UnsubscribeDependencies( + method.GetTaskSpecification().TaskId())); // The task's uncommitted lineage was already added to the local lineage // cache upon the initial submission, so it's okay to resubmit it with an // empty lineage this time. @@ -1154,6 +1161,15 @@ void NodeManager::SubmitTask(const Task &task, const Lineage &uncommitted_lineag // Keep the task queued until we discover the actor's location. // (See design_docs/task_states.rst for the state transition diagram.) local_queues_.QueueMethodsWaitingForActorCreation({task}); + // The actor has not yet been created and may have failed. To make sure + // that the actor is eventually recreated, we maintain the invariant that + // if a task is in the MethodsWaitingForActorCreation queue, then it is + // subscribed to its respective actor creation task and that task only. + // Once the actor has been created and this method removed from the + // waiting queue, the caller must make the corresponding call to + // UnsubscribeDependencies. + task_dependency_manager_.SubscribeDependencies(spec.TaskId(), + {spec.ActorCreationDummyObjectId()}); // Mark the task as pending. It will be canceled once we discover the // actor's location and either execute the task ourselves or forward it // to another node. @@ -1431,7 +1447,8 @@ void NodeManager::FinishAssignedTask(Worker &worker) { // Publish the actor creation event to all other nodes so that methods for // the actor will be forwarded directly to this node. - RAY_CHECK(actor_registry_.find(actor_id) == actor_registry_.end()); + RAY_CHECK(actor_registry_.find(actor_id) == actor_registry_.end()) + << "Created an actor that already exists"; auto actor_data = std::make_shared(); actor_data->actor_id = actor_id.binary(); actor_data->actor_creation_dummy_object_id = @@ -1447,6 +1464,10 @@ void NodeManager::FinishAssignedTask(Worker &worker) { // index in the log should succeed. auto failure_callback = [](gcs::AsyncGcsClient *client, const ActorID &id, const ActorTableDataT &data) { + // TODO(swang): Instead of making this a fatal check, we could just kill + // the duplicate actor process. If we do this, we must make sure to + // either resubmit the tasks that went to the duplicate actor, or wait + // for success before handling the actor state transition to ALIVE. RAY_LOG(FATAL) << "Failed to update state to ALIVE for actor " << id; }; RAY_CHECK_OK(gcs_client_->actor_table().AppendAt( diff --git a/test/component_failures_test.py b/test/component_failures_test.py index b9d257962120..fd09a17599cf 100644 --- a/test/component_failures_test.py +++ b/test/component_failures_test.py @@ -5,11 +5,14 @@ import os import json import signal +import sys import time +import numpy as np import pytest import ray +from ray.test.cluster_utils import Cluster from ray.test.test_utils import run_string_as_driver_nonblocking @@ -33,6 +36,26 @@ def shutdown_only(): ray.shutdown() +@pytest.fixture +def ray_start_cluster(): + node_args = { + "resources": dict(CPU=8), + "_internal_config": json.dumps({ + "initial_reconstruction_timeout_milliseconds": 1000, + "num_heartbeats_timeout": 10 + }) + } + # Start with 4 worker nodes and 8 cores each. + g = Cluster(initialize_head=True, connect=True, head_node_args=node_args) + workers = [] + for _ in range(4): + workers.append(g.add_node(**node_args)) + g.wait_for_nodes() + yield g + ray.shutdown() + g.shutdown() + + # This test checks that when a worker dies in the middle of a get, the plasma # store and raylet will not die. @pytest.mark.skipif( @@ -347,6 +370,51 @@ def test_plasma_store_failed(): ray.shutdown() +def test_actor_creation_node_failure(ray_start_cluster): + # TODO(swang): Refactor test_raylet_failed, etc to reuse the below code. + cluster = ray_start_cluster + + @ray.remote + class Child(object): + def __init__(self, death_probability): + self.death_probability = death_probability + + def ping(self): + # Exit process with some probability. + exit_chance = np.random.rand() + if exit_chance < self.death_probability: + sys.exit(-1) + + num_children = 100 + # Children actors will die about half the time. + death_probability = 0.5 + + children = [Child.remote(death_probability) for _ in range(num_children)] + while len(cluster.list_all_nodes()) > 1: + for j in range(3): + # Submit some tasks on the actors. About half of the actors will + # fail. + children_out = [child.ping.remote() for child in children] + # Wait a while for all the tasks to complete. This should trigger + # reconstruction for any actor creation tasks that were forwarded + # to nodes that then failed. + ready, _ = ray.wait( + children_out, + num_returns=len(children_out), + timeout=5 * 60 * 1000) + assert len(ready) == len(children_out) + + # Replace any actors that died. + for i, out in enumerate(children_out): + try: + ray.get(out) + except ray.worker.RayGetError: + children[i] = Child.remote(death_probability) + # Remove a node. Any actor creation tasks that were forwarded to this + # node must be reconstructed. + cluster.remove_node(cluster.list_all_nodes()[-1]) + + @pytest.mark.skipif( os.environ.get("RAY_USE_NEW_GCS") == "on", reason="Hanging with new GCS API.") From 4d2010a852dafc7bc2a8dc846e5c5f7fa1cfc0ad Mon Sep 17 00:00:00 2001 From: Devin Petersohn Date: Thu, 29 Nov 2018 11:05:24 -0800 Subject: [PATCH 24/40] Ship Modin with Ray. (#3109) --- .gitignore | 3 +++ .travis.yml | 3 +++ .travis/install-dependencies.sh | 8 ++++---- .travis/test-wheels.sh | 4 ++-- python/ray/__init__.py | 3 +++ python/ray/test/test_modin.py | 12 ++++++++++++ thirdparty/scripts/build_modin.sh | 29 +++++++++++++++++++++++++++++ thirdparty/scripts/setup.sh | 5 +++++ 8 files changed, 61 insertions(+), 6 deletions(-) create mode 100644 python/ray/test/test_modin.py create mode 100755 thirdparty/scripts/build_modin.sh diff --git a/.gitignore b/.gitignore index f8130b3a2f85..91189b6f9c41 100644 --- a/.gitignore +++ b/.gitignore @@ -12,6 +12,9 @@ /src/ray/object_manager/format/*_generated.h /src/ray/raylet/format/*_generated.h +# Modin source files +/python/ray/modin + # Redis temporary files *dump.rdb diff --git a/.travis.yml b/.travis.yml index debf450738a7..b48089d52ecd 100644 --- a/.travis.yml +++ b/.travis.yml @@ -171,6 +171,9 @@ script: # ray temp file tests - python -m pytest -v test/tempfile_test.py + # modin test files + - python python/ray/test/test_modin.py + deploy: - provider: s3 access_key_id: AKIAJ2L7XDUSZVTXI5QA diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index 293c1b8b6b04..0fb597d4686f 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -24,7 +24,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then wget https://repo.continuum.io/miniconda/Miniconda2-4.5.4-Linux-x86_64.sh -O miniconda.sh -nv bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" - pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update @@ -33,7 +33,7 @@ elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then wget https://repo.continuum.io/miniconda/Miniconda3-4.5.4-Linux-x86_64.sh -O miniconda.sh -nv bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" - pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle pytest-timeout elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed @@ -50,7 +50,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then wget https://repo.continuum.io/miniconda/Miniconda2-4.5.4-MacOSX-x86_64.sh -O miniconda.sh -nv bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" - pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed @@ -67,7 +67,7 @@ elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then wget https://repo.continuum.io/miniconda/Miniconda3-4.5.4-MacOSX-x86_64.sh -O miniconda.sh -nv bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" - pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle pytest-timeout elif [[ "$LINT" == "1" ]]; then sudo apt-get update diff --git a/.travis/test-wheels.sh b/.travis/test-wheels.sh index 1765135ec9be..f7870ea52d49 100755 --- a/.travis/test-wheels.sh +++ b/.travis/test-wheels.sh @@ -59,7 +59,7 @@ if [[ "$platform" == "linux" ]]; then if [[ "$NUMBER_OF_WHEELS" != "5" ]]; then echo "Wrong number of wheels found." ls -l $ROOT_DIR/../.whl/ - exit 1 + exit 2 fi elif [[ "$platform" == "macosx" ]]; then @@ -94,5 +94,5 @@ elif [[ "$platform" == "macosx" ]]; then done else echo "Unrecognized environment." - exit 1 + exit 3 fi diff --git a/python/ray/__init__.py b/python/ray/__init__.py index a507cdd2e7a2..6d4e0ba9a6b9 100644 --- a/python/ray/__init__.py +++ b/python/ray/__init__.py @@ -46,6 +46,9 @@ e.args += (helpful_message, ) raise +modin_path = os.path.join(os.path.abspath(os.path.dirname(__file__)), "modin") +sys.path.insert(0, modin_path) + from ray.raylet import ObjectID, _config # noqa: E402 from ray.profiling import profile # noqa: E402 from ray.worker import (error_info, init, connect, disconnect, get, put, wait, diff --git a/python/ray/test/test_modin.py b/python/ray/test/test_modin.py new file mode 100644 index 000000000000..83c11895ec7b --- /dev/null +++ b/python/ray/test/test_modin.py @@ -0,0 +1,12 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import ray # noqa F401 + + +def test_modin_import(): + import modin.pandas as pd + frame_data = [1, 2, 3, 4, 5, 6, 7, 8] + frame = pd.DataFrame(frame_data) + assert frame.sum().squeeze() == sum(frame_data) diff --git a/thirdparty/scripts/build_modin.sh b/thirdparty/scripts/build_modin.sh new file mode 100755 index 000000000000..96563fdb2106 --- /dev/null +++ b/thirdparty/scripts/build_modin.sh @@ -0,0 +1,29 @@ +#!/bin/bash + +set -x + +# Cause the script to exit if a single command fails. +set -e + +if [[ -z "$1" ]]; then + PYTHON_EXECUTABLE=`which python` +else + PYTHON_EXECUTABLE=$1 +fi + +PYTHON_VERSION="$($PYTHON_EXECUTABLE -c 'import sys; print(sys.version_info[0])')" + +TP_DIR=$(cd "$(dirname "${BASH_SOURCE:-$0}")"; pwd)/../ +MODIN_VERSION=0.2.4 +MODIN_WHEELS_FNAME="modin-$MODIN_VERSION-py$PYTHON_VERSION-none-any.whl" +MODIN_WHEELS_URL="https://github.com/modin-project/modin/releases/download/v$MODIN_VERSION/" + +pushd $TP_DIR/../python/ray/ +rm -rf modin +mkdir modin +pushd modin +curl -kL "$MODIN_WHEELS_URL$MODIN_WHEELS_FNAME" -o "$MODIN_WHEELS_FNAME" +unzip "$MODIN_WHEELS_FNAME" +rm "$MODIN_WHEELS_FNAME" +popd +popd diff --git a/thirdparty/scripts/setup.sh b/thirdparty/scripts/setup.sh index 27f1ef0e3ed5..da283bd3b2bb 100755 --- a/thirdparty/scripts/setup.sh +++ b/thirdparty/scripts/setup.sh @@ -67,3 +67,8 @@ bash "$TP_SCRIPT_DIR/build_ui.sh" # rDSN (optional) ############################################## # bash "$TP_SCRIPT_DIR/build_rdsn.sh" + +############################################## +# modin +############################################## +bash "$TP_SCRIPT_DIR/build_modin.sh" $PYTHON_EXECUTABLE From 07d8cbf414df48fc186f3b859512b50c2700c92c Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 29 Nov 2018 13:33:39 -0800 Subject: [PATCH 25/40] [rllib] Support batch norm layers (#3369) * batch norm * lint * fix dqn/ddpg update ops * bn model * Update tf_policy_graph.py * Update multi_gpu_impl.py * Apply suggestions from code review Co-Authored-By: ericl --- doc/source/rllib-models.rst | 14 ++-- .../rllib/agents/a3c/a3c_tf_policy_graph.py | 3 +- python/ray/rllib/agents/agent.py | 8 +-- .../rllib/agents/ddpg/ddpg_policy_graph.py | 20 ++++-- .../ray/rllib/agents/dqn/dqn_policy_graph.py | 10 ++- .../agents/impala/vtrace_policy_graph.py | 1 + python/ray/rllib/agents/pg/pg_policy_graph.py | 3 +- python/ray/rllib/agents/ppo/ppo.py | 5 +- .../ray/rllib/agents/ppo/ppo_policy_graph.py | 6 +- python/ray/rllib/evaluation/policy_graph.py | 6 +- python/ray/rllib/evaluation/sampler.py | 4 +- .../ray/rllib/evaluation/tf_policy_graph.py | 45 +++++++++---- .../rllib/evaluation/torch_policy_graph.py | 1 - python/ray/rllib/examples/batch_norm_model.py | 64 +++++++++++++++++++ python/ray/rllib/models/model.py | 5 +- python/ray/rllib/optimizers/multi_gpu_impl.py | 20 +++++- python/ray/rllib/test/test_multi_agent_env.py | 2 - .../ray/rllib/test/test_policy_evaluator.py | 2 - test/jenkins_tests/run_multi_node_tests.sh | 12 ++++ 19 files changed, 182 insertions(+), 49 deletions(-) create mode 100644 python/ray/rllib/examples/batch_norm_model.py diff --git a/doc/source/rllib-models.rst b/doc/source/rllib-models.rst index 883b0a6bb526..09c49e2751bf 100644 --- a/doc/source/rllib-models.rst +++ b/doc/source/rllib-models.rst @@ -30,7 +30,7 @@ The following is a list of the built-in model hyperparameters: Custom Models ------------- -Custom models should subclass the common RLlib `model class `__ and override the ``_build_layers_v2`` method. This method takes in a dict of tensor inputs (the observation ``obs``, ``prev_action``, and ``prev_reward``), and returns a feature layer and float vector of the specified output size. You can also override the ``value_function`` method to implement a custom value branch. A self-supervised loss can be defined via the ``loss`` method. The model can then be registered and used in place of a built-in model: +Custom models should subclass the common RLlib `model class `__ and override the ``_build_layers_v2`` method. This method takes in a dict of tensor inputs (the observation ``obs``, ``prev_action``, and ``prev_reward``, ``is_training``), and returns a feature layer and float vector of the specified output size. You can also override the ``value_function`` method to implement a custom value branch. A self-supervised loss can be defined via the ``loss`` method. The model can then be registered and used in place of a built-in model: .. code-block:: python @@ -44,7 +44,7 @@ Custom models should subclass the common RLlib `model class >> print(input_dict) {'prev_actions': , 'prev_rewards': , + 'is_training': , 'obs': OrderedDict([ ('sensors', OrderedDict([ ('front_cam', [ @@ -115,7 +116,6 @@ Custom Recurrent Models Instead of using the ``use_lstm: True`` option, it can be preferable use a custom recurrent model. This provides more control over postprocessing of the LSTM output and can also allow the use of multiple LSTM cells to process different portions of the input. The only difference from a normal custom model is that you have to define ``self.state_init``, ``self.state_in``, and ``self.state_out``. You can refer to the existing `lstm.py `__ model as an example to implement your own model: - .. code-block:: python class MyCustomLSTM(Model): @@ -147,6 +147,11 @@ Instead of using the ``use_lstm: True`` option, it can be preferable use a custo normc_initializer(0.01)) return logits, last_layer +Batch Normalization +~~~~~~~~~~~~~~~~~~~ + +You can use ``tf.layers.batch_normalization(x, training=input_dict["is_training"])`` to add batch norm layers to your custom model: `code example `__. RLlib will automatically run the update ops for the batch norm layers during optimization (see `tf_policy_graph.py `__ and `multi_gpu_impl.py `__ for the exact handling of these updates). + Custom Preprocessors -------------------- @@ -283,7 +288,8 @@ With a custom policy graph, you can also perform model-based rollouts and option def compute_actions(self, obs_batch, state_batches, - is_training=False, + prev_action_batch=None, + prev_reward_batch=None, episodes=None): # compute a batch of actions based on the current obs_batch # and state of each episode (i.e., for multiagent). You can do diff --git a/python/ray/rllib/agents/a3c/a3c_tf_policy_graph.py b/python/ray/rllib/agents/a3c/a3c_tf_policy_graph.py index 6f079713abae..8aa60645aaeb 100644 --- a/python/ray/rllib/agents/a3c/a3c_tf_policy_graph.py +++ b/python/ray/rllib/agents/a3c/a3c_tf_policy_graph.py @@ -53,7 +53,8 @@ def __init__(self, observation_space, action_space, config): self.model = ModelCatalog.get_model({ "obs": self.observations, "prev_actions": prev_actions, - "prev_rewards": prev_rewards + "prev_rewards": prev_rewards, + "is_training": self._get_is_training_placeholder(), }, observation_space, logit_dim, self.config["model"]) action_dist = dist_class(self.model.outputs) self.vf = self.model.value_function() diff --git a/python/ray/rllib/agents/agent.py b/python/ray/rllib/agents/agent.py index 18adda82d178..f0d9510756b9 100644 --- a/python/ray/rllib/agents/agent.py +++ b/python/ray/rllib/agents/agent.py @@ -385,13 +385,11 @@ def compute_action(self, observation, state=None, policy_id="default"): observation, update=False) if state: return self.local_evaluator.for_policy( - lambda p: p.compute_single_action( - filtered_obs, state, is_training=False), + lambda p: p.compute_single_action(filtered_obs, state), policy_id=policy_id) return self.local_evaluator.for_policy( - lambda p: p.compute_single_action( - filtered_obs, state, is_training=False)[0], - policy_id=policy_id) + lambda p: p.compute_single_action(filtered_obs, state)[0], + policy_id=policy_id) def get_weights(self, policies=None): """Return a dictionary of policy ids to weights. diff --git a/python/ray/rllib/agents/ddpg/ddpg_policy_graph.py b/python/ray/rllib/agents/ddpg/ddpg_policy_graph.py index 738c4e9ac130..eb5f14c2d1c9 100644 --- a/python/ray/rllib/agents/ddpg/ddpg_policy_graph.py +++ b/python/ray/rllib/agents/ddpg/ddpg_policy_graph.py @@ -199,7 +199,9 @@ def __init__(self, observation_space, action_space, config): self.stochastic = tf.placeholder(tf.bool, (), name="stochastic") self.eps = tf.placeholder(tf.float32, (), name="eps") self.cur_observations = tf.placeholder( - tf.float32, shape=(None, ) + observation_space.shape) + tf.float32, + shape=(None, ) + observation_space.shape, + name="cur_obs") # Actor: P (policy) network with tf.variable_scope(P_SCOPE) as scope: @@ -236,7 +238,11 @@ def __init__(self, observation_space, action_space, config): # p network evaluation with tf.variable_scope(P_SCOPE, reuse=True) as scope: + prev_update_ops = set(tf.get_collection(tf.GraphKeys.UPDATE_OPS)) self.p_t = self._build_p_network(self.obs_t, observation_space) + p_batchnorm_update_ops = list( + set(tf.get_collection(tf.GraphKeys.UPDATE_OPS)) - + prev_update_ops) # target p network evaluation with tf.variable_scope(P_TARGET_SCOPE) as scope: @@ -257,6 +263,7 @@ def __init__(self, observation_space, action_space, config): is_target=True) # q network evaluation + prev_update_ops = set(tf.get_collection(tf.GraphKeys.UPDATE_OPS)) with tf.variable_scope(Q_SCOPE) as scope: q_t, model = self._build_q_network(self.obs_t, observation_space, self.act_t) @@ -269,6 +276,8 @@ def __init__(self, observation_space, action_space, config): twin_q_t, twin_model = self._build_q_network( self.obs_t, observation_space, self.act_t) self.twin_q_func_vars = _scope_vars(scope.name) + q_batchnorm_update_ops = list( + set(tf.get_collection(tf.GraphKeys.UPDATE_OPS)) - prev_update_ops) # target q network evalution with tf.variable_scope(Q_TARGET_SCOPE) as scope: @@ -345,7 +354,8 @@ def __init__(self, observation_space, action_space, config): obs_input=self.cur_observations, action_sampler=self.output_actions, loss=model.loss() + self.loss.total_loss, - loss_inputs=self.loss_inputs) + loss_inputs=self.loss_inputs, + update_ops=q_batchnorm_update_ops + p_batchnorm_update_ops) self.sess.run(tf.global_variables_initializer()) # Note that this encompasses both the policy and Q-value networks and @@ -359,7 +369,8 @@ def __init__(self, observation_space, action_space, config): def _build_q_network(self, obs, obs_space, actions): q_net = QNetwork( ModelCatalog.get_model({ - "obs": obs + "obs": obs, + "is_training": self._get_is_training_placeholder(), }, obs_space, 1, self.config["model"]), actions, self.config["critic_hiddens"], self.config["critic_hidden_activation"]) @@ -368,7 +379,8 @@ def _build_q_network(self, obs, obs_space, actions): def _build_p_network(self, obs, obs_space): return PNetwork( ModelCatalog.get_model({ - "obs": obs + "obs": obs, + "is_training": self._get_is_training_placeholder(), }, obs_space, 1, self.config["model"]), self.dim_actions, self.config["actor_hiddens"], self.config["actor_hidden_activation"]).action_scores diff --git a/python/ray/rllib/agents/dqn/dqn_policy_graph.py b/python/ray/rllib/agents/dqn/dqn_policy_graph.py index 2bbff99246f3..c883ef25067d 100644 --- a/python/ray/rllib/agents/dqn/dqn_policy_graph.py +++ b/python/ray/rllib/agents/dqn/dqn_policy_graph.py @@ -306,8 +306,12 @@ def __init__(self, observation_space, action_space, config): # q network evaluation with tf.variable_scope(Q_SCOPE, reuse=True): + prev_update_ops = set(tf.get_collection(tf.GraphKeys.UPDATE_OPS)) q_t, q_logits_t, q_dist_t, model = self._build_q_network( self.obs_t, observation_space) + q_batchnorm_update_ops = list( + set(tf.get_collection(tf.GraphKeys.UPDATE_OPS)) - + prev_update_ops) # target q network evalution with tf.variable_scope(Q_TARGET_SCOPE) as scope: @@ -372,13 +376,15 @@ def __init__(self, observation_space, action_space, config): obs_input=self.cur_observations, action_sampler=self.output_actions, loss=model.loss() + self.loss.loss, - loss_inputs=self.loss_inputs) + loss_inputs=self.loss_inputs, + update_ops=q_batchnorm_update_ops) self.sess.run(tf.global_variables_initializer()) def _build_q_network(self, obs, space): qnet = QNetwork( ModelCatalog.get_model({ - "obs": obs + "obs": obs, + "is_training": self._get_is_training_placeholder(), }, space, self.num_actions, self.config["model"]), self.num_actions, self.config["dueling"], self.config["hiddens"], self.config["noisy"], self.config["num_atoms"], diff --git a/python/ray/rllib/agents/impala/vtrace_policy_graph.py b/python/ray/rllib/agents/impala/vtrace_policy_graph.py index 3d9e4214b7c7..cfa2f1373aae 100644 --- a/python/ray/rllib/agents/impala/vtrace_policy_graph.py +++ b/python/ray/rllib/agents/impala/vtrace_policy_graph.py @@ -133,6 +133,7 @@ def __init__(self, "obs": observations, "prev_actions": prev_actions, "prev_rewards": prev_rewards, + "is_training": self._get_is_training_placeholder(), }, observation_space, logit_dim, diff --git a/python/ray/rllib/agents/pg/pg_policy_graph.py b/python/ray/rllib/agents/pg/pg_policy_graph.py index 8cbb3a588b49..2a342c117fb3 100644 --- a/python/ray/rllib/agents/pg/pg_policy_graph.py +++ b/python/ray/rllib/agents/pg/pg_policy_graph.py @@ -35,7 +35,8 @@ def __init__(self, obs_space, action_space, config): self.model = ModelCatalog.get_model({ "obs": obs, "prev_actions": prev_actions, - "prev_rewards": prev_rewards + "prev_rewards": prev_rewards, + "is_training": self._get_is_training_placeholder(), }, obs_space, self.logit_dim, self.config["model"]) action_dist = dist_class(self.model.outputs) # logit for each action diff --git a/python/ray/rllib/agents/ppo/ppo.py b/python/ray/rllib/agents/ppo/ppo.py index 722f9263d816..d5e50832f451 100644 --- a/python/ray/rllib/agents/ppo/ppo.py +++ b/python/ray/rllib/agents/ppo/ppo.py @@ -24,7 +24,7 @@ "sample_batch_size": 200, # Number of timesteps collected for each SGD round "train_batch_size": 4000, - # Total SGD batch size across all devices for SGD (multi-gpu only) + # Total SGD batch size across all devices for SGD "sgd_minibatch_size": 128, # Number of SGD iterations in each outer loop "num_sgd_iter": 30, @@ -49,7 +49,8 @@ "batch_mode": "truncate_episodes", # Which observation filter to apply to the observation "observation_filter": "MeanStdFilter", - # Use the sync samples optimizer instead of the multi-gpu one + # Uses the sync samples optimizer instead of the multi-gpu one. This does + # not support minibatches. "simple_optimizer": False, }) # __sphinx_doc_end__ diff --git a/python/ray/rllib/agents/ppo/ppo_policy_graph.py b/python/ray/rllib/agents/ppo/ppo_policy_graph.py index f43a336253be..3762f16f9084 100644 --- a/python/ray/rllib/agents/ppo/ppo_policy_graph.py +++ b/python/ray/rllib/agents/ppo/ppo_policy_graph.py @@ -158,7 +158,8 @@ def __init__(self, { "obs": obs_ph, "prev_actions": prev_actions_ph, - "prev_rewards": prev_rewards_ph + "prev_rewards": prev_rewards_ph, + "is_training": self._get_is_training_placeholder(), }, observation_space, logit_dim, @@ -191,7 +192,8 @@ def __init__(self, self.value_function = ModelCatalog.get_model({ "obs": obs_ph, "prev_actions": prev_actions_ph, - "prev_rewards": prev_rewards_ph + "prev_rewards": prev_rewards_ph, + "is_training": self._get_is_training_placeholder(), }, observation_space, 1, vf_config).outputs self.value_function = tf.reshape(self.value_function, [-1]) else: diff --git a/python/ray/rllib/evaluation/policy_graph.py b/python/ray/rllib/evaluation/policy_graph.py index 9de59d269a03..c19da286b0b9 100644 --- a/python/ray/rllib/evaluation/policy_graph.py +++ b/python/ray/rllib/evaluation/policy_graph.py @@ -42,7 +42,6 @@ def compute_actions(self, state_batches, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): """Compute actions for the current policy. @@ -51,7 +50,6 @@ def compute_actions(self, state_batches (list): list of RNN state input batches, if any prev_action_batch (np.ndarray): batch of previous action values prev_reward_batch (np.ndarray): batch of previous rewards - is_training (bool): whether we are training the policy episodes (list): MultiAgentEpisode for each obs in obs_batch. This provides access to all of the internal episode state, which may be useful for model-based or multiagent algorithms. @@ -71,7 +69,6 @@ def compute_single_action(self, state, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episode=None): """Unbatched version of compute_actions. @@ -80,7 +77,6 @@ def compute_single_action(self, state_batches (list): list of RNN state inputs, if any prev_action_batch (np.ndarray): batch of previous action values prev_reward_batch (np.ndarray): batch of previous rewards - is_training (bool): whether we are training the policy episode (MultiAgentEpisode): this provides access to all of the internal episode state, which may be useful for model-based or multi-agent algorithms. @@ -92,7 +88,7 @@ def compute_single_action(self, """ [action], state_out, info = self.compute_actions( - [obs], [[s] for s in state], is_training, episodes=[episode]) + [obs], [[s] for s in state], episodes=[episode]) return action, [s[0] for s in state_out], \ {k: v[0] for k, v in info.items()} diff --git a/python/ray/rllib/evaluation/sampler.py b/python/ray/rllib/evaluation/sampler.py index 2fd2fc4e272a..2c6411f33510 100644 --- a/python/ray/rllib/evaluation/sampler.py +++ b/python/ray/rllib/evaluation/sampler.py @@ -436,15 +436,13 @@ def _do_policy_eval(tf_sess, to_eval, policies, active_episodes): builder, [t.obs for t in eval_data], rnn_in_cols, prev_action_batch=[t.prev_action for t in eval_data], - prev_reward_batch=[t.prev_reward for t in eval_data], - is_training=True) + prev_reward_batch=[t.prev_reward for t in eval_data]) else: eval_results[policy_id] = policy.compute_actions( [t.obs for t in eval_data], rnn_in_cols, prev_action_batch=[t.prev_action for t in eval_data], prev_reward_batch=[t.prev_reward for t in eval_data], - is_training=True, episodes=[active_episodes[t.env_id] for t in eval_data]) if builder: for k, v in pending_fetches.items(): diff --git a/python/ray/rllib/evaluation/tf_policy_graph.py b/python/ray/rllib/evaluation/tf_policy_graph.py index 40e540013fef..95e7a5d66bcb 100644 --- a/python/ray/rllib/evaluation/tf_policy_graph.py +++ b/python/ray/rllib/evaluation/tf_policy_graph.py @@ -30,7 +30,7 @@ class TFPolicyGraph(PolicyGraph): Examples: >>> policy = TFPolicyGraphSubclass( - sess, obs_input, action_sampler, loss, loss_inputs, is_training) + sess, obs_input, action_sampler, loss, loss_inputs) >>> print(policy.compute_actions([1, 0, 2])) (array([0, 1, 1]), [], {}) @@ -53,7 +53,8 @@ def __init__(self, prev_reward_input=None, seq_lens=None, max_seq_len=20, - batch_divisibility_req=1): + batch_divisibility_req=1, + update_ops=None): """Initialize the policy graph. Arguments: @@ -82,6 +83,9 @@ def __init__(self, batch_divisibility_req (int): pad all agent experiences batches to multiples of this value. This only has an effect if not using a LSTM model. + update_ops (list): override the batchnorm update ops to run when + applying gradients. Otherwise we run all update ops found in + the current variable scope. """ self.observation_space = observation_space @@ -94,7 +98,7 @@ def __init__(self, self._loss = loss self._loss_inputs = loss_inputs self._loss_input_dict = dict(self._loss_inputs) - self._is_training = tf.placeholder_with_default(True, ()) + self._is_training = self._get_is_training_placeholder() self._state_inputs = state_inputs or [] self._state_outputs = state_outputs or [] for i, ph in enumerate(self._state_inputs): @@ -108,14 +112,24 @@ def __init__(self, for (g, v) in self.gradients(self._optimizer) if g is not None] self._grads = [g for (g, v) in self._grads_and_vars] - # specify global_step for TD3 which needs to count the num updates - self._apply_op = self._optimizer.apply_gradients( - self._grads_and_vars, - global_step=tf.train.get_or_create_global_step()) - self._variables = ray.experimental.TensorFlowVariables( self._loss, self._sess) + # gather update ops for any batch norm layers + if update_ops: + self._update_ops = update_ops + else: + self._update_ops = tf.get_collection( + tf.GraphKeys.UPDATE_OPS, scope=tf.get_variable_scope().name) + if self._update_ops: + logger.debug("Update ops to run on apply gradient: {}".format( + self._update_ops)) + with tf.control_dependencies(self._update_ops): + # specify global_step for TD3 which needs to count the num updates + self._apply_op = self._optimizer.apply_gradients( + self._grads_and_vars, + global_step=tf.train.get_or_create_global_step()) + if len(self._state_inputs) != len(self._state_outputs): raise ValueError( "Number of state input and output tensors must match, got: " @@ -138,7 +152,6 @@ def build_compute_actions(self, state_batches=None, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): state_batches = state_batches or [] assert len(self._state_inputs) == len(state_batches), \ @@ -151,7 +164,7 @@ def build_compute_actions(self, builder.add_feed_dict({self._prev_action_input: prev_action_batch}) if self._prev_reward_input is not None and prev_reward_batch: builder.add_feed_dict({self._prev_reward_input: prev_reward_batch}) - builder.add_feed_dict({self._is_training: is_training}) + builder.add_feed_dict({self._is_training: False}) builder.add_feed_dict(dict(zip(self._state_inputs, state_batches))) fetches = builder.add_fetches([self._sampler] + self._state_outputs + [self.extra_compute_action_fetches()]) @@ -162,12 +175,11 @@ def compute_actions(self, state_batches=None, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): builder = TFRunBuilder(self._sess, "compute_actions") fetches = self.build_compute_actions(builder, obs_batch, state_batches, prev_action_batch, - prev_reward_batch, is_training) + prev_reward_batch) return builder.get(fetches) def _get_loss_inputs_dict(self, batch): @@ -287,6 +299,15 @@ def gradients(self, optimizer): def loss_inputs(self): return self._loss_inputs + def _get_is_training_placeholder(self): + """Get the placeholder for _is_training, i.e., for batch norm layers. + + This can be called safely before __init__ has run. + """ + if not hasattr(self, "_is_training"): + self._is_training = tf.placeholder_with_default(False, ()) + return self._is_training + class LearningRateSchedule(object): """Mixin for TFPolicyGraph that adds a learning rate schedule.""" diff --git a/python/ray/rllib/evaluation/torch_policy_graph.py b/python/ray/rllib/evaluation/torch_policy_graph.py index cb990c36f8bf..a762927bab44 100644 --- a/python/ray/rllib/evaluation/torch_policy_graph.py +++ b/python/ray/rllib/evaluation/torch_policy_graph.py @@ -72,7 +72,6 @@ def compute_actions(self, state_batches=None, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): if state_batches: raise NotImplementedError("Torch RNN support") diff --git a/python/ray/rllib/examples/batch_norm_model.py b/python/ray/rllib/examples/batch_norm_model.py new file mode 100644 index 000000000000..abd4b53666a2 --- /dev/null +++ b/python/ray/rllib/examples/batch_norm_model.py @@ -0,0 +1,64 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function +"""Example of using a custom model with batch norm.""" + +import argparse + +import tensorflow as tf +import tensorflow.contrib.slim as slim + +import ray +from ray.rllib.models import Model, ModelCatalog +from ray.rllib.models.misc import normc_initializer +from ray.tune import run_experiments + +parser = argparse.ArgumentParser() +parser.add_argument("--num-iters", type=int, default=200) +parser.add_argument("--run", type=str, default="PPO") + + +class BatchNormModel(Model): + def _build_layers_v2(self, input_dict, num_outputs, options): + last_layer = input_dict["obs"] + hiddens = [256, 256] + for i, size in enumerate(hiddens): + label = "fc{}".format(i) + last_layer = slim.fully_connected( + last_layer, + size, + weights_initializer=normc_initializer(1.0), + activation_fn=tf.nn.tanh, + scope=label) + # Add a batch norm layer + last_layer = tf.layers.batch_normalization( + last_layer, training=input_dict["is_training"]) + output = slim.fully_connected( + last_layer, + num_outputs, + weights_initializer=normc_initializer(0.01), + activation_fn=None, + scope="fc_out") + return output, last_layer + + +if __name__ == "__main__": + args = parser.parse_args() + ray.init() + + ModelCatalog.register_custom_model("bn_model", BatchNormModel) + run_experiments({ + "batch_norm_demo": { + "run": args.run, + "env": "Pendulum-v0" if args.run == "DDPG" else "CartPole-v0", + "stop": { + "training_iteration": args.num_iters + }, + "config": { + "model": { + "custom_model": "bn_model", + }, + "num_workers": 0, + }, + }, + }) diff --git a/python/ray/rllib/models/model.py b/python/ray/rllib/models/model.py index d5147168c2fb..561b636dc863 100644 --- a/python/ray/rllib/models/model.py +++ b/python/ray/rllib/models/model.py @@ -23,7 +23,7 @@ class Model(object): Attributes: input_dict (dict): Dictionary of input tensors, including "obs", - "prev_action", "prev_reward". + "prev_action", "prev_reward", "is_training". outputs (Tensor): The output vector of this model, of shape [BATCH_SIZE, num_outputs]. last_layer (Tensor): The feature layer right before the model output, @@ -108,7 +108,7 @@ def _build_layers_v2(self, input_dict, num_outputs, options): Arguments: input_dict (dict): Dictionary of input tensors, including "obs", - "prev_action", "prev_reward". + "prev_action", "prev_reward", "is_training". num_outputs (int): Output tensor must be of size [BATCH_SIZE, num_outputs]. options (dict): Model options. @@ -124,6 +124,7 @@ def _build_layers_v2(self, input_dict, num_outputs, options): >>> print(input_dict) {'prev_actions': , 'prev_rewards': , + 'is_training': , 'obs': OrderedDict([ ('sensors', OrderedDict([ ('front_cam', [ diff --git a/python/ray/rllib/optimizers/multi_gpu_impl.py b/python/ray/rllib/optimizers/multi_gpu_impl.py index 1affe8df395e..c548b20cc022 100644 --- a/python/ray/rllib/optimizers/multi_gpu_impl.py +++ b/python/ray/rllib/optimizers/multi_gpu_impl.py @@ -3,12 +3,15 @@ from __future__ import print_function from collections import namedtuple +import logging import tensorflow as tf # Variable scope in which created variables will be placed under TOWER_SCOPE_NAME = "tower" +logger = logging.getLogger(__name__) + class LocalSyncParallelOptimizer(object): """Optimizer that runs in parallel across multiple local devices. @@ -63,6 +66,8 @@ def __init__(self, # First initialize the shared loss network with tf.name_scope(TOWER_SCOPE_NAME): self._shared_loss = build_graph(self.loss_inputs) + shared_ops = tf.get_collection( + tf.GraphKeys.UPDATE_OPS, scope=tf.get_variable_scope().name) # Then setup the per-device loss graphs that use the shared weights self._batch_index = tf.placeholder(tf.int32, name="batch_index") @@ -95,7 +100,20 @@ def __init__(self, clipped, _ = tf.clip_by_global_norm(clipped, grad_norm_clipping) for i, (grad, var) in enumerate(avg): avg[i] = (clipped[i], var) - self._train_op = self.optimizer.apply_gradients(avg) + + # gather update ops for any batch norm layers. TODO(ekl) here we will + # use all the ops found which won't work for DQN / DDPG, but those + # aren't supported with multi-gpu right now anyways. + self._update_ops = tf.get_collection( + tf.GraphKeys.UPDATE_OPS, scope=tf.get_variable_scope().name) + for op in shared_ops: + self._update_ops.remove(op) # only care about tower update ops + if self._update_ops: + logger.debug("Update ops to run on apply gradient: {}".format( + self._update_ops)) + + with tf.control_dependencies(self._update_ops): + self._train_op = self.optimizer.apply_gradients(avg) def load_data(self, sess, inputs, state_inputs): """Bulk loads the specified inputs into device memory. diff --git a/python/ray/rllib/test/test_multi_agent_env.py b/python/ray/rllib/test/test_multi_agent_env.py index 5b4099b3c71f..5712390c05c6 100644 --- a/python/ray/rllib/test/test_multi_agent_env.py +++ b/python/ray/rllib/test/test_multi_agent_env.py @@ -323,7 +323,6 @@ def compute_actions(self, state_batches, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): return [0] * len(obs_batch), [[h] * len(obs_batch)], {} @@ -348,7 +347,6 @@ def compute_actions(self, state_batches, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): # Pretend we did a model-based rollout and want to return # the extra trajectory. diff --git a/python/ray/rllib/test/test_policy_evaluator.py b/python/ray/rllib/test/test_policy_evaluator.py index 7b4d6c8b5ae0..cf319a7e922b 100644 --- a/python/ray/rllib/test/test_policy_evaluator.py +++ b/python/ray/rllib/test/test_policy_evaluator.py @@ -25,7 +25,6 @@ def compute_actions(self, state_batches, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): return [0] * len(obs_batch), [], {} @@ -43,7 +42,6 @@ def compute_actions(self, state_batches, prev_action_batch=None, prev_reward_batch=None, - is_training=False, episodes=None): raise Exception("intentional error") diff --git a/test/jenkins_tests/run_multi_node_tests.sh b/test/jenkins_tests/run_multi_node_tests.sh index 86fd98af21d0..9b8d9295eae3 100755 --- a/test/jenkins_tests/run_multi_node_tests.sh +++ b/test/jenkins_tests/run_multi_node_tests.sh @@ -269,6 +269,18 @@ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/rllib/test/test_lstm.py +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/batch_norm_model.py --num-iters=1 --run=PPO + +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/batch_norm_model.py --num-iters=1 --run=PG + +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/batch_norm_model.py --num-iters=1 --run=DQN + +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + python /ray/python/ray/rllib/examples/batch_norm_model.py --num-iters=1 --run=DDPG + docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/rllib/test/test_multi_agent_env.py From 447604a9fe6a3cac68f9bccfe3f090466c9559c5 Mon Sep 17 00:00:00 2001 From: Stephanie Wang Date: Thu, 29 Nov 2018 22:31:04 -0800 Subject: [PATCH 26/40] Use actor ID for the dummy object (#3437) --- .../src/main/java/org/ray/runtime/raylet/RayletClientImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java index 9cf70c348209..0cf7e3d0df7b 100644 --- a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java +++ b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java @@ -172,7 +172,7 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { final int parentTaskIdOffset = fbb.createString(task.parentTaskId.toByteBuffer()); final int parentCounter = task.parentCounter; final int actorCreateIdOffset = fbb.createString(task.actorCreationId.toByteBuffer()); - final int actorCreateDummyIdOffset = fbb.createString(UniqueId.NIL.toByteBuffer()); + final int actorCreateDummyIdOffset = fbb.createString(task.actorId.toByteBuffer()); final int actorIdOffset = fbb.createString(task.actorId.toByteBuffer()); final int actorHandleIdOffset = fbb.createString(task.actorHandleId.toByteBuffer()); final int actorCounter = task.actorCounter; From 454d3aa07d59e84b31f3ca6cefc0936340a329db Mon Sep 17 00:00:00 2001 From: GiliR4t1qbit Date: Fri, 30 Nov 2018 16:39:40 -0800 Subject: [PATCH 27/40] [docs] Snippet did not have a code-block tag above it (#3442) --- doc/source/autoscaling.rst | 2 ++ 1 file changed, 2 insertions(+) diff --git a/doc/source/autoscaling.rst b/doc/source/autoscaling.rst index aca1375753a9..90c8e92f3d27 100644 --- a/doc/source/autoscaling.rst +++ b/doc/source/autoscaling.rst @@ -78,6 +78,8 @@ You can use ``ray exec`` to conveniently run commands on clusters. Note that scr You can also use ``ray submit`` to execute Python scripts on clusters. This will ``rsync`` the designated file onto the cluster and execute it with the given arguments. +.. code-block:: bash + # Run a Python script in a detached tmux session $ ray submit cluster.yaml --tmux --start --stop tune_experiment.py From 57512616e1369f66caada728f0cb6626d77bef5e Mon Sep 17 00:00:00 2001 From: Devin Petersohn Date: Fri, 30 Nov 2018 18:28:35 -0800 Subject: [PATCH 28/40] Update readme to contain logo (#3443) * Adding logo to readme * Updating link * Add badge * Addressing comments * Moving logo * Change align * Move image --- README.rst | 10 +++++++--- doc/source/images/ray_logo.png | Bin 0 -> 152298 bytes 2 files changed, 7 insertions(+), 3 deletions(-) create mode 100644 doc/source/images/ray_logo.png diff --git a/README.rst b/README.rst index 3a8855b2439e..c6c0d024174e 100644 --- a/README.rst +++ b/README.rst @@ -1,5 +1,6 @@ -Ray -=== +.. raw:: html + + .. image:: https://travis-ci.com/ray-project/ray.svg?branch=master :target: https://travis-ci.com/ray-project/ray @@ -7,9 +8,12 @@ Ray .. image:: https://readthedocs.org/projects/ray/badge/?version=latest :target: http://ray.readthedocs.io/en/latest/?badge=latest +.. image:: https://img.shields.io/badge/pypi-0.5.3-blue.svg + :target: https://pypi.org/project/ray/ + | -Ray is a flexible, high-performance distributed execution framework. +**Ray is a flexible, high-performance distributed execution framework.** Ray is easy to install: ``pip install ray`` diff --git a/doc/source/images/ray_logo.png b/doc/source/images/ray_logo.png new file mode 100644 index 0000000000000000000000000000000000000000..05840a7ff453e984264be0ee7d27a6df8a251f39 GIT binary patch literal 152298 zcmeFaby$?!_dbjxC}Gfu1xiW_A`PPmqO>$fNlB-4je-SYAf1YIcf*K+(kRj?DcvOl z{PuW!M$Yl@!Sj9pdasK=&UGFcW+m&+D>tyP2p(f$ z;Z)%s06%fX&Ugm?hi!C8`Vtmarl0G&yg&FiwVuQcX)LS9 z-Qswhkf1N#dfnB@X=ZqMp8MjkIMl`x%Pvj7h10V0P&gNT#6$V?9@-~?Dl_;tCL4cmOZIb$Vi^|JNA33` zW-|Pe@7K=C)R4Y;JjHc3u8Q_CrTC)|_Um%*ZEpE#48=ydk$1?~P3oW4cs~%X&{1u6 ze&wN~%l&6Gx87fUk3)G=Chno7k1E%-;5xE74Qn&b2O&qW9@FL#@TNEtg*^;@fAokQ z(VPvI)tdr#-Vp+Ft1RpR(ZV-x3>ONm=U_!_l20v6%RTVVcNAwJj3SOdlXoQwXJb9h>e1@+h9~PMM~RO~8R4x_`j`%w;x&0<(e5Mr zxSuR~pB>^1$9`obIWLSS1Gb9LAw1FwBY$|X+Vd#qCARiM_;8|joaMXM<@S$XqPe|~ z0e+D8Fq>y>KYszNHjyXbGI=w-QcocIPR@^`>6vzUf0NK{($Kr^-o5W|rS2wXaHbqS@$OJN z-tzr>U%9PLh94M3uFc96z|!tLeLFRav&!#!@H8UoT{bTKA?{&$(L+QB2JWt&A(CLJ zxO$P=TY~S#gB)s233eL-%1d$29T*QizWX}nkbEAatRn7X@9U!GaIzRC`K$6%38%lX zeql5@u!JWq&iv|d)5Y8Di%fPW`AIZA*{`WzLEQoegum-Z{&+rV~4dAd(*=NacN6SF7MjnNh=b6(B5Isazp zaJBKdl9zceeMl8d?kNU*IhUxLsAl{)Cow0=F3T>{&VNy9>>6c?Z=P@1Aq{%%k&HUA zkn{MLKSdYFslH@=>0Cos!%HUG^2%ABBth(pWx~c6N;S$3gRECsYFSEIM*3H8<4T8( zh)KQGHvW&edcLNp!U;(Y?PAQKogeob^FU!d&^`^VZ zJE?oe4J5pXz7c1mY;Zl`wMlx7603qx{C8hANr%@ApP0W1EmCsRam#YQQcqPc?<^_u zbAIR?>74wX=J4GEqBP1eQZX8O;tER5DqJerY71rLT0XDH(sN~#RZL|xt_n!GgxNn> ze&ZMJ7bN2+lieVynU!|WBGe*c+FZ!@v?#m$jAyU3n`+->5v}s;Tle>w=@=uqd3884 zNK3yspR+XX`K&AY;+4PdeckM@2}JcDKQJ~Xy!+;z8E5S^XECTXC`(UHe~o^L{-~6J zluGbOh<(VWNt|UBXEJB8xy(fep1#|?#+x=FZ#f&X?$u7K%dxn!=Be3fxaQ*_InkS+!G7vq2v`esc97y>3(ca7|-0HLKbu0dn#fpyiAv4`0f}^(v zvwDY1mIzILi25NvI|?nSS0;a za%8g?nl5ngWbiyBCMEkqx=7#7S3tXZ>7m6#jyD2|oEJEA&fBroai?4`(+`+ST1rlP z!0cuFPJkwZ=oQgnqU_`z+_cPXValT<0Q*^EPF zrk>UaBcR1!#d+6dmfvD_)qK=qrcbv_Z_7{Cd$Ph*j9zS5jOwvkWm_5NM$$mCcSNSZ zx+II_k?D%bfvMx2QyJ^G6YY#;#@bK!2~8O+W>sXhxx_4Mv1%)8Wk1c3EwFr35&zn@ zd}LE|T|}`;r8m(g%_LPhsaIae!P8A=HhS=F;PK(m*P$y93LZ?iI)BM)Gj0o@syf-g zi^>tsDO)kEYI|}R|FGC$s;9Glz9$b0b~fK`X>Wh?Me*|?9)hrVKe5xrYfmP4B3SRU z=BDPfz8>r^m|e}bj440G|JCjFI_;o-t%G`>QoejAe%Gtz$>k8gRlnyAvkeFZW|=z< z9rn3ymWvzDn@!@A8$L#!W~ZDm@XVxxe;c z!x8>NS_Q+-6Z7WjU2QWdC0rb&oOTYpHjYDV>(fJ}Z(P`CjJl&dGRDq)*ZoS9cy;(` z!0EP)gcZGshr=I38?VDK@x%G0h`N142ekgvJB#GpSzoCD8e0RKsu*QZN z%4vLIZR~DTtVf#rtM!3m?UNp3VQSo5n=iI(zP1gh%%}U6Xe}4Hw0bPhg$?m6*m_$I z_nmh8;Mle!gWgU3&usfvZ2Je&)jm1NAGSn*m|=fOUA8l;(cRuXEqSXgW{ z=>PUP#){a3AD*!jQ?(PdG0?R$wzQyDG&VQD;^a8b%YXhnKL-yr2Os}=PJT|FYgUnm zv9Oj-U%PTi(NSx%=U{<>lH0}ELOVj^aYm+x2gI;5Y3{e*y1;a8^7X*OOoww-FBsbo zA0Uv=6r=HL&wL&Ax?RHzcOTg&&-R02EE1<0RxC<*Ulww$gu5&YyQD7_H)cl)7|i5u zZuO~;)_FV4L>7L!ol|#k9}F9pkQ(lZ^`{RnX)-SHrGE4J<1av?AK!P$di2lV^Yd%o zc({)#@?LeG`41wZzv>xX!1W)$f`zES#;zpjlAD4={!?V&a@~~w1w-`sa9!Bl0dMmJ z`#*gN1RXWw^r8PMU^K_ROLG+$6$Sp|WD^pSIvn_~0?Gw@Mmy@BNILajO$+z2!2bU# zAfpsqt|a4d%aK2|UkEzv%838^6s||Fd(pqw0@;oRy4>Asfow1O_gWy^(b#K&{4?C{wLtb-Apf+9zcK1w3uLba z@|%lnOUm|IAbTy4zY)ei;Mi+{?6p9COTo5BoV^yvb~Mn;y4M2P7GL&SApam#kdExN zK(z=7Ra_}`&X{Iw+m!@(ZAOM*^b6u3*?{Sc5fHR_J+{jE|A~C z?OqFHuLbg(i)^!!|9`YVY-2t>gl*%#{ba!!g(mM_maU=Md82qjsidz`G(Pp2Un?>6 zRc6r^YEll6{%5o2w^Ejnn&ZA_^y)EHHabie(&P=V=<70wh~P|%%G9e`rmP+PkUQ_x z(J3{;4Eor@99b+NcKZw)L&=maoy8>i;cQjV-|H* zzvso_`FYR}Do}2*znY6mojFi0`zKQ_1Wt~Qsf9j*^oiy~c)ha>44jFYdrFyK-yY3* zw=z$l#+yO^WZ2wWud4Fw_7j>{|72t^rK|cD(`O+x5)^fWPi{WT*+TkiG8?Fxbcxhp z&onybH?7_{*YgLmsuqLg9{)?2SL+DD@E%x3NuG|J%kb5Jg(V@+g-k_I&Pd!w)g?BV zaqbGMC%h8iN@iKsiU(N_Ik^O{>mX`c#6Xa9Fp~1boQYEZi@MDa->n8-`d4!V>YDXK z7!%#`0a1Yi4jT9AOkw8Bm*JV`>XPjTz0s=+wANHT1SvHyEbAQ=V1w}v_zN7k+N6h44A`-a|3{s?YP_Hj3rK$eGkPpC+r*3IHhRjqa54Nyx z?m^sRkJ^GUv*gV0#5gjI8|?MG*L7SUy|7@|Rr$xH;P|bL2DcalsFf1}`~B$r!n%G5 zuO1J-7+!dw#gS@9S=eCAYyRd#+t?NZ*54m=@PLexNLyy;SK(VY#Ke8fN@ZsU)7Q0A z@q?&mr@CFNDh8r7Do$GbBH0~fC8Yu9k?WLAFo*H`=fRx!QHyM?__}e0GCW>S5bYN= zUI@YS<&izU^2;?KoqiiRK^YvU?o4gt{V8L#6i}K+)~K<@(NKIrk7^0;W^kZ($lMF; zsx4LRb$TfXtW~U zBmOBkKAD6B>bdb$0A&|>=j*&Un%NIh3*o*rB*)72`4vLFAp?V6{yP9o+JG`Qr=QgP zGVe2J?_TKI3Yc4o#`Qfj>wR-Q!K>2aLY2_ASo-@&KYwyaM(5sJr4${=4G`mKyWzoQ zFvNYN;&Hs{K!L(E?-`?S@_Kz^2cTcV;QiYJ(i2-y-W47etN8PG;ZrcY6$`R^G+fKc zJ(sr>h*otBoune!+?N#%x$u7b{%;6GUj>FO)OjctVo6*UmS@ja%@>MiYT83c5a;kw zw6;@08LJfWyzz+`RCa#g0$digM$U-ASjP>9_eg}b4Ry7OexkT_h9nrSb7z{5cCZgU zEqB*@E@OUm$A@nbAcdI8yK)I*6hU~j)MZNUs1e_T3TK)S$PZanG{od0#rJYq{=ijk zD*;zg(()&PEMAHUDE-8C<^h*|gx!%9qF{LcVz+IdSEX!dzg9j5o1Gt=!POd;O!BxG z#1T#iulF+{RGYH*pA{&4%0(W|?pLunK-Iof(fS8jlavCIq%uCym<_TC1RViwvI+4_$}Lxyp6LLX-?I1cYPxa#|~FZq=sv zO9vKCZ%8OrM8@jc{uIq>4AMJ2!2cOw-H3qqnIsOEQm#x(N{>@p@gdlUKCDQqqVg4Y zwinFsqC_AF*2pCK|FluL1o-8F+UCwo%6#S=G4p^kR@XZVj4B31SYu9U{tq>OeiTc9 z`#7lV-FFDD0oNvqi%9Q9R98q{T-u6 z4XK2LYn$Ej@S!tvocRUs(?TuHR&O;-r%Ia1$YONb=>e_=yv|@M-U1r$9-%~&{ZrEK zg}xI)oKWV1tQm=gL!44-P-Ag;^8xnCbFMfNtOTF}cODxs8T5lbgtMmEw^zpO?ZlY_-2WW@l(nLoIFW>E~#0lUDf zBM_;+0+gjKlFEH%E&|_NUIY6$Lff$WyzU&;z#sV8;w3QJ(c)-`&f)0MEQ4ChUS^uU zVm0wW4uf$KviMnPDBuPsCX6Lk*yC~3r$@grmrwnadntB2P96MC5RerK)g zZU9mV{N4&B6EB%(vvQU_ZMvE%5qI#rY0B5t#s5Sq{JJONF=}a+<`}-Kz@Dca_K=aZ zWKCh6A`HILAN}d^aC+#F_{tx6L?9D2W0~Tm6%4_n#DM#kH^>TyW#W;;ume;5-U{RS zc)Y-YOdNvucMga)1vx=lWhzt{F)D(gCSr3f4R6K++ZEFxM)Zxu2?68%O3PCWS@@0_ z0Vu^2kq7*HzX?OC?gQpZOWP}W3AI6adB6_`!HjPf7Uj;7{yZ7+2i9wM4tRfTUM^%y z!}ukfsZ&Z8F8nkZw1^(tl@oy%?uGq98KU=N63$y%^Iy*M| zkEg0`eqYiWhW!$Lr|CtKg7hrj^EN)@RjI%aX2N^RBdnLy4=+mV96~JEPH6!K=KZr= zN{Tk)?G=Z@U*)Vb+7s|9iaEXBCyN}$B0ra*mgeH1C7_})wDc#f2`Rvv!sjDSK-jbp zlrX~bQ?>SCA?C0xLTk18dRfyT%l;o$=gZE)bNoqFdF?3-7D@y|k3KpaK^k$o4(hfr zTv{N(zpzELEE_{s`!r(KJs(p&?mTQOGq7Q;vlN)$xTysUFLi!_gvjKo4PAudMsey1@RiIQ)ut=)&P z^BP0h!`>DJ?k*KR#q5>23LaG#h&GLJfs(}rktCcKBdc6TsNwziu%IC$7oe#2;2&hL z!GWL}&!Me{ZiM7KRMSRqZQTWTbQeUo- z53=)XDPBHd(W}lPSwYfrs8C7)!^xM{_k6<)C(myc;sve;Mltu-syHE(^j<`+SfJ5#MkTe%Q|LIe5 zY4E$q;y1&^`58J5)h%38F*6+vz6YkW=AMdkG z=#!9}h#L$0A8z^MqeW?u7Fi0sJ`II~qu9h)?v^#PR|dx>D?>s?Dk-nht-5@*i<@j? z;bX7oAr-tZ_{=wAU8zSWc}=mQR_K66X~piqg`k0RB_M^E=UWi`a(TXl7=wP@QgYT? zrEDg$a>l`WJd_qk!8ik>qcF$j!|v}Dz7x_M-Wq9)R`KBx?3e!_>`I-@cuUJuh>7>n zw-KXp0%zu`^9!6zR=-m6T>H3^o9p%dMB1XWb(O}#J5A2ej8!Lpt|Bg2JxQkPQGr_4t5+x@BKYeGe1@)p#?ibF3@mQr`uRb3R2rIP#=d>@bqQ!u zi|?6?SXc~@QOP;BV>E#B962vfEIFdC1U$&5zhFlOgt$K86&~d)9G;_eAS2ekN8!W8 zIUS+h!9JKWr`AJ+7?t}UE(v`kb+-S}fPVr8A-QPKFprXzG(ADqt`)}0ElcEa1@_4( zukd5jXv12OjI*PpILXPCOh<$~TMy0X#8745!1dkZk!QRZ|F5eDO|g&E9QA%r;c=F& zA-1t$RpZRuQcUHypu%w~&d5;fx8G;1DocY(F_I9yd;`73+C!58Eg!h6JSp;sPKYr#xg#+#C>NmiWBbDvT>*flU|x|_B~I5EkO z8`DRFtQ`U$V^l0^ppg(HbsI|9kQq4hcB3O<2R&Y9HfIQoz4?CBAYg;oZc~9FnXF8r zp*PNWj`fN0*HoF9(rnJ^BYZ9tQhJ=WnDKUe7$$Ih?YQQWA_^JOckJl^0(0#zPs>Jp@q+&S7Ee&uip)WK zmVm@Wn0oNVbP$_MWw0M-WDwiYhGPLeQ!k1a)HOLlw{b^t{~8J|1hb8K&LS?;Mr=0#=^}rbJGpxDSgjcB0GB?-@7^X{9cOsr=LphY@YB@bOy#( zhy#(~2I0r%u(0@xQK8m@?uuWEnY04?=gXJVlJN0sI$RAq7fDxgXO`F(Y6XVOv~IqY zZ#uj4;6G_R0~?sWHY`#DddGf>0?}~_V(oSM&7fzilQIsI+L0~V=Zo>3OXu(0=n-<( zd^NBe+E@rJ^eWn|6Qod&F8C4hone;A76oXg3MXzEkZbEMD9)`=^bPADcAkYL#QvcJ$5q`e-(d8tj%3TZvz*354V=eOsw;(km@VdE~L8IV#G2VkdZHba>u7Y0g zEQ-YTUmFK9^J;@N+1Va< ztK((N#Yv>as9kIf>76C2#uKGf8)&uO4Nu0(K;rCVH*jVUX`;-wD=1B`K%3`5 zdmxncE?()Ua=vkus-}QZOi)lZML7LvWrD6QV)9dEy_Ny$QL{F7N#H4{@A6Y!48*@4 zbS{DWj33pU_LLN@Ak9~|4JVFGSBg2P8rr7IX!NmU=Vlgno^hDG zxU6aM8)yG!i%H-z-}GBy+}D1-a@7afqlOaeKj(CLX-)7x-7}$LUNR9ry$dNd8p*Au zMcnEnn9j#g5ijg zqxANaYDa;;qgxS;Zt`S5F^FbG6z6fzd5)BF*TPK_$3b#7Z) zf>8Ty*7|FJ`8HrfBN8LLt~p#wZcBXAb0>GLRq*-u2Pf~dJ`+MY@jKRM6bD~l(l89v zkTsz9-$C(hFA_svP@NkQ3T1mK3jH#!fe}>qf~*+D`$axi&-jcP&WleAuBA@zUNc@N z1NF}xqa|q7u^1Ec8a6hd6ep@YykRInTy=&no8>zwFcP`(12 zk@YU0aSNCtQ(Z7E5mMsX)f4xjJ<)ODrOZz}Reh1&HA-PaMG+LX=C9gL@_;V;Sa<7UP?pLqdp5rd+C?m79X~M+Z*G8ionJoIJg8Wvwgu9W&mSl2PvR)>1;~!yV;AFau%`Acl;29%nNN zMNTm}>rb|#Wjet<;{+5`Z!9j`)mrWX1&=`Y^aBM?0R>Y}ZL*brFArGOGVcQ2z($D$ zkyO&?);8zZu2f9-d#B0BumF*Z0a~z<%Rn1SeVa=gb3x^u+~r)a`zk7(E>q?ng*QB> z=0nTXV@pkUEzAiEUVw?E!Az`%j#Fn>&LmKHIyqoDb;_XDq<8()NvvH>%@YpxG)$j_ zrly9xK&E3VHM@95vD;-TwXI}@yL`k{Waay=bEty8yPg`FCIf<-#goge9X;0o{4M}~ zxBB$8&c@QsUEwDKiODJ4pK`-HV7d3jJ)+EoVkT2tdM?8N_#R<>>&CH{bIV4Udo0X% zbyRP#^(O+eNYnxA3h3#*+q1!_`&AwQz1|3mSHhQ{FfIE$)p~KK8HCJVRTb=48-qCT zC2NLx#}sRO4<#CLZdh(4!u%vuvhB>fzaicpH@QqcIlXv&y>v8O2DWuD#G8yqiwcR&>BTuipJBuv{LJ+#o++$Duqi$I?8VC}HW) zm+zqt%e&1R=tlp||FUpQ4nNr}nlx|eIg4!~v>!|tg*EN&abz4Qm2O&o=Vy`pg&{-D zja9pIj_?1gR^mUjqEPJ2&uhc|;(WlwtzKY#M>c^_Lk6JD5$dx5b374v<;{>k&8moE#mTK#8gl0=@s%~*D=r8sR(Ji3(&kP2#d6Al8sSpi&#v$Js67>i4B zii@3N-ArAcz*CL-&pyE9D?2|l0Z9Z7{u*Ud(cXo?-r))4P{_3-`~C+SKR=2Fdl?*# zKr?1Q5U%oLlzPl4RR0&}Ue)_S5%bU}fVEY27se_P{N6o?b{IAK!?U_HGKwRZrZl=t z(e8A2KnrJgC#-#=`RLw1uj1U#Aw0+yv!nsVMXg$Qb@P%itxqfeS?J z|Ha5gLH}@pewP2sEGhpWVoe8&;iM_1snYLx*(+fOzE%ZX27HrlSDd@+25EAzYU05E zYr?*qJYMS5p^;pC2eZ2-7E3wUfzdNU0M?MNs&y8>`4N(rF(EmMA8XfX!}_~>lCp4s+^Ql+WrF1X z3Y(*8Zk`1~ePy$|WW;023VIua!oLjux4#1Wnv2)vuaZwq;gn`EEC+o?ki= zy}Dxhe;O-X{f9_TPz`;G-5yG*qhTK{WQ%-}b`9o^a%h9XCP6LZ~9+V(Vv{k8Hito6uTnab=%F1-t_1s;faT?fZ zGOf5xU*%KgZ0*D1YY_~}7IHT0Pp{c(yzK5j>r6tbjr~FrGOT}bt{)%4r9ngbQ$n2k z64Y=?P^OmE)Cw3$S7JqG8I)g}5%5!H*;1%HP?;NPTWTHpT{kZaGA8|qUK2d%^}%JnME$#Q6`VR^vj z7I)T=x9lO?1*od~9R}zpQsCf>>-Zab$R!rl;c}?Pu57WYucf``DpMu<>DKDa2{oB! zDUa-+^)|NyO)5ac6c7y;VarGqCy$#ry|tiv$)K=B#n zYQ4p+-OLNF(**L5d%ePy5QA5d97|Pozc#(zMpnfF+@Dvc=GN^a-Uc zPHyf?W#NFfMI6ZxWa+MwgWa&;V8e831LP0{cqe^E*kJ-gSFg?<^F`L)7-PGan746R zq;r#483GUTd+1BQUUMJtg7B9J1)iVgsAGc|v`LO9OrLYnk5t^~ujcFN@P;pci&PsO z(@f!h#vST$s()~QqEg7OF@7KQ$4`)lKo(`>X>si*YDg9wapu4+;Rz4tK<-7WDIWTV zB??C;Zrs4m;%Tdn-?DJW9b$)R*PE`X2<|ZCcK#(m&)^AY2KjOP60@gpihJqHZKYt< zVYXvI@MZl$#l>DTKw^)KLLM1xTC4l>3Bh5mQO>lOSO+8GlXqYi%h zp9K&>@vjBIid6@>-L*iEEi=gs{@wf-Y{~`BRyOktvn!Vs7o3O_5omVL@=kmpJWR8I zJ#p_r@0D{)M(n5kwT>{xMx9%p8!ox;AzZm81+{7aMcSW=kTQZcWPh9qwBam60^z_m zU)0Iw;smQ%NlzUsTM82@xN0)*wUTS;+h);sB*l3*eBnny55cA2+;wP8!;W_LdCJ1N zU+eMDob_!VXC&m-{6v|G-l*$&Oe`*x`96^xH@#jO9_7@_6h zol&U+f3?rmaFHBCLznx@UF5|?d~V;ZiIT8d3*2ByFG;#pDp~Yhv5LEP!ouX#Del}5 zsolw*6r>JXi_0c{kmwc&!H`wv!hzy&a?*CT@`VoU;2C6sCgND%S!UNWdh7No#}_9> zWR^Z0F4~VFhMi|10vNV~xcQl%ZUeW(>22z&`7Ymn@!VhmcF0WYQ=6LPvw1GD$&H3A z?CHgc_KPE~wjah58*cA@;4E4d7~`#cWj)Ao#0XOA0!G=skMpy4bDDbyB1XvWS08EZj~VJiR3O3$uuNsC8Ea=+K6k!SDzXBIhJfev41USTTNWpqe<3FX%MoPV&unuA&CA#Z5o9?6Q{=L> zUOe?3Mf#m2VNbU+?7z-7qXyWhQQNEC18ohhBI7mdK5^3{5n0M5bB+4@HOjK8vhG(8 zKhVrK&w%4Yvdg!eLY}SaTTMe;vLiB}D9!xh4qJq>*@D-|gY(S_(O^fd(zHor=titp zj3v*fLpVL8_k=k8WF@Z}+>D8CEixNkWLfrOqsSrO6(l)7^g29K{D}yR6fA1gUwhrS zs*8gJ<$FjjXzQoft>z0J{vOXQKZIhDEq`DrD6ID`)YxZKyGJ{7@x!CE-4v>7iB~ras>S(ux?;=kfgNj^2GynZ zU%DvOjDo_$2n5s{IsbJQFvbHOe!CK_nNZ@}N?FM0z2@NBlR^05VRM1LG{mTl@)uGz z%4t#Vk)d;5zNy#8&8tc$qq66hDcf|McF$~(z_a)F3dTa2O((I$sso#?2lwEzd8+(a zNpiCT3DU!@%7_H}^x;pso37pGb47#)FHxR2n)&o}8>&2UDv*yVXt6*%7BZckIZy~J zkIbwcI0xAy2Q$J8F%zCE!jrtjth#b_Y%Y~@<~sH2;+e%Tu1wMKZk|DT%HkuhMi8y5 zPx(kb9LcdafCT*?Zvp+UnQC!HC^Klfiom`n^0H{U`9mjuQleARDi3FGkhK}CqHjjQ zl5f92e58xf1=%Bg0x?dVB~O>Jg@d@P+Lq&ZFegxUbRRh%kQK8Eq{>3c5H<`Cua7oA zB0O^rJHOa~6TxNE#lR^XHaF~b-{WG(1^=D^p>q3aGuT&$>H-yU64I4VPmtp*wqcWN zk9t^)$abBuXnnwvFC0`XA*MfvOUvu(7{}*`uMl#J6VYs|vG9IQfitvebA_`(B6zi= zI&OVIv!hLVe(V5cvEgK-^@4Agez)Vfg(8P*Fw43!Vclz71tB}Ch~dk>cZG0M6sQpc zsWxv{xbm*HoIuDkt%+4VRepAgctdoYr6cM9<+{E+Y zIpaa~2ZPb!Io|DmvDz*5vusEZkH*V5tu38@oQ~rJVk2u(eyz&)uu*o2((X;47vMN# z$(x=p1V8yHS`6W(V)|bwK;}MBH5x&b&x-I%NjAC&*7@c3Xli z9(TA;nL;b>7-#tHAy^dzP=+{L=qN1LEH#z0Uv&+->F*9Rvdhs#v^FeXboK6^p(JEWu_) z*=RRH(76V}pX7iohUvjFnO102c@RDT3LgMfV}5>DtPlMF%J;ewCV$|-{jteS_El74 zK@T;)GA+Ue0L8pMI<}cyYkZAiH?cgCZ-89>HHA>1gzH>Fcva>%TDr69U6sPdwPT`Y z={=WU33i+&z z?T(!W!hnx_3CjO?Y~$k)Ea1EtEz4#Ps>Y48!AR5-9uKP+h_KP{#e8FjheZXj0vNUa znhByRBJ>QNe=-?nZB2C8_%6~4+pF?HwhGg(l#mMILw<$H4U9y!*u>;p67HyPzC<9P z@{6)m(Fq@O!*LO7mh95m$e{YswTc7$l(e@b`NyYPOj*kg#PF8lxT_SxsnMKV?tm04< z`iD|1<*hOf_Tf0nlgItFmK+dI`&j=w3r+JpK z`fh&Aobm(-fJy=AC{wt*Kt<@hvbXy&=6yjV5~{9IsKruB4bT5F4hSxPliQRCjFXO< zzsi}0^AHQrxL2jXaNe%x8otQ@0}wD}#?XC=C@_GCg)XYbg7b|3!6Z*04i~9aff<5q zvXvFRhW4Ov;0R@ukr>+B)(vuF=If82urEs5l@72F%Af(HG0crQV~OMiUbjWx)<-j# z3JMr?a5xoMd71uY66lSXSO4Py;f{wsHS4bQ215n=U_>9ApG3WoO|NS|1}-Crz&jZc zu-wRFo-%DFMt}ih$B}O&y3aA!8dnp=X__)xKv)N za0+8s^)RQTIf~C)=A5C8!nlOr#C6J9Y!%Do_1;+oW&^hW1@S-h{OhA8bYgWPY2;Tx z4TV<>lz-HBsC87UqEiTKzk-dMA_-URUW1%uM>Y3T1fO^!zTfiF8s??BxI=g!Jc6|P zA~W4sg(m!wzBqmj-`7V^TvTo>I8uutk{!p?!*_hW8mz+xEmaT(N5CtRq8i3s^j&rA zl~hCEO0Y;4trmOJz#aG)8u~hVqxlPa1@@o9As%DT+s$2#>rpW>)0FR0r{iGWCrcg` zP;v(o#QzmBxF^yJcsEH!A=C_n>nsCfn|a36`_j6xLnQ_Vt0chnp(w9BjzRU`u|vE7 zF-Njfx)hR;iA+Mrbt}i+v5K3SuIVBo4TOr~7kRt(m{eaQ4@j2uO<{1yHe^HlQvzwG zX;l}eI2XIbKI|H@cJ|a$0hSHz8QtV}b|u3aLVy?NvOKivun)EX!^`MBFXAd)#Tu>1 zI=f-#ajmL8N%?lVlPtMqV{6^7xBokEI!x#+iA?<09^yqYYFUb{TPB6lRSH}U(~_Fw z-=zA(Q|zp&PE$Hm^A=_YOsIeKubk+#?Ac`}F1SkpT;bvw7N-wcJfaeBXl$TsxMtKq z@SL81WzTRD0{`aAqP}qJ+huC~Z9z-VxH1GqHl?a8_D?0x7`S9P_Blwn?V^NSD)5(F zjdZAD0Y_c%2xh`9w4cpAJG7M~qoyHpU0|lx(E>IQpPI_8JJ-r{Q?=yIEdpqY{~a38 z{h9*|_JPM1+SllbgyRq&>!pb;=NOFQ968W=KdTMbl~cqFHqdvI>T$;EcdtOydc}Yw z{RJqH?ik*J(Z?X{gz7PK&VhySx^Ef4d9=Df_C9hIYkAU!?xHN=mTHv`G2zyC-dF;k zr!m~0aw`T%GJl;6-)a0DutBlcFIf%aHwUPn_0QHTjBs(Cev$bJ>v~$@`xfKPVBdFO z@Z~pz6(9p-{VuFKC#F#NY!|P^%>n1Nd(+Z8whul#d-%qP0_SP}%z7|*;(OcXNexav z2{>vi?o&a=>IKp^z6TkO6Yt6Ya=RY_q3eQ&Y?)cvBYYc@AB3qI1uttgIjuA2 z=bM~5(}~#FG+h||#&A$M+XfGG4$|Ao8*%FU=w2?)#g-@^WllT;e4FDCq9k7UY=H3X zV=Zh+HQJ*|SJ$_2eX?wb_Q6hpjV%s`Ech^1yNOQ=FEL2!5*d6%S-h10nTbh&RkQ5e zU@G3jD4{7o@5*>q=YTb@N)3j*zo7djL+xR`(vp&=;=ru_2yTfT`fQbtvR-@n;+E!o zn6I`$O6V4s3$49X?Q}nUodUq?c%w^F1^nyXFZef;owlS{z@~s!|rausBK|8NBkDhxE%+ zWn4l=36LK|*MGsp9i$yL@!7~)DTfBfn^hZ}p<8PcF>Z*FYImx7N4KguyoXOaZ{AQX zk?xTkB`3FxX#Wd_=o6q9WD%Zv!wXJAzhw;0FK$^^jZxN@``U>w@OkJ8E?M4Xy)2FH z?-v% zQqI9|Uhf*mO+Fc(0=}K={c9d>YXjZg^q3WCny@Q9I7Hlo8|#~!%fhVHH#rN}s?>fo zapXeaWer6N>YJ`_&8hrF22ZP6d8j8VQm6OG#x1#!wr=ioDm^W)D%%N?y9Y3*qopo2 z{AmwjS!3Un*SmZ(3gSe#07qzw10)nyGy*La5i(woJ|=1FhBCD; zW{kxvzA`wJqe97moNW}_0M-Nm+q>1vP{X7pO4cmjC~(TutjciB2i4U#SN}Lf3x5k+ zUPIuzBPNb@Deq{MVq&d_>_uWCnE$sQu9`5=JvQK*og5v^N*F~KRL((r1bjqg^ zetQM)6$$gGEL@xEeX$wY!c(<0n{o~S%#7YhNc4+c=?NtY9G`aEZ~3kpcHg>t&ZH#P z=|=BW(7?2ETp*q6b;b3`FC(iMNF~wui+6HBW!xK?8iJQ{wV!8g!_vTYA*&H@I{ey? zFkD0Uu}zid8bf_KTjnLoQ>^Wk9wUwE@f5u=%S@ynO;-bq*O3+KTX3!Pql_sGQvgF6 zM2Um2$neR|m=}TSHBKSXYxH4Uf&9&Shf=WWr09--y!*$+gc$cm3ciWD0ssna)xZW=giI=Dw6nx#d3>;A=U| z{)S!W$I8w-5+L{0xN(Lf>GPS)7Lhkf^fpWQjV1+p;L#soPKPZc=4I*&?4Q<#zsd2a znOKVP@BW59>zyJ=Jrh>1@IEa55sxex`|u#0Q=_ZJx%zMSun;yNlGOHSL|_urIyklN zpmf%xwh>CR>J-;9%cO^6ie&EE=x5oFt9Dg$l-k#LQ7B5Yckd&qQL^mm;OxIL?X$M|n9tK1_@-+g#*Fey3>#Tf*a7++dWaIE(?P`3!h@V~g{})v zx{Dpl!|4+X!DdGE4q8^VgxA^h%#dWXtBw!!6v!6R#~rI)STLxjsBKMB^S;nBO& zyq$f!)u;#DQ8s=X>o=ntYH9HuxH&9Mf|{sW%6guei+%6A1g%}~)S8AyqS>>G^kbV! zy+4EYg%NYnGF1GzP4P9ZM5iXNqp(Wcq1=*7p~jnijtdT0bZ#dj+a}6ietD*aOBkFA z?`2<+#+ivUCsWXN6YFK}JHMnR4GLcbIhbz^I5!S1PuEmsi-aMD)&WZq1?^6b9JYxN;K#fE9f`lZwh=LKw4z&dm zN-9&~QTEKRs2}%|At>>OOp~RvQIwQzA-a^=s^7n^*(R)cTo%S9rF+)4sAc?(8SK~K zwzqaJG}p>zAphbI7&`blyq6)r=5t?d!L;x7+j<=%XVhIr-E6*84WVlQWDc;Mcvh2b zdvZu{X@k23orbZTax$%=MoGyn-5j<>s$lfpnRX}ZFGnAQN%zOE)lNRucV+Z=wejxsmp9nJ*c2I_ zrKfK4NtfpqRW0;CLUl9?3Ie_vz@9I1*9h$_&!`Gzh~+u+aeUJvI>YVD5kv^VkkI@? z(z%=v3eHkk=t`x)&_?E(N)VdCbO=F~ey^Qg?3W>Z9F$N_Gl}Fa6V&_bKHdWLDssyD z8T0T@*bQNyTrZsQtgTvGo1tE z(Q6bj!cB){eJ00_ob$m#3wuXt;r&;CPJ8yW!b)ztHdM$$VWs9c6YL$)M^l$|#V(Xj zxC)g_i%-(xHK+PAcX^efhrk$3i9W6mrY_V)9K)QZ;g7%N4B%2U?L1ChuR0 zP;(HVWop?LepExz`>#;l(mK?zdXfE`y}k~k>?P$Q&q4!0`wsCIZ)hT^T=rAcv729; znO9@hY)7Kvcp^e@7q3#M&v;zpW}lH2Q%D>Nugfyk%C9dMA4jh;>ER8zWjbXjEu^bY z@wGnLC@XLxY-4lMMv0ZzDxzypaI-^)|AE54CAUa*?D_XgN^8UA^}IN8%%G5OjUoQX zK4wgu3J>|!aEAN9iVdelxypjd>$b$JnmI!~KGpolex+3U`^RK>3Obgnqu7fNrJ3N6 zsD>kn3MUWx6x(sAnEPMX=Kis%0^WyxI_U0o{6{=x>_o&fUFQ{!?t}S&RWfbg7pB`1 z?F4nWYRS^nwgC7eq>!^(b=~a1j&Kj;=*y&BX>51=|@tiKtc~WS^JNh(hIoYV7EEW*|Z(V9? zcr-nF!W=aIQ41WC)QP!SDqD4)Qlvq9v7B~LT(7?W$G)wC66B?|>XR=Fo_>sCXd8k@ zUqN_PZfSq#&qMb5)lJwhz}7z#?}JU@os3M(H@EyAQit*_Y1HQM4`4+zG93uF>_ko5 zDni^ZG@#A*U5mgcU&ai>Py1jbcxF+>o7{srUS7h1)-#7ZK3pJ*4Fj2#JoQXS{rWdi z(@9XeTRzuzff2xP2rl%I-cWy*v1a&ol)S`9od&M+7`9e4$`&&HRUtg)}NxKe8UeVl!K3X!(0&D`ldZKG}$&1w|4+e`7tu3tKk-V)fC-)5Pu3_i?;jTIK8P!@3FgaUPjijdPqe*K~w7=i|<)?L~ z<=QB`wrRHjd|HlI@VobohFeEv9bKXHGtgE~*_2@scn|Fj4ariXFibB|#|VAJ6LsgNM;YyNY>%3utabUa%T!W^h>@ zjO|}%y}`Nlr~3F;ZV3GzN>X5s$p)#f-P9li2Oec_6mgnK#aat(^^Ydi z552zdH4Sf_xY@b5Wfq^bV{U_}&qvX`VU%hpJ`PmCjnLrIj|@R{adukO#eo-I;FrN6 ztCOz#|2kO=#4m%%xd)Cw0yCFLs zuni^!0HuRAM~v!{{FP6EWr>}J^XtD0p+EKQR-te8?vbQfKhBtqsbN%OUv@M&=2DM+3{HQ)~4oW~j1 zUwe!Mz~Fi3x0r?2Lpj{TTwn`ZoxEh?U0sIjmk7Scet0qYvja9Jva(mjaSeS31tmLq zd7P?a8LeTv8M(pS0~zfF3}OSK)j2m#+$%12B>e|E&UW2T{`#$$JjjH9vjzQqlU><}uLG@ncne@W-u50QY3 zKe$s{90S=yKH^YunWNe_|5JHBZsq?P-kCSTt=#p$E9XE2skCI{!=$RJ5O|P9K;L`# z-``=<29aC5T-uXc==DQnH2(VL$P3*GIzQ9t%q*u^It#Id_Ft?>bQ)yOc#cVT_-Oyu z`18E|_*$m4hRjR2=^Ydv5aEL5i8Bvqe>Zx=@U+=D z!0&1qG(1_Swc;Vozp{}fQ|jR-X~V4wiW~1HU-oqc2<&`y*!^&7@CHQRx5fg_esUmk zJT51E%A{v$VT_CL2GGOSJcYkTWN@MH5}2*1BK@7SeZ4+qd~53kA+NNQwKE5m5suq$ zKzy01ZGv`T2tBGH(ek4W{d>ZfP>e)uXdkaHe&!d{lay48I}TffTaD4uztNU6?AK_jZkn&xSWXhMhFrE@m(3_Z5O-d#?*Up zH~)rj+JcsY66Aw$B3n0W42FBl_sGU+3l*}J*S^fX0_e|dbyqAhoP>{Z+f_o41d#SY3GW5V zl%jZ_YZKnV<6CcVH>rcu51?|D`}fVkIDTZRiQ9si|3ekVxAuAoS!C|mP#GnhLFGYb z)5?4;4K%zbmnPim19#etW3e}&sjm$>Q$BxL=J#v6F)4jz93*>Z$v>qvjJ_TFNT9!N zrGDhhFMYrCVhwIoJqvh2eA$crzn~A0co+_Jt8{(?-GtZJx3~6E2t|ai@!*QAwkA`l zx29~I1gwtlrgHy+>odn>j~=r{9{Z=a*CQ;_mbGnNMp(4-uyRWVu9d2bf5ad} zi6%G(Nbu)0_$lbhP~fz#@H-iNP5AZhkB{U^e*K5NLH7*99udO9sfMX(sU_vmoz8A+ti7J;KJgbBZ!I3=RuE#0xCjq>$2ss#fHxq+s^i3YBUGi@#oaR9BH86 z3Kl>{iSkwd!HN;!2-t0K1dMOm8T3mK@Bfw!8-h=&vh1S%>2?O*4%!2-I*_!+nPgVf zY*p0ue6gQsO*1ss+h;BjML}sgJB#D@HMn*@gIdqBG#=!L`qQGvNB!OXXFXRRU8Fud z9$8Vy0`)#WX6=zAKLKs3f4RRM?CX=}qy`V^yR;jxXSe|tk^X(dGpY=qF?M?*zU1TE zyH&YTfPx`hyUq_3_0KG?`mx_glXnen{P--WtuXDLgS}yX}T+;O zyX>kW8KQ*OV#c(pYtVLM$5EiN8JJHFj(^UM4@&(bBM61#j3DVhMnE*ilOHVp;Ys^F z<44WF0>CA!J+mBH!a``-)VQ6s$aDI|x`a1)W3nfL#KaoBrU~~0XY@9V`Pb~4j|^`z zfEHyfcf1qkojQSMf5U9x)=p@iQ7Mq&|013!QIMvMYS%R;|DLGb!2cE7FZa^2=dDm- z<1`d@-WajMZ1NV>dZGI1H|hp!wcsw-wwn^DQhf6mDZ|LXn(8>l4ZO+U?MD<_+tFyv7Kvh9fin<3Ytq!2;r)8}GGf4L|3xgUeBm4{Xal zZcaB?394CWbsj9R|M-=PJ1|xv4hGP2e6=?JX^eOf1u}5O($A6Ht=(tik99Y&KcywN zex+F+|3vi{ngTBZ;UgS*+_;+m+Nao%V ztL_#-s#Mk0DL+ZLKz@wgarDkAm6mHTZ}FUK-I|jV(=y$-;)oLael+#;qw_Uz!-U7> z7&#usn5~J)ABh9T`Pe5L$AK22PXR~!7&yN7WJe=-G0d! zQ5?acU9Z;?Ijg+kF`P0NT&o}}rj@eQ9gLde6kmOC9G@!>f$k$>c8)o6_67rUves$& zRZlcl7c0InR9Ju_b#RngO*B`DrR8>Q)eMrW-YOk|B+WX58 z+GQ*6-moD2|_HID(!y5cHjw|{ESr~zni zKQhkeQOPbj-uUSx>PMhurtRb0dO3(MpBZ2OQmk9eUgWSH(qYrU_QuZx|=0`9_Hb+Xm(;aT(VG^4EK4o8ue;cKS2>wW()cZ$E99S+=46+yT(_N@OaV_l<&sDF_Uq)?& zf!OXsvMJwG!^a(4f7~4AlJm8AKrXfzWXD5}5QL&?<;Da(CtP3{Lw)Ow_M2KcITnTF zwovnwwNUxXq+k4wXbp9nN3LjU_ zlcWC^GHItFk0A%%m(n_XRFx=bTprALfAulP-G{4n*V~;pF?)d_DWpfF`c?-RMLU0E zDx_~Qsr8`BPOIZ!o8yQ_T@gA$_GvKuqRs5t(bs{q<<7}O6ETAo=X<|a$>WKD4w2X5 z=3XYm-z+6Hv3L|$tPcT$wgIs}RQ^UxumU)aw}^p0_dn)9ep1^)kl@a4pS1wjT0;Iw zo|*nj+5LsRemm_Hp_3_H4+U)(`=}9u^r%`+#~1cYBFW+Td7IS>+vx_L1;al+o-=BZ5B%nzwLXCH?wNb+*p8rQvP>tF>^U=huyksD`#^^A=oZ zAfCd_D|mBbJNMjdlCa9_`zlW`)E=-xurUzKf+92zD<9G8nG@vUlQu z_-Y(EQ9B=NW;8{IuGjnMJdIqy?PD*v3o0!x)AwKdsBkKj;{SwNJsFlFE|Ce(C*hR) zFg`w>_0FVJH=J7F7dc)ymB1riHd|M3F72_DY5r!0aEdYcbvc5Gmuzg4JT>J~?z0n% zi9GAe$s5u;Upscb4Bvtw&p_ZIXQ0OHXn|IOyz#us^!*!!l~p5jh1+wr>9DWa!NXDh z^zabmV<>I2{)SZPaY&GW(SdQ0rrWfg3f(qa*aT_dZ9+&pEV-ECmy4I~8@aUiYm%Zm zd!HU7O`)6BHk6kob=rgO`AHQGIwUXFy>7e!MF0j$sLf?Asxy7dFUFb&5a?YFZ{weUk?xgtoLgx^!v;sf3J-09-rsTwKD`~zswZY!R z48=*d1LdOhmTRJU_%AsTt3r6;&7qx8@CPdg-rq|A14jGgN3xhno|XXn_7el6>vO6N>$Y(G0_h*YBS8JH6UNU*HN2 zsW%E%a^hJ0QT0{tMbdrOOla>w<&W0{RvaazOFhj~Fpo56;gp7X@&@cHK<=50r43O~t~ibn(U zSqujsX`%9PaqVl_q-5d`66z{`9nqjj*>L!v@)$_=`8<{UKWrZ--%E9%u=~EGoV37a z$zJ@!lhM48{WpOzBU#~MPewl2PrjlW)k)qm$gBuwl{3Q0$5E1dY$rA7C+^&pDBCFS zdi0g(^MRkBB>jcJYBs>D}1Q40*97=}4Jo0 zpr%G0Ur7+Bl1p&3wsh#V7M8!Q9_Je8sEsV9p3g_(=3Ky7qi=3X*@G7MiN5O@7a{Cq zLmF=Ne3uO!Z!bN?^KarHl7WXsh6E89^I6Ja7)$MVu62rqug#H9!y!0(^8)tP>HO)+ z`G5O`r7iFaf;6)K?kr)B(V=3qsksvm=J;-_miWZ^o_eju{a8{w*!eW9{k{v2{HCik zA4Qdj&>PQVhZOhw{%Mkiou}B8!Ci@`* zBlODO8I!cw1-I)M*U!9zKLesf2ot1|xHx{s$fUtU*!@k%K~b~2o}|;+!d|^hHkv{> zS1k5+_?F?*H7g>K>ZYOUW<0Q>Cm}&Q8U-h^W0&bo29JN?-~|{LloOf$nSh8kBYv&@Lhh+mi~e0^{6JRD2_U1_P3Yb!$YtzSWb>T-jS^YXNYV}G_mp4 zo2{b;$(?D#PObS`olMWUhnp-^kSsD=X+F#79!SeYTlT_Hx)x4iMR?eQ5Gq|Qnn%yq z6_iVaE651yc68HQJon^wbmXU|3K)R7f5UGwVBM@I+&BbT=wT<4tXke zk5noUX92<}p|whX>1@d=E_&98ZPHV`3SDn<%=YyAy52}e2w8x4NxDAmn|Kv9UHOW! zaM8cE(!U!I5=A(lXr#L>>TSO!ySMIRoz(jccn$*2ibvFJO;08K8@%Pj;zH{;Jzhro zD}*==|IZ)f$+W+9zX-1@8Bt8X&qlceqq?(dW+H#~u|Dj)I&yGTf4@Oal6fN~Ai1w7 zz6A^p)f|zF2Se<}r14Tz9wNCQ+iVF;3i!nmzwUeqHfpC;xqz~T&2N4EMBb}a=r8={ z3&mCKbia|vwjYn!$Z<}22wVMT5M&)?{ig)~6Gj84J4&5y3NC5^wV307gkcO>SNx_$ zo)a5hUEjGrj?CjGVwJ{oTWv@X<+MYv{68I<4H%=;D?}Rx(_=HFmM;2NnU8f*9Kh2B zC{ZmMi0p^qOxMPoJOf8xIEw=b(kA2Ee^53J8+S5#{vJT_{!P+Xpg@nwlVx=D38lKj2WccAo-%xDL1$D^IfsHsi9sUPRI z0_%|{Ml~I5w;h_eNzLFnA=u+6i|ku7#u2K9A3#WJ+2Gizc3=jkXD=Go{j}2rd zoNx2zA+NGpJ2(|fcI2~5Mc5WI2?S2_IHYJecGE(J0kvTAAbx{#CWCysiQ4LN@!bu@;ax>0Q`=ji20h$>{y_`=(4+wK7YypBFi?^grf>AuV^I0|UXM6nN6~ z#h;o)6eDdJPL!_fQ$mD} zuChDcz582%rhPhRfjuG2fKb^|UR+Q9jj`I7aacC<_V?%nHc?=`!SGO|pP%qGXU~v- zo{{xS6$~>m1imYRr}gM#tKAWUQ4b<5Q+)CjCC^*I#j+XK8y`_|d4#r+@7nej%O@9U ztLQ?+5it~~qdQ)b*^ga%sttBI#Iqes@PR;RRv?h~!${5Fw6xBKEcID+%l^Lv+6>kX z@Hg`}|0GS;lB%018mWEpIdG-hu$%-}!lS`An%DpirhBh`C_3!=hkM?>wZ;Gbb^rD=4^5b;ZHn+2C%b9@Kir2yA<>wmUC$CQjLO}ekiH5xInl1;TSP&T6k?+kCM3P%!< zBeXeD!CZR|AW=>B=PcZ4AWRNjh36!N$TZt7R4YQioWxEu(r}61#sS404UoMQJYm}S zZA&FN!IsV~87~1^8-W%F4)-UPAT%M|qV$Nn+tmZsAHId5ccR>dwSjdA!!~=;SS8Zk zUJVRuT@F>J*gb^>+ByQPWxcK{_u#4v%ly+P+r z|C{A=r2xy%Vr39T{DDBvo8M-DqC^=gRN?0ov{p*`*kn)MIovnazd4moqkJ^S$2{1ODX1y9HsqT{tBg-$l z_7-#Ad2OB2Gc&T1r14qz@OhKHk&n4zjyQS~hRaOAg+TJZiIk~@OXaLp{*Rq}^u%z? z=guek*ro}?X4P@l?g!_A^Pt`M{J77{syZxOpd$IH`enmR@qONn4TAQFOZhic_+d>- z$VHW4yAd%_GxkM^YlHgtAa8Ee1I1yoXh$TLa?A8gm_c>*J!V? zKCP(?muiW+PJ9&+!;WVb>i*>&)h%D~0;7YWyhz)+ASDTxTuB%Fy_PRiJKFd(s-bzR%2Bge)n|$m&+cC)4s_;Yvgl%}JJ4vbJ9=x+w`&dmjkAy()b99-^)=1&GmJhVg*{|RC z(C}5r9>4X-C*z@}d;p9MX{lv*$by%WqGqbkP_39usv!QXHt?ZfUPPI&Hu($&p-+RFRV4tWk^#l^=X}-hyCrSa6CMo>5B2@I>v23 zaJMx)2NgmT!f7y~VXwl#eRonDWfJCJ-r-yMXhT=(t{S1aoJjZlBjhktzhKXzS-N8B zU2L<|jfRUH5jQCABLqz$iTUO81YT#K49=JqN?q_3c_M(bvi}cah63&r!*Y{60 z7?#T||LQrmPs~FEmN0k4TQW^z$Ux7p{j{h3jxreKA0x*0CzYOk ziG{h=c_ilQPRkfik5!WN(7|4&&EQGl0X-8vCq9PS)F9y|zx~R>Tw^WY7uY-J)sXI^ zvl~wK?&maKy0{)pdkLAf$<0Pg-CSHS+E2@+UCb%W8sM-m$k;A-qYbV~nOzcDm=fK^ zESxz?E6feGNCM9-)>sx<$d0rrR$ay{_}j09H3YtI0ogthi;owm>f7*)_h;jcDrdR{ z;?EJuJ|E6OP06~9t&0a!_2@=lm8P|i6=LpPdQx8kMcESCY7CoC?~^YQgNd1p=b@<@d|N!SKWwCzHkh_O(ii^qG{a$20FmX+W1@7pN@~bJ zlFZ%d)$fsqBO&8W67PvvWIg7=d{S6c$N>3)bEC8LYY|De6vF+af#^V~MPp}yQVd=3 zviryGFOjDH!lUBWq;8cv%k=P1i6KSq2d2w*7#;f15{Af==^>^8a(49MijV4~gw9^! zCv)||34_AqPZFmAO`9K3m9`(M!Z??ACd^rF@xr&zcEZE#_0JlVKUmF+_#kz6%z`0J z$N$FiT1#4IS+7mD_yPeQ%tIdS78+TY-f+nZXw5l~|J=&;`mIuy`!#s21Kl5%b%+?H znpNchtajqXH%~`q^9@}ztV}g7AI#p!y_N1m^dxZ4(IQW--$*^JD=iwRK3mci2mud! z4xC+wl=ux|>UqcjoquBlm3PNf*Hj#LxOiK-0Pi(~B*a*KASNeRk+VSP+IA*(0TWc) zduwOi|LKZdc(vzVtV6UG4u3tkA zrUZKLq7$72f2_UJ7#s^{jQmE3GaVva_51eAE*$2ftsva>rPC|#`{E2ube+gB3*2!! zijt$B*T_WtA8wD^@8~WW;d|XjBA6h?TFc$8+qMfI9CF9+2$cyzDtQu@`K_4D$Me17 z2Xseclg~o_v~paz zL^dgr)hc^oiSb1dc||}5P4D{V6LJ-D7GI!<*0`q^`LNK80 zey{o4_Hc>Q*xXu4HkAN+vE5wmbTdMbBtRL(c#@ZK)Uingi7#HM-l=BaU{HQRoAa5@ zJxh)4b3bllO!}P5#jLqIN~OM>8h6EYq{D)bq?<2>inhIN#>r$|u1U36ctFQQ-KE!f z->n8uTu$IL@UCQrH;XX&g?(zYudE3x%#O(O4eDDC<3I)+-oRniXCQANEilZ8aNa$K_kg@t<5E$2{j zp3~I3w@cw!xNMqz<~|OPKc#?YzwbPA2>vomu8kM9Y+LhkWCdKSB}PsfII7y9!X% zWr9}xD&XZ5+jkXu8DXH0aC>pe+k~MNL8e?+=#*G<5@P%@HbXn&-3-%?bQgS0J163c z7vAM^R$qRCriBSRPn2^`4(lD)GFE=b05XCVQK# ze0dEj$zs4U=3ZrE`Uea$axisxmE$c0#aXFd-uBUudl&vlIh6F9r)gK7J~mR1@1I;& zQi>XY1-$Jousu}6EN+dwG`nmv({c-}{s~z93Y>l%puIe_VtXiK@Zx1kw)&1h14Or! z>@BVbAr~OVVsB$##52@2nBH`F92iAC-pz#w)NP!N(lm9uW)g95sre(gvweZ03sWCs zI^?Vz(af5Yk`QTe%DcHV0b);|2kl)z zC+#8J*ix}yl`~CB_J&<5Za!;dbUASEJhMz2#hIgsCu|Tiu(H z>)(}|rSkU~@5@UtKYcEFS^&2!1WTVR{odtlTrv2zEq{&j@w9@o;*6o7bFqd?ub;h` z`2ADWzXGbynr5Q`{WKQ5LYe#peSgVU*)zUWszWb6o2aX#X35-tc=st++aIMkxVNp#X^77#tfvefifpF9w*5($bKZkc-;7&pCMSx8-tU7@(7jewPG-ez68sNGjdslLj*-N~Wb z7S-3VRCs2KFFtieL=6mni;Q-rv~XCJ6t4m5SY;xhIBoGseN)O)jEGz7zGT+E_|_vn zScL|1aG&7kok=eJd!lfp?*=Lv<0I0m0@rP{08gdvpLE~x8U zVA?8DkKtg$AF*RQrf=_Y`^BU*i#S)#SPNjb`jWNF-x=) zWQl7+^ePT=*vY!~h05fIoiQ)sDX%^jyq9Zf)plC-Gl$t0&NB;a>u1@e_(*ClJ`7a4 zbi}+(AvUnkopEtA6?;u`-0Nacb)9oE30@kMw)k^uOpQc<!P}!L!J1}T#n&IuU&3j%j1wC*J*G44bEA&~v(Zgk?9@W3bbwEUHxPn1wt@FxK|UjmRT(T9as_F4{KMQ@gy z#o*8f1{><7=_7qfIr@6ZNTkL;ja3uk#o9ONR3L2gM8xZLl-}h$HC(B1LFEaFQI3)6 z;LJ&GVLm!K_g3t#n*H`&Zsh>zQd?1Be`;@nS#G6D0J+ZJS@C0aEKxIoxRp?y7d*Kq zmZ<=VI0s23@+(#?DakA5KrLgJmxr_$G1;8PK6Ffn`Ztgcz3png2F-T)npJO#{10kh zo`rmX*+vbMIOm^J*AAXKQEN>9qwxgr{JbhCUCf@v!OcTZ2e%I5`5AMEr)e1Jl=<-c z9DhG!vsc^*G402~%QT&Vl2Z+ukm_TOGAnC4AG3BZqm&X&neWaKAee=!PufgQ_+>#X^~kD5s*=7m-g#kttesD3)I_qWv*9R>0 z%B`#q1Rt|%K(@u}&FzbAZ=9qnl?EryfLE`CH;*>i%^FORCFz-WlM{^S1iT(7zDtFcEvH0}X1pj>u9yaLZWDQ@NtXJSm5)Vc+!(sTEq! z&SitXR=vOsMwTW6Rir~-{ul2%uOu8ki9EeOLZK-E)P+S6*=u!g`APFy&-wkYcyXBm zbhUnTD@Z{RQ>h_DqHhhQOq2@Y_sm3ynvS0Bdz4v8J(J75Vm=kI?upyEI_XAD2ZUyICJ{ziW$(p>P-UPy$9a&d$;>~bx4RucwdxC|f)avv* zmL_%Lmrm8k%?v6>>3px3v^ARXO`?s0Gitr*_5!wJVx^_g+N{sjSb%$~B9!T8Z#GX?zqK)_ z(KK_uX~OmnLf)r{9q8Cv-m#gnBbn)Ee^~ox%t2Zjf$jqh`lP0h?Dd{L*VWU(5TqH@ z?{aDUO3mbp_=rWwE#cw;J08*vFjON@IBY(9t2_Oz9JG4D??jE}c!~az*#_t02-EO; zttEr&=r#ZEY*~vsI$tDvOwn5(L=(TAhBp+9aP=`qU|D^Txa`+9 zv$X!2l;_jWjKs4?eOxx}_r~04!$OKO-cybc(F*8U-|Isrv!0j22&ZIkhszdTGu3c3lcv?kSf(J1_a?#=iSR z2JvsdOQ(WMM;OSpKfDTzs`q^|ZoTC7JUqWXLdS?M(J`g+fwxO5@71Okx9cLggTl$* zUB~;yPs_G`ye;!WrZDa-5u5yPt!x&QOezh zo2Rq$G`g+@SA(;DfM12)X`Wum8IEUON~WOqeo4p?m)A?Xcv)=ny2SM`+^iD7zB|fT zuWbLp*hPE>-w#C4p9sseEdOE0ITK%rEYh&MvyPP2X+}0*!>n|8X}c{ESwWDG90nC;wB|<=ll}#s+86Pt7|-kf%K!O=39$mfvmz&l+=7E` zsokv zvF31n*yKCWH}APu&iUe$gNGoSaLe&=#r5HLInnYFR6o`)VdU~l^)gPFT?W6g{9Fy2 z{e0U#12g)1gE@v6&x}bg&y2MpK6mZo@o>dedT@5{nD?~x15t-<7RX8E?xHE z(8Q(&l$VdV8>?)%c?_9l;`B%NH-+_MJ>uGIfkin6-N zX!t0D>GHz+SDZy0w^!_YV(=lVWJ6l^a;j=>2Ua(GBUu(PlaY+@`2h|0dvkJmu>D06aiV2`3(o8jzGi(iYF-+no(g_QD6|x*&8$XhfC}b$! zft&CFMU&>Z{p@kqa!E@>*F)_3{@#m3A*tLo*-yhB=|*G2Ay%9=5HE#+hKi`vY#Xf) zfA}tds51})ig)US4F!^R=m7Uz*N52__z%>L8lbd;8cfQhE_!bLy-|k=uj=g$&#WvP zX>GXJyS!aJi(&!kY+%9h!NsSsDwFlq$E~W@d9JcDqb02A@KbYU_e4%oj0yOop%Pb5 zofJ(Bp;Qr+!F>iC0H4ZhFh?fb!DP;bh?`?kcAWmcPw{Lgv;C6nO3=G2L(f8rLO;{i zgr%jsw>o&q$~{4bVvY8D87X{U%wDkgDV{`486icdz0{;>NkJH(=CHtBO039q>NQ|N z^{^b+Hz|qg_NP2NqKPd}FFGR8YLEtlviJEnXvA+$V%K;s6l3;%+3V`po4!$ucnj4$ z)KWMHV^hY8D*JOw^e+=uwo{Ng6%*f6KqtvOwc>B34|-W;9(8mZ*WC@_0sYn*TFtNE zQC9%>Z052*iD{AQdmC|!XXFjLWlhA1;xb-))^LWwQ$ZM%1HE@bkX3SlVOUXG(4qC) z%?gRk_Z6|$(YK5$GxqDp~RKmFMV0RzF#p5KSvzlCMoZdAJi z^@_NmMJ6GTw?U!^xjwd&5Qd?Ot&H>vW2jL%voDXO!yBi+icIzM+Yq6bllr-DA$~T^ zh!+(y`=x?tCa~(L*W+mP+*6PO0(1fAo{5?%XxP}~&wzwm`&-kE`)+9?zw8U@7B1M0 zlTycoJP|3bEHZ)ZF>ABqweNNJEs;6h%+)Zx>wNz`n_#M`8Ct?KDsfr3{mhLgY^$93!TRow6EUh`W_-Zs^M7cF!osl0_J+Lme8PW* zmk=aAH1%fG4)-TL#qihpD)OW-ah5I<^QFfI7h1EgxBVl3k6Acr?Ip73Hq<=YGUjvgY z*AxTML%G6gcBtBti-?xcM$!7nuI;I~$ zS;h$ToBYq7BT9V9iC->f+sPboQ=FrRwcY^1Hh743xz1&%q zyni!!!}3Tov3fc=Q~hDc>|>{GFH@HLQm^)yJf(AcJ2p?}98rF_>c9o>GGGQBggJs% z+_!2?B2!#aqCMHRgGTIbZTyOkoxgStssrm2RZW*#`nwmXCM9;Lmu{R6&u2>6 z&u6m4nFA92gmP>*r7nw?np1r!+aaWWjm05%?DG@95$eReDw65iix9C#SVC5|lC+?! zh3>L-TT;gabBv{+kdDMM04EtIx%?O;AJv382rELnk`$MVMJB|106wPw!4zJn6LQyGk{zcS_V@R09dY!;cZ#4WkK zk@W6nVS$V8xbb2xP{|z?BPuaTN7#9o>p5VVdSb%A_F(Q8^VxOLR0k{JrcIfS+oMQo@O!z5<=^sqJ*_T^RIs@d_-f> z2NAH#sD%lA>=FN8tKhKX)E!cxod@QJ(1O4}rJ zhl~|&z97T_7Y-TCzk+f{I$=)N2lLIzHbPl>Cl3OQ=8O-@3Ug7ox zWW-wWWNZ&;^c2x}gz&J}$h3rJOzyHq%1Ni_fse$It$@j4 zVjJ`6VAR_U$}M zSh>v-KXp9IFaN1slYPo;uV^sgs0p@2WaAl!}#s&0iU0a5U^r-Z2IghS&?_Z%jE zncg>Tyao>VwSU&3oPvnRst8T4FMkS&_~R` z&q*QNwh4M_1u6m6;*{wS(4H%q+-r3>{x#k+oD2osAmNM3df`0%0~eCRSK(YqSbHL$ z(Akn@>ZVw6;0Y8=$nV)Vu=V0KdJYTnmy?j7;P8iEV<>!X64e)&DZ1fNhF`p=0(}D2 z4-hGIES&i?GR$4J-xr&vSz!7`v(n37YKgFE(O2jZ6YS+Rw2tYBUTMKl-^P`quUlJ= zaF>tlB@xHovU867-JSu(|JwkNO%XMh)7abq3%X0D&HPb1+_Prn&fR7Z#tP@);$u$Ar(|xE z<17P@g<*Ya$id@4a!alFt)N34F^R(Iakk?i4s>a%ZZQ4#g!nJW&5hE=RU606#-=aJlkhoP{kK;PoXkzoYfIlXP%w3j}Ft0Hvv|;xpu+AEttQMC^XKiO-&b`n;ky7#EuoNE{4) zs(4;<9iW(4VzPN|r72^S+iN_O;`cSb_T|#6t3M}Lnor$ljl4ua-;vKu-25qcb=e=L z17M(7xuDR%E@S?Bs!=~TUq{;`P^@_23CkopEq9&lOU= zJ!$Z~;hdI!@PkWiw;T5Fq7iI(_)KQT(tWXOx&7Uka1jpQ=ZqH6Sh(zCi`q<=0dTSVN1x5a#UWJ$e4i*mv5M zFiqFi-a91ntrQbc~@0NZ}BqI z@mnDqR(-WU$X*_lBX%_j6Q}tq56=Uaqky=A(3U`X^K{a1d=tOjNKOS&C12IQzKc?t zYGhMpuAR4T%+6u|n5IwvrCkn#=CmV%>hn5_JspdAFz|qOSvAw-|B?09QBkk$_pl5D z0wRsl(y7v214wsBgM@TI&DQ{`qg`aY91c-_eYyy}}4umfReMKSm-ndO^ea=qvkJ`Ajc!-PJ^F)<{`M zUk1jRwiVI}I@ibiLoZLE6=)nE`bk{b!AgQ=NlDwzd8%Dl`%-XwXbnfnGr=Ej&Po2# zqWeHq9Pu@8fm;93^I-B(zBR=sA} zd*nhYA)sM8B<=F<%jBz^Lco8Adm#KbBeY#V?1;6zFfS1&{C_ls3;M$xd>?`^L1GzO z0{TTO-lKz8d2fuVTai8IUi&f-279y8o(OX~JxvG*Q+cBiX!@Q$l;#2)=cGZVrlOh< zFmW15U>cO`M1Al#rPaHq3Vt49NvY4RKeQ=T658V*;)!1rMhvvVG*I?`J?U6vjl!{@-OE##hN1hT~bi zI5piJdz}3YBnhdvC01t)g zDmIznhb#90>CSoD`LfCRZ+%nCg`5qQlXoG8O}>l%5cazck~+fH-!~CIDnlgEnzk8) z@S@P;IH6W=|1cXo%G7$dJAB_mSjOlV|=W&7^>1+yruF>eE{lH>N|C+p%jYoBGkeM^@PQ z+xMo11(Z2&%Q=GhA`Dy=b~rPJqiYVO>qh`*5Z=?Jvpc_O%2w*0!`VIaFTM*YO={>$k7%SurHO zaJ5pcRDF_Ay=Rb0I<1k_^;n-+yAwg5gbD$8xXg`)6_Pt{4hFXQ$UB1~9y~|jpR&-( zD(e}ZQx0Jvl^4{1G|V@*VFt|KQ29(kpkrf2q-wy@dmsf9>B4>5i(8eEJTRs!pzQn; z_kT5`*Ff2@GK|ZM5+^D^cx!xA{pIyXi8Yp*uo02iyjs7;tE@B;B|qawpaPtsQ&AbH zyDF)+TzkX&-Gaasot31Jug2(>zjl2F{jKM1WbB7dJ;^!mO_wjwx|uz{SjGZa0sycT zt82tctEBqc?ol%z9Si8)_-VD8W)!<#Li-!S00Xr!wfr_-PfJ@qSTE>uH`lG=F~Ka`?WnxC_tbx1}{ zpr4EvQ#_K?8o9|Vb;C?YAOef{PK_Hy1d%t+nkup;z%w{d8gfv5e8V2&{^f<(dP8;Z zZ?qLeu@3Scu;&UWh=Re~*D2Gzi|mgn51?=GFRVN)30D=c$k zRe%v!`wP#m~mYI-1XQ?Op<%PaUiS0&Ixc}@pJJf-B;3@T74^mRi{x*->+3g zxqo`!uBtpW|H|VO=#MR$1yz2I2!wkwAH1nB6GeIO0NRXR)YI->Jjv#O(GUzt2Gb8I zvyQ?{JifX>j2ggPqUICW18SxW;B_=0?^}Fj!15c^gAEv!+XycEC9GMrF0;eM6lGGIUgo2eVv|==LeGhvOpI~L28rf^A(pGeKSM>U|MM` zBrm@oQJbrtrh(BbOgcFTXv!`!(vg1o`GI@&N`)W0f{OJHf(FS(Mfjcv8419ilL44! zSU|$$BVQnL&9(ux4M+dQqD8}uhOE#B%AeF$QvnTRS6L&j+Y>wkPSbC_|F>qxn28ay zc(n-a|4KaK_!m5`ff&o`jD|>W)n3UOnH8sU%9&+^3K*b7zj2-5J}_K%|AHp89TRBx z-84s3d_FPJKgy^v^hwgKE|I?3uYa`#(%KU={^D4@m27)=os1hobmxz8M1hcH zZ?2A0COwVNWz|+;J~&yPx~=M-k|9{~V?p{zbJt65VQ{N;5d*gG^kzy=)0ZtEFCe z*-&hr3&_xea6wEmIL{+0C=;+8kj@g(9y8XzO7EN-3MM`ja2%zkD@I0+N3R|V58xkr zNjW2bxG#Ebegf_@d0rk3P{id!@^KP%S*D1=hBBkOYSq*N?#3)pwwI6>+SQLN*jGLers8asoJB*ZoNt{Cn#AdVq_=pwTFx9gP?st3Q5X8?Q2)m_wMe>I+~!C*93BQQqr573B*X(bBA`Q znHve+bDW1M36kZboN|ZrvZ4D5yJlAzK3WzATlc8(V!XwEnG%{{d*A}ofS7-#>XTo) zjims5z5rZpS-XcSknHLt-D*CBaTb9(jR%oBOm95#!w&1rdqxeuqH(O7UzvK!_rLPR zs$%;O#a-lM3h~(uDM*mEde3jvC|}XCz|w27%i_^6JEdc@3=n9oe$Gnk5kXoNj-v#C z)GXz2Ng$OlAiWIte>M;k3fHim?w89eq^_69SSZTcOGWg4fcu_;3$)d1k-A?3i`mm) z|Ad1DY&jK3x`@)^vpDW%Iu}x8n=rl0UcMrL89BwTl4dzjYr`Z+Py{lDnoxixVt_{^ zFA5`dxbO`RMf9x^Rw%7q25YxA@2+sE2X>#B%Bvfv*^j^OdcjKW#*BBXVLd1(?W8AX zSNIn3%K4!`Tz}5=E6e}EWRVeJ_j;;?S!DUnKu2dhG^4_ntat%dX|aeOe)sCwFp*ki zOyz*DnZi*yYNzLD^5*%-jsbN9cWG>L0@RJkawamm!dOye^Ua+x3P22l8~Aj^YEzl< zF_s>Hc!3fliv)dx_Po4pAB^+gMW~@)%uz`v@2t)I?viSutvvKqQY9u$`P~fQz2nvn%>JIb|Ss+*Yr%S!o%*! zT?;ci->2@Dyoi9f8cN|k19mnUK$&qDEAc&1xVzb9lom51xwwpUpfjr53np|a7BDqE z+sC?$Uy;1H(JDw*02DnyS_`dCdC@^vzR;nK^sWtt*kng>qqh%50C@nm5N*qp_9p@^ zfk}f=e>?DBk6lRIaEz)lX0pofwT-CGCaJj-Iy`#HV4<*R;?>50|F^qH#j9DNxy$-D zC3+f%rW4)vbgdJOR45htzY7eIk%SUs#TkI4BcYvrK*aO;&-Z@-sg3r9*D;)u#L$pf zRHVrBXZd3kEeP5`g7o4SjMGAZ4$njnMAF?{B$iG}M$a!-R9glf5(6DcL>>&JmJ#a# zSV7BJsJM^%(Ti{fG9L_;UZ(V>AHfnbQzNG9K1`o`Q$(M-k*}&mY&z{8YEU5&)?%rB zH5m6VW3w|TrdMBX8&9OPewz2-!QT_vS7I7M=gqqD8~E=HI2I2byqXz-uGJl>ABuZN zrvjGNz>xjDEAg$f#JLnZ8gv^yqBNmfD=R4iK`*tAHpqKWigmI23)-&wwTph>n`IOQ z)007`u_Jwa=xbO~#T#||k^gx>R)h!KmC`>TqxNwF+tP#)m&!jC8uel}zv1&t!o&|L z-J`v!lGr&Th#fj& z&12v0Xis>*M{Lx9ALQf(+yES`!N^vEsW!RkoxrZe35B_3l6SJxwFhca_ zs)MdDhGb^%Y!Z0hvk?JsgU&CnopObVfgTx-rIL|o#%h4UE&xG$jTjd3G|2^uIkM>=KNd&z}#^??Ib*ZvPZ(|Mb5o<}yyrl#8 z{N&7Ez3bND*3qo9i(XZdes}hI17Iw3+ABM6*w&c2CDWvUC|GKF!2&>)QML;X_-3=UdDTQnqdaD*j$NHJ zP}Vjx$iGczv;zAyF%rC;D0Zd-u1lJ>c`}1;16AP&y3d!Gv8Z;$LsYRLc0v~X=k`dZlImjE$1=bzI zR-rkWH!;TzYYH12^?rCmB&XTwFVY*DGGIK_OR~%FLvAY2x!mPoa00_9Qn$H^?M{UV z5iX0}-aIIozO*i=6n0 z?=cbDH0#w$Km_+AtAgmsi_&xUpJ zr{fO6%$wp@kBi18r3URao3;8eLTq37Nx@V6AcosH?3v<78`foo;akc+cGkQC&KmNI z4@r?DtNn(5AWX(9AYqS#;Gu$W(8M7&34W+TdNGURD0AYSq!wDqQ3^Ok0EK^CKGCgZ zoAY5`Ls-R3TA(Dup)#1uVOMIAhHSYC111cCAy> zEngcAsQ$;Eu=QEaW~B9C30q#0w5H{(U2r$yxTZbV%vLOo5Ymj7i+QA=2TibrffcHO zx((A3VIqCMU3bME~xavCkGIUK;T=h0^ON$7Nk!+`7#qj8{Ot6 zj)g-W29$db%@QkbOygU3Tf>yaH_G$)J1h>+9}}tV&}74mbj;fF03AK^Rwn@&>%9F;|YP<%U zF=|14kF51r;m8PtX@my+U{H5m^R_F;)~XK(QlF;H_TsAfDwM+`oo1jebc>xTtyRcK z)_-puMSu&|XCVjZ(ExNM=I;&i9J@*?QOD?$c`XAfO^~3cS&7d?6gs4aFKWVCVbjyr zc|pAiZevK!qS5O?!u!d@fU$tS7ae)( zdgNSg}+YZ)fZL0?M3eBvxMLu>gU6OfAqNCX<9D{-8C;CKo!t#-q4FwqXVV9 zSAc1&mwzU?ycX3+xTIfHAA|A2KhIrFPICF9xc9^2xt#jyKl}x$E$F_5`U+FhwR43o zo~9X{5P(#~YLC_P-4;gwO!MpGvCk9Ubr#l=%aNxiueav4-eqapk#o29ynKhXS776~ zeEb8injTc2q(*D_@&=ovE${P)cU|Ume=#Al{6Vs{d+iY3F-P=tinS1^Zq|#V^1Ge3 zy-eCcH1-gc7NbtT*aB+e#wb^BTJIqNUY6 z1A(5J?xWq$d2PWV^S5Hq+~{7&N-mG~w*Qe$vs7%h+n*?fsq+IeB7lK%MgT!y z(nKnwy%mQ@zOJtzPYs=8ww;b5@*6j=0olN$OJqm{f_;5+-9H>pz9dL12EgVHmLe!C zl{q_(!-4FZ6c3TE9~j5+VguicN)6)(EFV#V#F_T$9beRM9l zH!ZBYUF<8?o>R^L@{|iYwnluLdI|Q2;8|Q1v7{D?h0fUOgS8+Awh7K`X?Q%3=LC#2 zDEu3KETGqcb|WET=&@Xo5}5%Lc|NpD>p%YlYU<{WB{*_E8c8?^ll&NeSFdXq2AAM5hUa8I8&O&wUMofbwEFc5*6A`s0D zPep9J;pak=(&H%S1v*xFn{pkUJ$vcak2TZ|*d@1Oe;cc&9~=5sGG=5KYXLuquFfocm`a>?Pg0js&%G1dOL?12Q6mAz`IpM9N8 z_=YnHz(*MJ%>A*S50Ee887bP^X40Vo;nc|7a|ZXLZtA7 z$PDBFs5}Z{%6oMC-K>ZHn9x}uI84J`wQm_&`SPY(l})6n`5>n&&oEr^JkeG7r79p; zWJ7Nxxx4}|qdGld?QMTnSO2`!U3uGFcicfaQ$K7S9Z#^9UY^ zIsBJW0eOsB$Fy$ZSz}mc+`D7#QgeP*L6BcM|NhP|Wo!55edg9g3?jSVPY@OZ1>`+< zqMYHb_o&dD&rB?3a`=6$fUspGx%#CY?-5xPvTLMo>U&k8yI>sR0~joa=%*GyXT%A= zN<6)BIf)4N^{>nv2D6-w^oza>7yxsh*^jI($|#UhRoYNbGUybv(6Uh`RMUZaR_aRT zXUv9?ls7oMEN9Xxv@38qJ+^-~GEIZ)p|-M~HlS)>XfyI--!lVWrfEYYZb8`Y5v-(5 zd&*+e+qr6*7sg!7)act*P+vX5T733OkwYjyLz8}YOYHw10xer4vfeX+!vGY7&|z3j z(&)SC4GF_29}UmbqNkjpS5?a@(aP=Q(yjGMt_m@apkpdX~lpG-D+#2e1KbrW&u=E4E6C>E= z>kwxj=3I$wf?({idJU`@-1D`VMzU^0G5Ff|s9Udkb3yznA6W|;-`cPJ<){2p`z(z`Kek;>;rakiDjQAHjnm#`G*{9%U6k%k8Y& zXYj0PC=bxl$*##AGuRo6vD{}Z@L?$@W+`n+Dhf8L>0C8{!4i!mV1K^hT|TyL3X9%= zXX;t_^iIjx&joS1e-@v?C$4yuOx@pOa{PXJf#kr8LTOa8yu?Dq-G08v4NV4+{6qX$ zG%2I6MzN)j6mmS>%6c_$kqFKDD>T|l84U=nb~)SLy^akbQ$$a%;B2+@*7RnW#OIh$ zROShzQelB2CkiJs)RyHflxO-^YH~Pr{Nf=MQng!^!<$H3#UKo=Wio0pg}5WWMimiz zZ7+$D0hJn5Bn-8gfCBaM%_(<-D(a)c>77*g)bj7h>6gweU#G9V8d-PU6$yC=NIwO4Y?|Co+HZe-q+*9#H78#0Q2LMoE(dA5{u~pa zXiXSvc&to!8Li8(`?O z7gFQF6-gd-(|9|g9w`m~!nbSU)}oa0W^evUlcVkz`!fMkUP!(e#HM)9x6c#CtAADp zEvOS|9y9#nJNTrOcg-J#dxkKXfxAwG(0iMN9CkX&UuzB$WO|1sn2->3sp=D1aJp1e z1M-d@oiy-uxYU2WXL{f-e`3MQGIc*sX&aeC$jtXQ!2mZSUA01H9A9@GU4aCPjPZ|h(Tk&R<(O0YpB<6 zZ@3KP`WzX@-Rp%;QgZ;j)WOwivfiE?eeDaYbJAGqp_F)PUM#acuYANDmrdsxSvl_J z)U;LleT4{E7B#q8sCR$wl_w28AZ#+J$}oLx%G#Y0X;9X3Av-UfM>Mc=uD{Eid4t&r zNd%K9`%J#(cbWS1IT|B-qd3m#d*g0sg!}K*^A!VPp3L1w9Z2+xJ+mRHB3YTrY@czH=FI*$ z*%2-uy@+f9qpk#eSfSt*H5!R&apv zbG>|1dBGRKdYvR##!rk_>Bs1su;RHBHjm3#1IaqlT}^u-)%L{dul*rg5SVcOk5+=d zHPxmJlLdWv>7Fy9+l{c@0gx;3y6nPt<;uBv_eJDKI8}2Ud%rv92UJen*1OZlD|vGq zDt4?u4(|al7oh!@CO!Z9MnYmeE(0LAqyF64JrM`{06+Dp?i^Q``~32G12L2T zy=xE!s_W?pa(TXj(d*DGdNX7~%LQo^nrqD%h8cq8>rcN}OV1W;sp`yylDmCiQZ%`> zBtdFvuV|U1m^@_19Nb#ok>v4Vr*=%ZUm+Tx2j7$-*gM;)FG+2YkbituNxjsB=tgUhx%SRmZ zAy^2fs$bmv+$eu($;lw=tdUZ>8C(%l2VY5qSJ$wQ#}^vAMMOstv`;@ zL2sy6Uwn?HzO{<1w;yvBy0PUJ*3Dp(FU$<&^*~_C4g@GP6lD3pi&jLpojQqVR-Tj( zq7Gh*%qBTszA59FvTr!hif8YAURvDXG(wwr{j}xt^QeOb^KPsI&B1ZkjCaFMV1fEq zkchVNP@7`a=OQL1rkm2Fb>M_}X1eSJrxnJft3-N5mXtZm+T4oY(j##hpLLx?WPk2i zQSH)mE>o?xyiFY?9lr~hfcY%wzjEJ2tUc(J#}YC^0=-8+ar&5B$tk+OTLc-%%=c*) zG!-SC)E)mjqGH&YFr?G)b*8F6?kVZK1*z=1dD&+RWDKFGuEIWc7#M ze>f82gsp%R1}bWueE9MWdqO|ieL=xRkky0s;-~=6wrtxo^PBkX8u~=5x5aZ@Or4h@ zbJWjD?F9W~gf#)AvfGUC)(|dI^&dlWyJWba07i^7L&X$d_Y}s3-4}+aH~3YgcADd- zw_wK-8p`=XM!@WtDZCu5@mA*db1wN``yLq1^29xGMUp|JDi**Ul=QQL`vkuqW;W~Z z_C#B-6!y(-U_Dl%Qnn>pax|(YND2t+tW^C-`olbw`AYVRTu4_V%o>jd>x}El&06k_ z+jN^%MVp?R+VQ^FhVMbXJ``Z#${GGYO1>B6n#Gk3vEkg`IsK!-83qGUUEWy6z3chD}De zI&+R4KAf@tqpc3-j&3hoGQl~|OFWAqQ7eq8X?kU!6sU5S`ylFeY~rABe&ZieceIgx zcSm7a6z#LjBpPQB2?GnBD2;PU#?L-0i&Z7!VQkP3nj!hWDSad&@y+{b*IX*G7n>%I z;`C&OiG!vbM`4P0?}9NF$f)SP(*jgA6((pTbAOUjO(`ABL%{i#>9ttY9>kWH<7RWv zz*yR)sKKfAyDFLrJd3s8P#{!JG z!xIPfR&*AjA+(*;W-%XZ%Xco%w83U2D-SM3Sk({uZEKTwXgUV}WsgqyhiZ%iQOiz) zoMSKuq*F(IyY5tvk2_JlHE}3j9XEU+3yrLhr=2R?%gwKPIQ$2+Q%xK6721&rvfB}V z*_{Q`{FNC#g?lk8{Z~|i9@8C8_65%TFDHT4qp1*$J$b1FFb7O1vrW=<@=bGy>Rx?b zQr()8>Et6|gbT5eyniSAeV@B8gj+aTF2d$5QS_VWZsjJO9RYvgikzRykzMT+N7sU| z!DE+&F?P7U`m4mus+^js{SGn-d_HYBJIYV)L5#?6$|%>?9^=pHx=uKETGqa^V@Ui@ ziPivLh3eOZfT=x0`lRY?FT~v5^y%fZDctvuGy+a@XhiqREt?IKds#9_%(R>ek;BUu z!MjX8XXFxC!NNf;S%x$gm&7pNLFT_ct87EZo@xFYUblY9y*MWL2utpVh?Js*Q-DZm z09}Rb)W-cRXKk7pd|BdBcJe$vpewg>IYZA}fuVXrOH3*0nMMbXKgq-E#kbH&SIKOP znXY&w>X<-qWM^$|RZ1$Mfb)&Gk(9xYS+1vp5k^|eZ=!?dxI9#ZBoe;h0odB2Ink1q zMBdP)uNy6N@dzz(L}8g8T0l4R<6N))daut43#o((5dJ5$h%6ATcj@>c@_u`#M_EZT z%E{WD(7?jw-rUe+n@kT9`W_geFF$IIptYmODw89~$bX)v8CN_{do~Yc=z|NQzD3y9 z7%n92XZdu(4?=*y1gNW&q+4#>YsfozHIRDJ{9n-T*gj->N3C;e{6V>}D*Pk5;usIQ zr#G2x+Jv;{)se3JAS(-z(@&})Pq`lpKSS457n%&+6rxCS0X7azoC{W^q`UHsDcHU<0;!nSyGbE zQTrP*N*3Idc?CnW{vjQufZl-9Ix6sM36T6kMdQJP@d=xUCfI6A-F8c(F*ybxlH|(q zZVcyE%P*C`f5Tn(SqXh2ehkr^R{d$zV}75UGl0B$8HMTRVU}XU7lr2HfIz%}B2VguvlQG7M4xYKYvIL~iKN#2dz8T67 zE~0(=eeC?6MwDN-QSI{^4@+Mzuj%1CIW9gV!tfb-vHDuim)+*xEzJt_=GcjYvYbcW z+3UyMDW0A5cRL0~C>>H(eE{&@fw}^{iNv$l=BsA6coSG@Xps zosqRXz!5*BtNNHbL&L)F4~QZ@N7 zjm|Twc6T#kDz+iowSr7jc}okyeSgsoDdKw(2L}8+neg@!tKL1e7eUKWsGRC7e(9GY z71h^eoO+j&!xvNL_}%<&&3R#>(jEw62b!Xh1Kov?Yn;sTjK{m%I>QLO&`e5>RR=1R zc@dezi$D<@Q>t7n-Mc3D9r0`o*t-6&!$3x7(E3N%pO} zaB}J+E7=x))u6Xs9-TvX2S;5fulfZPLI`%n548h^?GQFn_p&#+?h=P4jhJn6_sC#a z2aPuR@+|1NctW-)(eG6!r*M1Lo?5sgha02u(N)Fyo;q%q8|ZnI{wUsaU%t6QE28WU z>+pdWQhxna`l^r-UyS(!iG*hIK!-hN+MAsZfa~USfZF7hAl#8!hzorEk-gocp0jLwLi196%q;`dNf~EJn z?7(P`Jk|F&0L%d~DyDEN^`^uR^CToPpljIu@!3~)8P?mEtFk%OiUKG3;_aCqE_|6y zp6dCno zLthU0(C&yrliV#?HLI^#e%&J0y)Y|-)crBDU?poPO z16_j~YACmPg#6Q;e>&NF*LvY+)LKL;-|cK_zDxgw(}S00NF@ZEzePSDG;T7&-b zH!Q?HEnAgt$JpCNwPx*JW2W@&3n%t<-T8dvD?wJ_`13mu)F}m{>#^ej{q0&;1M<9j zNUElW@u?Kl1WtIyNJsIdVwG@1SkvWfEzBhjfQJIciVOi3Jz6w*49Y~?l)u(lvtU-v z6gWAKy&@k2w4Nj(5ETAFxh^JLnNTrMXXI(b_psCa$d@}5Un%zNeWSqWmb?B5yYuLB z*ix`U__?icw z)>z_olOy?2*`P#*3$WM6y03lhLD1U)(vEJhd(2dPI#GS$wK4i$#lXf}Nv4r!_m&^P2x921Yfc6sf2d$t zZQDVRE(vI>TCbFK*rXKZ77{+OnBaX#8bQ}D2&qh@fc*)eaF{O2V#cVU$|gN)jniQf z8d$3uY*L!AvG$QOcMf(PWE}QkU})z2R?n`++OO)v%=^y@1E>jKBu!?eKwxoHXq$cB zO)UPYIMw)Jmzj0Po;e>-%ThE5SADG=ex_w~16c}m1YKoPTVP}w+1q>8X^#!}0rUwV ziVf#Dt0Tn{1;0#8)Vi76>gjFR=Jt^m-uh4H31SJF2~tvV^0harB;wnWqp8xfid{wn zaJki}9aJcURA`pIdtIBR5*<;L>+Z~%WSk&!nE*%$_+{Loj~d|u)V}Gc#^d3w$)_&qklYu=H2Lg2e^B5j^z_0Pjg=fS5bop zG&hvPz>Y&6#)ty^{pTJ^8MzSGFue2R&r~rU={EeeNW%h}cgGi%hUW^O#{b|)iv0bw z#xqqttwrONY)kOyw@yt@knS@It)z5&5)ui2JCA)@ta67H8OsIQjwG66$Nt_gedv+| z_XVZ)O7z1ns18s^%ZC{KA$Wy4=2;x(ba6~cxtJ>9f)D9OhFsaAO$vDH@Yw(6^v-zR zmXc*hCIM*dH@g?3_*~?B4;nX3q?3xMj}bo!(Ivm%aRegOlAvu$DLNA?({uytSOGyukdq$)KcnQkqP+o#R>s3r4hMdrJcU2OK?Z&TJ=VH;R-2o@} zQ5!8hpLHp>qx-53=}f=JGT-L_j^WM)oV0j-JYkQ`oS z;KHNIs0Lc32=gQQ)8jY;)1 zX1>vHq!BR@MB7Q_BF6Foa)dP5xrzH6elZd&aK4idblqUr7t{D2?ylz5K#6kFDmnl@ zw+bIW6;SkH88V9JTJqN0_zfbzT9%f;=2M?!3IIj5OFjG$3P#&~8ObkvYld_=M_1!?8GI7)EakI` z^A*dXe@-C&(|hN4x0V$AyVaUU85lwQ;HVLbn-^&MPqSzV#M)7y8?O(~Y929svXskK zowbaPMd*Yh<$`B-_a0-=k@l!9JMG`OSe~=*J-LViv~-;T zHgm3D@a#-Vi~m=wHaFmw+u;md%%I|T!-nY!)W1x)2`b424r4$k(JUi9B~qh86t#?m zAKL7FVs?y5`YzwiA>cHcq;%#qB9`j%?5Zm8(R~^U1+E5D1Re=h3@u!5+9s(!7LiA7r zI96aP7VAd$&^zT=rgY`C>u(Az8@6d_$j%gg&B4^|ZiHdB%}&I>J-boOw(xoyE&1kD zS+ezcQH2Wk9iE`YI>XuN{A&^(XaAEejbTP242fXadstA?7P`-parHgv4S@X6X_9Gj zZ$0>h?+%2GUWWx=GbY5~1YdWjJvpQ?bcfLZ`F+dC3#sq!(Q5j9sy0@87k&+gFC^>X z%6l_y%m<))6NvLkO4X&&##KAuyoB!u1++W%DYWUh zCR`7gRL2G({zESG2;GF{@Dq6IsR&RK1sYwRk!lou3Dj#s`=bS1DwhAEQ$A|Ps%C}d zohpZ4_PoaCse}Z8cY##SynGro z!$|{BvHnKC4z^f>!ED+DoZ$(-1^U@}%!|=QtPn-sRfI+(g3N{!lGQ$TB#-$y=6jcs zWgeDq+t}kUU8-fum?uyMx6#KaKdd5kjA0}$O=1Cs&II(b4*=GG0Q|F;Qpojj-EI6% z`a-z5(607#f<4{WS171Tp}CJQ;#HWi@lP2Q2m zx)|ca+5*sYxEjDs&r$;4<&Fr3DKLXedPG@Sb!RCi-VVd5`roU}Yk6Wgn9_m>%?rsu z(#iPrUm3$U1q`VLUZVvgYiDCu>~DrC_fE80x!wM3@14lr%&u^M zF;tjEzE@eJR0Z55V%*|K2tD|7Psc_T&%ysg7lUDbk)6qwlG|(c4fL)RWugMre%G<)44@F zlLRpc3Uw{hK#o~Q7+05M?6&w5u)idb^Gse1+NJ04E1`U==f{2s{wUqPQ?dMR=v0VC z0_a>4&}1(>%VPfC+L=oW_rYbFZ>1jB49ObBUq2sDzbd1y&;M64y$M5vt{8xqGY2*L z^Z#}I^**qE5lr+K7!VwT4n;b)63lU4m0f&1^+!l~r{N=H3+!p9CNPo@TKCR_RPerF z;O55#Wa@yc255=lrtRr^Mon*SH4#_5KR9iBil~ogf$QVSoyc`|SXnLjfx5{tx_)^~ z{Z8{2Q(EHft$qT3#4r}l1l}(U(7%e6qYT)+dPf^-H-=O$uby19Z}D?-DE8c~7WxaJ z@h}x&`VgItw{&~vz@dR}jfw^eQsgb=npbO>J}?gg z6BAV$0cvrShs1 zOoR7AEwG_qhjWe9GdI}Jl6w{PJ*FevpWnq@{6!~R~b`2zo{^Q59__XmzrSaFld zxjHGoMpV!8^z2qAz7jPl;VQ5*I6X1IE)!!3BvHlSK#%Z7bU%;5`!J=-R?(ViIz?#< zNxtEPL<4Wo*HjHgy$%`g=-o(s0FQ&m+V#V2$}`sYpSLP_@a7ZUs|HOXf+a7jX_mWb zNiD0MAKpV`KsGEFu2lFMWHXsOp1z&@vdQ$&TDt2|t^$u@@8`F!dUw-40Og1Fa}m0h zqT`NJdi^(*a4mFNW0y_?4wYS^ABH9TuF4VLhwc+UFTgqrRUUP{0EC>_46Os@N78KA zrDQKgu<{HX_XbXQN8JIPuUBFLFs?A;qSs)TY|R^55Aj9Q7M{ylpf2GmwRk%4m2QCr z*zsO$vinGn@-c{6AVpk`6_Ao7Pi$dmc@c>#MhEt%u1e#?s?YmNIv`}s@BrFX@w*Te zja;I~;6xD6K>q-Ylfy#0(V=kmk?RcAm5^d}*(Gog?-b1a^(nUdO83fk|6bVg0WhX) z@lv8;dWjm?bw&lF=I^*8n2=b<)D_Lo*6WZIuFymlq>4O!=|4{wej%*`Dku1h0?Y$! zr)Z4>(CVhFbEK@40ui5oGnQ3>20{p+D~$(}%_9KhM>sv3eZAO1f!1q<{LHw^Qk4#K z^!%6O2wH`dc67XhBiTyzpYUhu6hw6rlutn31{TU6{^&jTW*%XGf4r;ZT=9d@Aveyn zi){Z{s$ipkE6MK$K(Gz47fIlY5f(rUkP93~#U{le*GG7z3RJ6-D3R77f$u-AZlXDo zOY+htJ~F}5h)Vx}h3-hsPd3K$!msT2!}_W2A6sW$I>mzYUIBp*?=w4yEVztl@H%h3 zExwymZdsBDO+zeOnNXJb*HKJZ&3-A{vtm4k?91c&=lZD)k+{9*?tX&g^tN>3i@!Y@Is#PW8`eG^yFkAVP(6qxi&6-XbuZJ;}h0zKjwcpu*us+MAz{ z0J->*O-CQkiEe)r(?_raatc7Q#*9dj;Gol$pL-=Ut}ythy| zjPVeqwhw))QosyKN+GCzTf$$BP9pyO$*2BO1&PqyvbwHEFeki4fiM^S5{*u>qBx$$ zX5i0bfttO`tNfPHx%7-3{EAsjN~yaThVmtd8OUtIuDS}CKc(vXq9V&I%AudA@hC4m zof#NP){={P`x20@K42$q>=cnav3-d zLnR-AUG0=8Nz(+YIP!z=-5&p01fshV8b<}UV&?u6L4$+Z*Qv2@0XdZHFS~1JP3vgo zH25AIW!t>4EAmO%g;&>;p96$Xz2BhM;HQ9P&c{{Yr$yMyfB*&%GiHi`e}xw5 zXnf_%X3rNKzusQd)Hb%%+Kk#e3%_Vm?Ql#1SEyDc|HLGcE!!U%X2E;W6972|YdpAV zRv=4mjy=Pi>^GB!!PnkdoR2SbZC5#j%?;QJ6;Mm={nOtA6r>3}gc`KpLzcnbv=wdY zmyJ)1U4Qo3h=$cPM?G&Xp>6lprQ73ksZQ$Re_13K4wz`&6+$_?iXEn)&Yz~6BZ88L-|3USSySd7Wm~feX#gf8}6*jb6MM7?Sv)4~)qY#-yP5_aoeJR8)H~fBFPZySlAbGHrza z^l?&blzjvMA2vx~;+6AuZef2{Nnd|D^}XRxB@#sH&#TNo8k=-;FLEf}7Dvvz~G`k>u)52E^t(MVvZ25XJ_`10fGnmJRO5 zK3V;C0!f`a>7U1lwW9q0)(X5O?@zU!cl!$^0n}LRunGW;xUVE;W3kij_#3AZ=}W1sYovS)$w@G-&*oyYKeSf~LY_iB3Li-n;G}C;8Y)D| zl@$RZzYQ|1PkoF*CQ*4f{8b?=FfB#oO7I$kqfo?x+t54}eUvqe&B?jEAbnmZ{c1bi z;2byh=D!-z66W5}z#+*2W?(jzS!F`D1pM~5MNJ*L+%`kXL?~gP_VpF|DH?}bN%V>< z#UW)g5(>Vgx)%haQG%mh3exx*%WAk>JToAYp#`1q{&%OmRbd*tYRLUZ+V?m!k^Z$3 z0DoAkv!LRe>5hLG`_Ub#Mp-jfd zt6(HwfUw(pEB)j_DrI2Kj;Lw=ZMvO}t>)?PKlM`c1vY&x)64$_G;By zyMpT|uwMJRr_!;V$u*N!2~g#+f|~+|_}i;LK}JB2cGpUMyo|qUi2@YC#0--_H2HsS zI$yvz_md}8gz<}>Rpeg?Ta!#x5id{f<#B6?n_5F^P3U{ihgdVxqcz=b{~a0c4Q-v9 zTm1AV{=KDi@eS9+J5fjsH3Gj1OO4rd*xcd9w_E3vI~(#`$pjBNR>bcgwlH1TF^v*M z8{07fIqN$~3L(g8h3iShv<>HO)Y>B$`;E7^;Xuo0C7=IWA2s_8jR)d8ZODKXhL`I3 zx{@WXQ#VECI>KB2XFTB^78c^IKGyeQTOo^EOCE@PlZX$H43&0WH8sWM2)j2V-0mgLnITtvDNpNO4i?3(wi(&V&YEkj_J)9W-bI z1G-LbbB-IoF=>djsYCt-4 zc?A-k{Gud;lRzxMAfwRBnKd=bpD;7y4>LvQGJqgX1@Qh3-)?Rbtg6Wx2yRdF{&ojy z3jmI@GPI4g7v7mfRb{}Nv-WH?FG2J6$m_p8-p?1?FG+!{W5A8)-6-`pz+`~D1wCq` z$i9BNw?FoE{1*JJhK5YEEUp4Y>_e=9-%mnHxTk-jeyb5MxI&B+y;K8?p4J|6Z#-@T zuzgH)0ixYnSN{6j`uC`As8#$ba}n**<%mW6a79ueASv>TK?R`>>Q<9?e$8^jh$x++ z1^ta^knf(aEkOjt5H!CZivzO5YQytqxW1LWn4`LQW;H%6#488Z+k4i# z7Rv(!@@H4I?)^2PF^I)N>A+4*=sSORgkh)k@=+GcDQ5siwaw>8ynb`&bWT%Mk}2@g z9kiQ#!9ICou~HGCXujVcWu?4hFS$LeF}j^Bk7sCr+oQwe{7>tqqO}QKe-Efq44S1L zwdbIzu}bJ;az>KrTN`;)Qm=uJbP}nOXrC7FRy7j}?wam4U`}vpu=$MOP5lQ>zK>sK z+}!UNJ6F-y`s2H|u1>M(1kQBF4dv13^d1<;;LMh zh=b{FwD)g7rMwu9mOh$(PT?XuSjZMOHZkg$BsE6zIKdy%Yy-D<|G)UyiIboca z4@5)g)#6&g)im}@Jg5Rt<@@HyJ|-PSlA=Is=u_6fT@4CT5UdMByMn?6Yde9(T??M- z4Zdf^N2M)V17T5lh7SQ~vyPsI7sBrXkgU*oZD;C7whNpiw=;;h$&Xg3l7?GZp%R2s zRW{aNom=zNgu9%@5&s+o1q>WG)f|uV7n23NPh)9pZd8%9+b6o?!e}IsgFBT3b&i}b z3<YpyDD#|o?&V(RgEfn( zFepV_DOLYXT>7zZUU1L536RP1UE zK0F@bY($}5m!D7z*4N~&*I)1g%Gls)5m)N|yG~hk$Ls6C$Eti)yHCq6&>y-%flC2I zXl=pYi&sks8YG|+6e18K-K8jf%9dIicid}l->b#C360L0Pb%t)c3k8gK#v$-&G|sj z_IPnsx9_Zo*T7DsLU`pCB6#d=uB#HgB64t_pP~|!WV-if!$q3r5MYQpG>$DZ>Cuhp zKc{H*Aqxth5q{o6G6|GEI9Q+&swRr^!#Pwsd>p2Sk7A2&b1cJjkuWie28dm6y$iwH zf`5O3j?n)^+SuODwU?f=J2_`f{0p8RfVO9HOf3L59U2hO>aEeP#{glJ3kZ;f)bsB0 zf1uur-}Y>ZA5@zK={ES2E|i%woAQ0QuOEgZ)!Z@h;ERoxXM0uvwJx3hX7hqEdu)4R@{-#%9Q#h+bLTqoH z8*;-=FL%)$(^Xx^$B2@5N1|OjRh&+?54ARVBh3AQbPd`YBO>?$|G*sZzeGo&1qrw{ z?6bSCV$nlAi)Wrtqzqe+xmCDTY?Gb~FYISVhal`Q&;A!&oJ$4zR1he>zO!szzees6 z4@YUrXVm~H0ng7s*T%6soN_8?ch~e;Lj#c<5c&_t&srp;hJAz!MmQBLLSKNScqjR< zhyEvFXA*%9f9dyLx8r8N&tim*0wjSw2$5<%6i44jjWxZ?`_wZWM4p}|!_mcgL3FrG z1+GMQ50|$&2l~0_%|)Siy6qB@w#!qhG?|&&58tO)Rggdbc~OK7xCyCFDFd3uHSMt? z^L`{%Jw|T}yMoq?e4X5vzC%$O*sD6Wl}CZYtm%n;Y*(+r50a4PR>Ki#e-{e|@dW(o zrrh+(5!gT^Z#G4)k|Ku(QfJe-?`f`Pw|+T`Z-hxs`<=|L0}X{gXSAm;S!EHL-w5;` zoqxIU^yf`j5H)*gILQL+Zu)l)c=tV37Ba;P0v;zN(6WiX@eF21GWXauh}G2}FZpoO z90T+LN~KyuJUJs-E|2R%9%WV68|+sa_h+A(Ta4>TQ;H45rM-hQ0GhM_5i;v95K6OE z@3@k-4RfNG?~AZWsrLg(3ErBd54d2s{PWWS{1O{j@n!p1p;*NPpQO$AN_AA{g)b|4 zt00`jJP+!=f%hhmm8USl*%b`S5Q+ z35*DbG)k2}cah?iw?0!D{&9F3vbMZaKTFw4d~sF)QBQXFR$sMFo$Q{8#;B|VPKZw7 zM{c#SYBlxH=d7JrC?f0~d3CqD$!k1EjG!(HQ~*!YJvDe8pP5>C-l;F9#vlX32+F{~ ze8FpKN61y0(r04?>iZcW^Ssj6!+`!{p{K!Y@CeYuyK>0c-at*+{#pK1?tV0nDP(s; z(4mL!$jDuRRVtafH@r-$nyV1tpoOF;+OPBF&!0_jWm8P*g%WKTfi zsontc?BR@pjwf;{ z{4!^ODm|F@B?xDR1|*HCvoDxH&IE%5@}~VUKHcv8nWaQ`ZfMzse)AkL8_6a)TE`iD zANEe%G=uv|_Q%_hNE?zeAlO2SKUB2!BtGsdJ{<&%lfJx9-L0MnWrlAbecu%P90%|t zz`4tS@uDg70+yWWm=PYqwX0Qa{n6@%k`AO(+7eSYEcA6-1gmoZW+;v(ljn&R_9UbH z7WEy6!9TY#$Cwi9^z1;U`bePCGNHQfh%@~nLfrz;OjrzGax=PG^L9g3Nky+u-p=B` z!~DMn*V&;cpUfVI2OfxLJU2ZOQ=E+jzTSfVrYK7t_&tXaA+dyVC)`|2<@q1T;m~Wx zPx3i#MY~Y@RXRVU$Z$@FcZQ<|`Mv)pD*{0I^kcG~!(prA^uppVD~pn*ku=+o55*XM!758Z68ex4q1go@+Z4*(>2q<|Af&GXw$!R}GyXt%o39up|jV@y@E! zO_(1vZBON9hCIEwm*;Z~mC?ED1AW>Ra*CTFy|P3iAh|hZY`0|j*SY!z+clBZ+{2Uo z7aWVYST#_sKJTWVEX&PPGDm;+spw70@b6<8GryMmtrW$Zcb1yh^FCCG$;FSZtZ5)n zG5`w|k8M4^#Cob)(D~+L%;QS`Y=709m2GC(^s*Un3u)o9aK0JBJDHGo-%VtMkRKrE zf$D?xTG_9+4|GzR{&WY~#RQD6a)k^C9-jeDhYKe+g~2Vdh9EQn+XJ%YW<>`g=KSPv zOOvZGAgifKy{qlj{5H1B%9?s++O4Uc=+2`_wy>Se80Yy+h0L3VPJh&yFsUB&=cW&5IQ&rHnT6Hk>)#=zZHI_KkmuFaF!@H|YNDxE!x`$mk$AJNCz^oU(VL7Jv(_bj#eYvNFqW?S-6R+5+x zoul&pR?oZogeN+Z@)p;uaapNe5yg9~SjI3Irp}Bk=B>i6*|KhwJ|K@T3H%+uulp{H zC-Fx7omWV>lMEQ3vbvi5IZfuv{^D0tRd159*8WL)t(453*1-%yo-@&cvs8`F@NZrU zi7|)Us>4Ib7K)SgeU5bfy6$9;iYq4r_|Va+ArH1{<4BN)6Jdo6&YGUZEGHYW7koiJ zgc49^?tXJ@9b)GgHu=-YOkH7{cKkbn()aR!yp+XEaDmwb5C;()p3m(K>b#CH`^ zT^~J+q2_=jx7En~4G(L_r5p@*rSBF#6O?i?i+&q0KgD#kbesJs3_Xz?81juu1pMTv zqI=U${0;T`tof}LaVkQ$ASf{(vu{3OQ2@jKEG1T1+S?oRxKun~r8N~RM5@|75Wh}8 zX}Ef>jZscLfEeRD=`fsqtREp9GYd_ z7BljZ=#LwJlI`+)f%-52n$tWQ`aMx{SSMkh?a~e8JX%aM@}rUiC+W!7WP-i8?*f6N zXepRAt}1K6uMg^(qs1wRd6oA9_Sb%pA_%*U>Wvr&3-}$1mOOfdg(-EM7l}o&I;RVF zUr&Bppoa+A;<{cXc#Q=yhJzL9(gWeq$|qnu_VKHU;7cCyh4yYkAMRPq@2)6BGo6fhP}kMJjvGc2+{sfy;n-kE-0XNB_-jVr)?o-x^iY zN=w|W?p&ZT^kAg4l=k*om_b)wAfB}pv2#I0bPy01J zEyo4^x~R6w31z{*R(1A($hYkm)c%HI^brhwes`~_|543{rwrf6aIsD&40RU_XH8k6 z9^)D@<5w37fe9q@n_mmYJYTnJb@MAz<$Cnq2EUy?;M2($@E;=qP(|8rAMap-9Dc1* z{5dCMN`IfcC20wRjG|YCEc~c$%RARMmM;|!AsU3|zYH)b^vhGK(|#TGkjkm?t=JXp z%ey}-rj%syZVMZzyq9d5=a#*827GD=??^3v0<0GV-S)ge0OAnGN;Z@9e02%?g0{HKn{Ex!q$%t> zCrU_j^9{o;^^_}^$;#xCxx6WLAuPYV+al1t<`gg62VjU!8tp-LLKgax3hkpr2R;gu zGN9Q|E2~v8IF3c|psQdk$$ov4D{j`k7D$VONW`Cq4~T@n z2vKD^m*aJ^>XOvuT=5e!o*OjK=(-n@6=Fwux}E<{hMleaM#nmim%ih2g54*^Yy}ll z(wD8m$!A_kCRc}FKjo*Q&UDt-i;Ip_(iwMIo&PMmd`H~xpFLcGzJQng`ZJ(|71}1z zK!JuOU&mUTx0_tsQ;>A{!L()>-D@bCLP!Z0z&c$&L=vz?8d>K3^%xN3w#`dh0#`Y1 z`b?fbUTN;W+gt)Ul91AZ$sV%3+<)ApND6%rgz1IcU?Io>=vh+L@uQTDtn*sE;0L{E zT0zL3=a<85P3_Xm!8d0pZ`#-f@hJkilW;Vvfg&GD(Fn>-XD8Rw=v)vk(b=M&|J^9a z(X0}0Eu)L{TiKpfkAG^8U5${@3}xolTPfQjy>K?y5b^~7WkGVHUvgRhNl&BVU_&Y; zB>CUxkK<-F(sYyr_ShOK_vFHM;P=e)QmnRx_0XA>F8b7D>Eb=A(sg zThT@y9rf#Q?I#q)PSXKYSc`__K(8T(VITE><@gkDwm-SqMl(}h( zy6pT?V}V_`b28P39cb(9;3wHZf8*j6&sqi97f+g~GC9h|ZAvsr?g}?ip_TcZm2v9A zdS-Fer5QI$S3G;O#TOq%f+EW^pP*yT7OKyyr*RX8qJMthe}8eBu0Xs94pgf7bf#}) zw9g5g>3`CBJWs`0`;Mz(aH+Ox!J_?&T)Dk>%$8XY!^qJrHANbhMVDQVUrr+L zlW;d$?lQ7gg6e4cfM*scxE%yv5#q!3=Pk-QuWlQ5?vu(aC)~QGp!(&kv~5ZNF)OUz z+a2q100rbd9&6M0lF{#u7OHB~>8~h>K%4y@S~SWyjCd1CWxN>_$G`o`ucm=gU|*Vs z@eX5kMc-0wdSZ;U+IzuOjw+&6JpWC7MXm;hvVVvbR%~AyUFL$Jc#;`oT(A-R;ZkM8O5aHBC0_*da!!$bez@JUPGgcBAw~IY9e7c!{xi|VBTfo zZ5EJyJ4gz6UT}Vu{iUMmUBn;KTPe%fp>=E7PJttD`>j-|X7%Osf6BcJCz*^JH{|*_ z7Tqtf<~`sp;vtmAU&D{Zac-mgtm;~Xtn2-0O5-|{CQ}73beF2WFLEu1VNbiZ#9>e< zIvo7P=QeeZT5L4stsa%7rG)FbT{g`rx@l4LJH5eLJI;Km6jMn79)uYxX9FJdTa3na zHJHf?*hx&ld`-jtPVML34uiybvriyrDuY%=3xBuL%Vj^)Qr}~vB>)9q;5@HD!Z2;- z9+V_6)P+71Qh?VyH8LjPHtjElc)5z5u?!2{FY54!AQpn-J*XQWayD_NmUzz#Q$vfH z>D?UNAU2*ful({_^Clc_$IR4V~<* zanLZvo1JEY50AFU$vR4;IxI{J_0dHc*{1z1(dFnmcE_REhk=p3jG?q0(r@%-m1oD^ z9(VA>M4COhEk0{FRBaB&b#6w*8&X1Lp8zhdF46oZ8cuzV_WO^_9Q)n7UPGAfA)F`F zZB#TW*c=32ss`-G{RC23m~<4>68++RM`_H>5x%}@)rdYxpry4oB2>~f^5HN^KJ9S^ zV)V7S02s;&@Kb$0>KU`%vp{E%abWUu_WKMY7=MwSc%3PkP4Fh+?E+~jK%Rs4FlOH} zbuj)fJmU}e2yw_@;kn^8bI%RH))Y>QcqV_iclzBig~9Z3QN2H~XL&nMuJYD<;O4XRThrIT#da|-V|Zb6B3j+Tn*Jhrlfx#ZH)fOiz~Z6bu$yd-#7v`H z7$u+r2w=mNB%~7sqfwCoIJx!Euq1$etjTS6%l5L zwu`gk^Kfcb1UH1aAN?h5R`{m}j>h}FtwNDDOoYAdfPeU}jbOgkZrB@3YJEfK&d83R zyJu~&%3+!?7^8i+;@5dn6ypBy{dwLVo`rx!R9bUg>+|XoT_1_7m&Hi{gT^oWl{l}N z)t0s-FUtWHq|O}omeu)XEVx4Ez{U)(`3h#yo`@rxYD8nzOH-;=Gu>@>9$qJS4bKPc z#;-QsH0nkXXed9g8X8ai;Q60+SO=!01kkIXM%E{e_F$~$lOZ)G9Fh@hRB55sN&Th{ z!74uqD75^`lPudD~9{QVoxZXSe zZ6~&*|3*qi>wO+V{>J$e#&-`Ssm=>lnpAPjG`krR_JUV)ZR_$fYb>L#zB&Js%o9ug zFAQBl?O~r}vig7ujYJII=!Oa&XuOz;6l4h6+-SPvpH(uq4=S#>&}4jm8G6sLq846_ zBTcUD6H)*ORY5ihJ=Kc03zzMndoljD+K|_g<8q49S^``S-^0M=kQ)o3bgFxuRenUu zea2yb#lJB2$-uJ4(GMCw>cZVOT%(=o#v>_s$5-NO$f7ih>IYSbNZlB`% zsH8jooiCiuOY}f2ELp1DxNF#hU)IT2bR{`Q0A*ga*Eses+rs?GKjNT0CER*H5x%K1 zrMGciafdI@Oyu2*xGIInBxMHY6eG*0x+_UhOEWURfY+gK`348_m(`MI%!of*v0pvD zi9E9Vx@DW=$*z@qgM&nVO>M4TA}Wuw^7^?*T5FVZaruYcho@Z?B=1%&Iim+)_PM?5 z4n#sV1A!H1vdFPjnQqyBzpDue*qamO7Fwy8;ag0SlolX@{f8egk4}B*ePjCo7uqL| z3{L#Pv;lGdgZp;d!?;)VTBNEqtXT=|f!O!_NvGMgZ4H#VL!*#6o5L#x8g!1wDlLs6 zGnhNdh35h9>HHoHh{p?du9X?4edliC zp5YY-JyEL(4f<>NYJ1k;!K(8zDGS_MczT{a%{y=;h{bI>J_B2L_<#(G_uut-hJG`d zrX*A1ZMyBFHnzLLfcGc~?N*ZHw@ zLe@9E$UEq=z6${~rovaWBtW#Sg7eoW6fMzJ&D0zBlj~%)3onhpN{xJFYK#NlEE!bE zf3tFXGtGqHZJg?O<#1|9ZIWmoZ{u>-)Z!{72Ku5ow>v z-L4CaFUIMcLH6p7h^oA_hn%fiKFi-g<-@G`w)APg^RjKZ)zfh8X-NYABadtvUEMc& zlsBK)Tg%gQQ32KhdJDdc;{MSJT1IB%uTm>!dZkR$!k@XNT8K%a$izX_hfxDvbk&538?PeoOtkf63pPek60aR^COrhJQ~n z&(L{R*|xW4u>XA6j5Bxy;o|*j#Pf_SaeY(EIOS9^MgiRoU8R%p(YPw>16>83!b;1{ zPcx4{@p$0Z(od^>@Uda!t~#?cDf9znx1mk_ z^BAAhlBYg6v;c}s=Y;y%v_W*GE?kIA&t6)7N}}LnCk_8gI#a$sLz-cp4PSm<+V3q% z^yrOSx7dOGGNM|$ocw?E`KgcQFrd$DWB71#B3cz2oA{NoJ>Ex-ec;vn`*NAXDW&8` z$g;fhzze8~#`!Ot*6&rv3Y=$^12NAA-Gl9RB7G?}B){v+pCBU)iTjM9z#4@;1HwH@TlLI#ZD+c#Z4mHfau9B#c- z2n4#}j}+%7Y@5`@+nQdhMasdTbeYCp^g7{U*Hkp!56%=kJ^JN=?A;J-@!9yZ-%m2W z_z0in@7gKEv~EJdce&Hmx;1MVLdm_ISZ73}?JE%fL_y#;L4Ip~9~(0&Y5_mUdm(Hn zlc{9>lXbGD06v#t%k}3w8b_!YtvONr$?hiDlMP=g(rzzZ?*VzFv{nX_|D+%fO}dZL z(s@YoT4VjyT8WlG>TK~j5TCR{DRbFs9KTI-6#H4 zJwi8Md9WR(8+3XQ*XcSmR*v_5IJ<2jmZx)#<`NB$G?jvu^y1BHaI{U^iUj}|&cOo| z*$&^+jX;$__cK~^0DC);%FIX>8Sfo^oRYUXgPAXagI1r(bFGoCQ0+&zYa2DX8f?ES zDqNa20kjs6xOsm-K9YzET!Ir|`LSlt(>$FO>0LE+q4u(Yy?wYpvXIK#mNZ`DdY$1Z z6%XUs4qZO!Z(GjAnG=*qKx*lV%JzAxwwRSxc>0Ywt20W%eCmlwe<#j&y*S$cVp}*r z#aAXfs{K@^Gi@KT0E(>OR{-~rJLLU$+8smABi*VBH|0-k6u;}s$FH6jS*lvxqfQjZ zU33-dOwqjBO3w}bNRgR_wUO8(UBMn94^BA_1k8OnGC6)O2!8@5oD_f|4W?5);S*Ac zi(98NSX&b1T2A)weMMh{1ZvO5D|@2uDH7r{tM(*4O5dU=opEtLoZwpv2RHK~Nq%M2 zRgD^<1p!x!JLj_|aqO4QDGi5%h1BIv(gQy%z9 zPO@2j09{}$RXu=(&#E_b&h^31wSPRh%c>)jK}(bx9jVX-;G75Gcpr*)Q=O- z7ASebn}6J2d<}l-#*p+_IQ^GeFExZK7s|RNA-IX-o*zzk1S?g(-P45!7(h#!)%PXd+av zVee@B#|Rw_&vWfHn_#>;1afqGl=XYFVf7c^FquAOio?*FwBwR}1vY>WU^Oip zI~8`6T1;4@pL@bLW7K;_sB>jXDBsS1hbq)njg;7G4p4k??I`4tJ)0UfWu0(Ki)1Nf zEA%=x`>;;ws|ShJsH567cIJ`w{@zAx+t8m-L{e07lJVo0QZn@(C7bLF)U7y`@usi1 z&N{yVswd3WLo)bOWJ%KXiR1mnTuaqY~VoycsAfoo9$>ar^Y-X8?G)B@mKKhb2GMbtg{xz<(8 zgEwN{;s0{vT9k2!T|%#ccPI97>~nn{X80q+&N-F%;vSesf{270d1u5q^am1*!7SDe zt78|#V{&{U8_e%S+NUt`7x527jifqn`W$`Lu}vvd#%I+Wc(vbOe_%KOB~5xNoYX>o zE|2AOIBUztHPU+vNMC3VcB21k28HE&aU@WkmzSrrlG7>fsjtav-h9W+_VEG76)6np zCqLwdGmwQx?-M+$9m!%E^W?M4vJM1@*PUYzo~2K$*I8(7J>>?^sc2iNiJ)WfZl0#> zpJOwv-Q6++Jyrg}7ZK16TA5x%pZny?fZWe!?$hUm9V3GrgWc&J#@s76ZV{vJ{N1Zs%?-}nzO(?@sv%Shh z;!oh#tM?r0?sYD=X~(Oq%HK}RH~s*W^zI>SS@py}73Wl^k0kn>_}%7HdpgRK_t{iv z-I*Vdl`O*ahrflT={e|4_v8XC3~^)09+5` zpiB=;<|hh;1t4vq_FO2}eq8mQ)|G-VSvKf0Rpw-k&Xh&4NqTgi#k{`iOQ1;=uh|a$KP4D%>EkCPR()7kULD&Hq^%~h(f9Z)8D4} z?j)Y|ja;gGX~p>6JXHx@PZnYZoDd>E!igRBds9d*RCpMPD_@@xKN~&td|dSg5OV@`z&X%x-}EG(By!U0%w|T4@3sHZI)m2sjkkdd%;?VdrAqTPq^fq>}(jX zN5z}s+$L>9m8h9bTbY^vA2ptMBs%@TETf(Lz1eM_(;vX@E9H<)b_=})F*Tloc^Fuy z915N7En=GDjZv6Hz`O0jmg46Hs(P5^?ka|BR(09mjkxz8yMh@V$kl%41Y$^%;aCXE zq4l0~b&?!43PH?tq3VR)~(6|SuZOhSW zF2|WqIeaJjO+X1xOx^vKa34A2y->$dhp_?8<6e(v{_2CtYd>{HvbL(o>@0OsWF17S zXo>AU0J3y-7F0;QBS+L0+O{{Y7za{2P_2{R`{BEE^dtN-s$8G`)gSVPiWxobdtSq- z#Hx?p7J^gtUTm#uiN<1q(nJ%YlgS&?dT{vO$_3ovyE(xo8_KkVVTMn1qXuhb3)N!C)9nAba9MFL5)}qtZ+QAxXNg+$=aWy`iGArmlOFkoDj~Gb8 zTe%}~SY8g=BwL!O&}j!;f1-%{1MjCU7qSAYmxsQe~XWbQa?Aw8q>uxmOt zU_isC^KV)5TK6w?HY}eP-%@N#ZKETI$hLU9-74i(8(cQqnYP zr(ws%bJk>H^@I;1Ys)fzl?Zr?&)`Mb66t#=ra}W8?l`{~*<$K0YjOpKLL1260|%8&llg*_dA`i}YOxchWe1ZqMIdI!i0HW0%_%c?cjRn_S?-7|VsLv_V13B) z1X2mQ1qBO=#CNbTf$FLLK$5#E`N78RMrGk--n9kF)RcBlf3p|a^pI)1}Tfjd{ z!T`=f3v6+JbvnGvs+?g5lXbvpQRX`aS*r0Jp!1Ul-KjMb(XZ<(X!ugSD*s~J`guA1 zO_$3UOVJI%@_6ikMUV%+u|%MT3RtVJ)vUz@Da9hTk@zdm9doS&q>s_Kki|7>T!0ND z0$N>9pT0ey?PefoGR9>j*cbZA9}$Rr8s0n$W?C=)ESTcb33_~u{zpEAIaU=HBf&F) zuD2fmNx|nBNzSA9TF-RiOG>z_Q~0H4+2r+cE(@phv}la{d#Q#~?^G;Wi) zl1ftOLK-Z_JXZdI_z&(0dEw3zENrB&%;+<>&Jwvz$m>*g#3?u+HF6uD!)#<6BcKcf zPw|QJFp}47h!-Ry;jT2`bYs$WQ+CwT-Nc*yI%+BE!S`+yZpCxV-+(HBgu+8sLyq~+ zMP5IZgf21I@b=^l1&C@HWa)TD+{A!mVF}vZ@1Q)(yl2Wzfg- zebnKTTR`3q4~+|0BX_q2W`amYmndx7Tws0app6N?)^-Ig)ad*wKFYuH)cF=Qe1yH4 zQ`P4o^dGDjndu7IYy4wUMb=NJ>)hUITT{5@m+Sg|CkV*}v0DzhpOW^Eg_%ahlz8b8 zpEKU~5M($2G%dzx5KRQn6YiY-k+w=NyJS+&$FKmsF{Q%>#(NbDft$)kHrR?$T$D>i z>;7&+1V@F%n1%f1SwGhvG$HfI0}V2EZd8N2asv*LKUl_nzW4m$$pPbNPqRK8{L5-& zXbPG-t_DmmGS%uS6lWNmf1=I7g*d`?oM)lbn9FJ{9ooC?77ENB>?EHN=8l!lRA%*( zkqBaKUADbVFoU!1PqsjfFESKeee|Vm3LI@5{DTS{l1HpSWn7-H`yhk)sxFNKWD5h~ zb%=BFJ>^sbElF^j#x?XSnIJaU&UGKx~6Hg%tVxyav-!Lh=Qk z<;MeG45G=!Awte$+>$TE^Z`Twz%ag8{C#c`J+XBQ^#ZzzZ$ZR+CulLZZa{;XU{WSZ zLYFNtJcbE-HQO$9{OALXi8yO`Gjjl-y11o0MSQyLmU($DF~osO{S{B!8kYWrqPpd- z%q~0k{}zU(?BzgH((m|aTTtah|F03W@2S9yl2os(N>Z3ZgxZgP+&^v;gpSafwxA93 z%eK-ygj^aQL6Y%we~e@MkTdFNhM+2>tp z{a3Z8ff#-;UD>B?p}GaU2nHzW{%$~ZK?$=2tS~(X1-82??EkF@rr?HZ#`K+r0w}@A z8Uu!Dwo&cf?fynKdMb5_sdTh+RJJM5P95S!iS*5GU8Hzk1fJs6YnD8?vcQ_3alh4* z4-Qy+QCVP;;+Gn3`d=4`iry(XyoEm6i25(Z)XRX9nyn|6ihM8GF>^$jB9Ih2vee85 zRt&Y>e=7-`2@z&sPW{CW$k*d&Grzu&rSaQ;a+`t_(vR<{Rj^mYN|vMnfSEf&OU92n z9nsUk^kkH#3{S1E8go5@v;!^h=70L-P5}eIh5nBgI3V1ldsF`!h|~+hK2epWjYZCN z;9FwcSEY)nQkv}w#s3uGFU(bq68K5xAZ38Tq!5KLvGbdJU=3HjA5HKkBjgH;sFN&xF9Oj;~yuM>4#zzBPhHfLVN??7K5q&NgiFsY7?+U>lQRyVBd%+ z%ZdfXG-<9pr<{sM$Na6z0>|5R8PAZGpj-Gb@9Sw!b8OHHFaV#oOe`?|I%}*!jVAs0 z7TRAKy2`VZQJIcg#3*(_M+6>b0-LljyV#~7rTGEWmd;dlF6E=;?RS`Bi5ee@k;XKz zRyy{Qm=fdZuRsJa)46#)t<1ou89|u14>ZK|6xTH2;808AA4HVIq&$Y8cwwtKu21bS zoS#TsD0zIlu`7FWglu@T>wOzv8JQ{OC4J3OWwB_byDpdSmkx_$=CUf#OWlYPj$$$~ zqwW^s6<@@pPJr7*lxL2Xhm}F95@9i5#4+TqxjR9(^1UC-=O^yA6nvPo5(@iS=h|M|6xWhue< zUT;qh;LIx}k-<0kmVDPupWYx|@TN7bn{>>w`x@@TPtkn`t1~60Z~B?->FDzJc8o@r zN0YJSVDso%(cBV<{laZzgX(a8IlHA?bq4W&;1f@JQ$Dd#pJBfKboDyjHSem3eia=d zBZsF|YyAVd$HgNOwz2|OxA_o`o$G*m0V4rCZU#qP$Ltn9{*=&g==H0N|Laj3vE3eU zhZ3_-N=6ESgJ=w4f(Xh#_pq*cupo)X#yaM7!$-{4FEr!v+F}sU(z(@aeX=F;5F1K0 zdK5~bF?!&4;WCl<`d^qDjS0CTm3XEyUv#v@wG*wljYKdMKMdQzDa`!0QdDD*nJirGofUp;!oCC`eVBn)M)EHmO2_`eRCq0veP51ypP2*JK_wnO^CiCJfa5=VgQ$=dN z)X&T9`Ldlb!-KUlcQn~YQDeQDMtB2NE=ZeL7!+`ZTSxJ0@71*5!y3T&W)PKayc^0{ zJb@iKfk=ecmhk;DHpas|kGeT&*R4SsL`9b=0UdStPX6H7shRWDbwC5>n-^dQYE--f z^|LmjX{?ar1!b)L`vm;SRL){KDz83elsC(}K+|8%h=H>{_K6Z82m+ARP_3yab?Xl{ zZlOh)nDmD(DmpHZ+GslU-^$FYN)i|gv4Ia&maDway7T6LnFw4C!x9mOOZxGmtFIMn z8k1sdyw*~d6pD*tfDKV6@%7g?QRDE0I!Pn9m*Mx97dnV6DS$BB-9_2DtKV+`FZ?`` znl+AQ^QBhlhe7!1YpodT3pf8g3Ejwfo=kmZudl^btDfg2wW=Np@F;m`{q9+oQ&PeF z5H`GQ*Et00M)e;N-o+7<2b(FpG%YC%oMv1A4xy;p$>fIy1SeiOB^(Q7%st5#fxJO* zLNFvfz`AdTX9Yh9Lo3@F+KP}tV|K8?x}4pc0T`t{a(mk1+^6&_SEeuCIF~X@Gf#^=9KH)TzeGks*R|HGcm|tF z)vp#PQXyY?%2Qxp1x?W%-9+umguhyOd&52difL3BEh$6Vxgxng@wDbvh^NJRI_xt@ z3NA=W$3}CK7nq?0&*p)xl#QEGIU2Toc@@00ssZ4YCIp$g0P`9rZKg!A2Y$D?z6^MFv5`)arhEckgHb7tQ}}6{Jo>1W28SgLKU1!r%W>xg5*ylRU6hOW1|9*w zeDe`(Yu&9gf^y9@i4FvN)mO4d80A*92x_u@+J4>G3=>?D0?>K2x7r>5!)J@xXJBfk zdZ7BnMUVM6sN+bRsxJ4<^^01U-^>%I(1F~r5#6|3$nSPY*FEGu+{j>V`iW9c0ySJy907;c>cs02$j@)Sqc?>!4quIyb9 zbXJri1NVk*dM2JwOuU1OVfirK{J$?F;H7|-WfA~2xJ^amwOgv~X^4oXg+bb`RKg+I z?E~ZjdHG!dtj;e_X}0ie2W43iUhsSPsJh)iZIJSEh=Y`aBl;5qbVQ< z`RRpO&qFEU_**BcU>p`PjUZpAFUNY zs{1~aA5-!NJvXiIn?+| zeQud`gBT3-Bt4r01G$0q0It;xbv(TPsDm+YrGdFZexMF<1kp_g!es5K4=K_+`D_pz zO9}>F?`8J#4%UlgRrP}DzJ6d|Cnia}Izq+Ckxx*_$B?_sUz3yT9^H+W3K#)S!{iuF z0mh(8M|>EIb5$22t}I-vTkK6BkC8!N%E!NbEOa5^O$_%>j6p1{09tdu~2Ml)g|c^bqU{ugaEfzfd zJ4xg<+zh$3r~Ry|vOEf0R6s_Gwb0AkTqeCdnF6b`N8I*?TIU3PEVM!YQku+WYXMq$E5%cHgT@MPi z(14P#a)c>pFfIOLofx$dAtSu?&qDtl91LM6sDj1Nz5)gbn5ep^4fP*ATH92@Hx*KW zs45O*CuhX8GU92&4|o7cJ=adt9&F{;)K!!m%$JsI#ny4eaR!vg)M(aC;pMSRsMoY( z=`P#SAFg*_rXLJ)Q$)K>b*7O+wQP*ybuDx_BJlhqbV7%i$)!FXg*$}sJ9@q#1qtSC zFOLAQHZzO@ic${h4%l#gpdFt-GQnwDH|>{EgdgNX>C_PUFiYU@pIhjnsPZafAj0ffO`985rz8P=D-g$1*M+mG?6KTl_z4U@XPO zVW9u`>vog8nxAeQ-avGQ>jeV>)e+u(Sq zfV+dVC>oFwUcsX79=iR^>-0@q0T=Bu1-qQQgViz5P;oi#$)AT20rMNK;ZK5rmtdGK#FGC=|Vj=%f@*-*!4 z>@=S8G zOhKPiC+=$aFz8-CC<#XEcGRNQ>`N_$_*{?_Hee#;!4LEYTiL>4WjHB3KOAkNP?6SElVpen}vc3MVYwQJpmLBS{@{aKN z#eSLwL66Xv-sf_E8A5>LopYkW0q=;)0-7i@4ySFuZRt_hK@s0mT;NVmeO=`XBfxNA z0p-rSkw2Vhzf$;IUEhpgP617h)s#CwR&6`Efb%5`4A4kSJqP?Yccqedq8|v#y79}wO?cTNMh^@Bn9wk>k`R!zpjU?>cRR{klV+c zDGRo${jArqOt9-2%p+_{(&qY2qZ=6remVDT8F69NOjQexQ*n`6dbF2PDa$C(h`hsvsk{AMQ)$urET|nYFyQP zXAfCmntICJmi|zPeQTK-oRR0N?WrtSbvDkAi*xfV6#@4wW(5JD3`Y8)w_$uo?6ol2 zF!}Yamq66fEwDM#Wh-PSZR*19#zP1C1%;F^%FO@gXn;@!iXYwoA7x)15asd(Dl14S zASs}L2uh;@f|3S;)Y6@zbc%%Zs=$>NkWP`7?#3V#3F%JhmhLwnx(j%JzWd%k`tj}V z%sFRHpEEPs7fFBz7vhR@Su@z)VdtHb%Jr)nR~YQS#^@{3|P%I%Ve0lIj zbX!&;p|kP>*#NuC&06C(IwE@{e6G0gP#AiCzHsKUqAU<8+U*pT*Bcy+Sobge5)W|n z2vB~!Q(gDxYR|0%CF`h>SfwjIo*6E&i`FIRr$o23+DY5hx8Nl=Kr;LxI_Lg=0@iY}h8hcD zImcy-w1&7EIl=2SLru4YNA_xnMP!KVl%^G1RZHS~IuH6?4kUo)4v^tWtsGxJY1Ok2 z`r9|3l6b+D>9CwG`s1KAtYW{j_L5(DdvMd*Tl9hsdtRxxJ+F`EZ$NMW4qBF+m}6;obgEAoOs!k8L{{zG`ku{)pA%4NG4_Sib44z=Y1TIij#{L9LJ;Lz9q877I=qk$23{ zqo&P#w`TuZdBqxDWph?geU3Yv1HZ)9>o!*21EJ39u<7d|6s2B;U)Vminukn0Xb6hK9%!ha6jNTtl(0MhnCxpi>Ezo;L^7{%dwmiwo0LT{F*gIaCZEHc(gUS7Y!S-#Z zXthf@%rXm##%lr*PY6h1^H`Ji#p;QAO1z2|3v-F>UgE$qUZcg4TxQlLl;z z_*5D^Dc~}>v?2gsaIR{JnSjzv0^4$0fA@VQsa4^ji`^>nRyu0=a7UF-4cdU3m@&9y z44;C4AzJzoF;o;Uu^1Y1V07&xz`Cc6 zQ60rCZSwJ;K~b?^yLvpT{7%>sz|7pJnbHMhGap@c0B3mS=lPr~Vf`l9{| zg7>>j?g9N|+LBGl`;W+ViaLvfyRKCWe0zqCI7+U%z0yM;Xu;l2)~Lc8X^>9XwetM@ z!Y_v8m3iV!o!P^g%LBR7_{`_DSnc!(KMPga1PE%*NjD(&u3_B)cLzudqLDo6V<++I z$b3hPL7Bp^{#KUuEIN#ys5##z`%Z?|ldh6aPONClGbIk$W*2{W2&sUElz~{~v>+1p z03P^*+PT*Tw@-nAysHeR&EV;)cIt7R#A{)KP>zlkt=z0zDyirGV<=~H>MuE)gFSa+ z66vk@GQIaq-`yyAVsr~V`b}H8{7Wm-TLCgSh&(I<)(Z-rwz9df5$sl3zaCO-$HZk{ zY2N!YQ{1d;!+G~0kHvAwor#e8dPEm{k_&ZWfv5!uUGfXpIXa7!o(kYQr`$bp6XZGU zK(I`;&=vcQ3{As0u{Dm&;V0-GYlQIiSi%tRS|=?j+dE z_*shWmnngQw3QXaUQ)=H@>x$yS=j^_`W&>Z=Qr9g>?a`g;0rC2QbX4Jk;H!5gXJ&_ID<*I^KVldLVR8u; z{weV5d<`D!dV9|&TRc2WQgIccBGviw`I2(Cj?txDDve(!#{4!1Aa6kT^3Izyb7KV>+@_dIDqQYKX}isuwosSsSK8Q_g-iM6B<8pB zU$ik?XTvqy;E7>$BiV#!UYku{8(d`k0;+G9}; zshv|sR$UEAM))?wZc@MWAR4%aA$BCAkYWMxd?JBdbtVnwCf+hD_U9$9g8zPG1=@8Hb06I!sKObbZAJWH;_$sxTkOii+ zo7k!|zZ(UW&Gi6_xSl%{F$rxMIkc%;kK-W6myiH2D$j@0A?|xVEncM3BG0*Ad5_9j9=3 z?%r>b>(hhFx(XeG_sa>NzHH%%-ltIb&xuWklq}=!6PDQ6h-UP?cj!cJQnHpi3qegc z-VOs}bRN_cXte5r`>)O~g~}#&tV6x14sLP)~{uMUlpgaT{qb>23dZvI#O;ffr zNMzWZ4{EfzRPcP)kn=KS8oz8gI<&1kim;h$JHBVk=7q<@Z}B$IDZojOu(`59rD-EA z?XHF!xM>>6HLd4vO@x`i0dL}K7z&SkVru8w4(bWs+D~b-PvfWVJA>`ab$ZKBt$kKp z&cW0^wg*%D;KX>y_FRB5&F3WLnZ&P2-bz)IH5wF;N`JUysd_6SkcI#atwAZGaBZNb z*D|{!e7!8}BPeZMB}R;{W}-=jtyoUu9y@QC0lGvj;SxO!5NHIrYH~wYNOyf&A>pHE z4ZgHPx`43Cqk`HeJ@dnr3K1$h_)57Nlz`_!rFim|v#KeXO6VO^j(lDKrM<5O`d)go zBKkF9-C8xX0_@sbTXn!NQ@_;)qQ}&kzmuY>wL5s;s6I6+eeJP@?XxR(;%ob&)kn50~bM%9yQrw^w7B=(;8y*Vpk!)czsqn?oB`Oqta2ccLf=R4Ig5$6Z{9HR)$w7mf`&O$b)lE+#JC%#GdX&9fJ8mPx|9~PskT{X- z0NSOq#myVO2{Gxw#|Y_+;eXD%7E=}NtfRHnW8B(1i%q`AItmh-$>FXzSLc-)h0_`- z83yk&g=TXFo`gLh0E#H!We-=8G~x4JqT=9;+O(dTkKF{x(XJVvc;8ruO>2jyx4{GU`kGj(lr^O^80QD?4y16K+>RxNsh%v5iwzXVR&??I^0XmWM{Rf16?Py3 zqEIRv8;>FYQm0SFgxqe%Cogy!DMscdQV5Txf69AKxlJ<{W2AU~&um8HCWs;y?+eXL zmHo^s2=8~j``ymrpMjI;y=7Ek-<6ec6| zy^hvy%9-T!qyk>t(osBq9-DX{r?Wc7Q(UUzF}{yyo6nLRSPBt&AdMyM8-AJlYWYh& zPf2#aobjoQCd}-<)uRkwXtCeYSX8-tK<6{SXmOB0{+Wgc=7Z|JRc1zw4+>24s%b09 zpd;u3`qR1Yz~OZY3G;_X@nw`AiG$j|TpU zNY^At_(4j{0y?zA_O9ne=r3@Con_EeJM^Y+iWlu0uaCSo zZ57KnO}nJCX(RwPVieTB;m5A+XccZRn~U!Lnm3&pI1joR%2kY~ywabJfZ_mjFClO` zo-+D4Bw-d4e(Tfp?6uBqzK>5B@4#5|K9T6O2p>UsbU>nXnUSf}u_ z>1pQ5$~|2p4Rc4lc@ypIlVEdMUgBJe6`r|`=3Bu*t(l&SuZ%=)pg)JU-!f=E|GoVv z;38P{>v!FxA^NU@F1p4?vRSb`Fr(!(mF&?xBSrb>kKeUuV`Lm8=9X$=nq(aVKQ_mm zg*jt!dR{O+v)cfK(PV3T@>S7})GbpHH8l~;LXqMxmtTMPlD-U76(;>A?b7hsma?gg z{g{Mo-=1)zO!$Oc1Ow0aCxoYE&oMu@Pt6V%h!dp#u4?2q0&2Wa10sMa=+pCL3G7LG zKGhBf-aMBizVukyOaxxJ4&!amgIe=?mb<-163Tw=$IZ+(fupFv{b>dW*N%K_ikY5J z*2w4{#s%@kJk`P3Wn5 zv(NTxJk&F1nnD1(+bcG2MBxov$!6XsWT_D%;EJOq?B18Z+<*UN-}DQ`AQ+uwk$Tgv@y(Y^B2{ z?k&%2h?4zq%6wmeLzkaijx|pA+o2xwMmQyM#Ak3g#C(M0TvW)o_f<}LyxfeMp%ZWG zf!fvusM~t`9yDqJ0w_VDFjr*gLAePy)6?E8Tym-1Qa?H_%jCkqrBVG)~F#~(z4;HZx^rwgOW}oi1 zzm*aWA`x>nk@W6xBVpK!PFXbv(+d|I;~R(aS1@-6xqVjKI^@;FF*B2>&tymUm=!D- z^XK=Q-Go#*V4-F_yhx4Lb{ucRt#qd%Qz(E6P3U-^)z=}j9X3a5;6km?pc#l~_ChN{ z^|o7j3iuz86{1BfZ=P?Kz?uwK*HkoD$i0$o zcWXAk!(9HyYUC9;y9a)*8P^;HwYeR7+!>;M#jTyv7rqc^9^@7coN_5=Oz--qa)ZuY z1AYBoIk$=Lw`%o!qn6@B_uy#l^H^PYzYG?+G9KTCaLDyL9^v&@&M?v+|sZ)P_R9Xa>ZN#)vVbv_+_#M5gU zYWOcDZ$r(IHZaxEjMp{ox}Du|6$*Kc{m|vex2x!Za?`+LPW;XGrbLT&Nd~ zUys93XdHPBQ2T7)FVLZ%+`Bu}B>Pi})95Qvj_aK{ElnQK;Bg7&bZeT#HVKn~(9gkD z8WUWmy^?-asXc`+V^D#E?so+M8szX`&%+@h)AEncffMtRb_v`IbqESL&*OR1o@}lz zpQo5YPKb?w`n$G&&|e67p>wVWx1`_}+S!XGT?h5N@!NSCN@CWJnK&6;(!$6jf>ijkA--H2Ckt4eNq; zkDm4nAB&k*DB>-^Hquxp7?o|jliaxs18NL@@Pk_Pty`-d8Wr}u&|wr1LceEx3dyPO z_wUC#6`&8pq2e@o+(*GA(Xy*|V5pyf6#gvJDmDF__QnV})`WVvnJgGId_)lG1($Kr z26v_85k1xY;wr`4iVqAj*ko92KbVkf)_z@kw`BXKs zNCr4YZNfsC)D?2`&emi2k+$;HgNzn5-CRjLUDTi*wYJSA0{1sKAN?l!FXSlz@^IZqX(&uOTo54|aws!L%>=aX zrORNu>2K&neXG`FcT}FFM&L$^B}E<_VFk*)HfNqOGo5FE*OFYmdjb?>>CyV{{X}?o zH|SB}mu^nLseFcimDPTa69slQl@S@OK&ta7-iTqy&~JPO{~SYm1)x1+nliw_KgP;9 zAX@r7qq8LpT&#eHW#H_SYQ}5b%(ceWX=G~0X!Cj!v?&?UTfEP~PgS^G^rJk#LJptE zm;Ln-0u$0!;f}sMg`0xpzu%aHEhA78CGL;e-(g2Gt=e9m)BZfNozi9e1RQpyAc*jx z`!KpQttO~S41Rr$sJVd-{$OzKnlVePDe0<4IdnP35qkdVEg@%a6LAH?Jj+Ok?;Cp1 z>G6v#OkqrGITvmh%52{ekQ9@3bFS*mN9dDqfGyA6S1@u~51o@WZxa*W+b%2+mq!0tSwWfSd?R*2|3GJ&8Rj zp18Cx4dSjsj0m!z>G6Pu^D0H$>1Sz#Tu-3E(6I=Vvt461JX6#~fB*gK12sYd!i61m ztXH;DCI?>wuZ7Ma(C=1;OGV4XN7O&H^xOFo@41$kUX9(yDn9tgvfj4fXGx>pjcaJI zcJw{|Gfu^-auM6bR?ak&!NLx$q{v}TAV_erJTpVTx`EkIV9;0?qFr2SYm~X@2$%(aWe=aNn25^Yh(7 zHo)?qFTzJACZMsfJizkB@wrJvMOT)m=8J`S%bd3C?|dJ~YtO7&F&q)!W$WHFnt0?E zVpHLX0cNE4U!ZgjaQ9_yZxe2q4bl>*Ffi|-MJ^>x7A?S*YQOdSONr`T@}$tDp2qVCbpXKepZFT3aL1R`ZKv`jziIBg07=L^$h4)nq?AsP-B>PhOE=Mhp^IVt@Q~c&^>KW6k0Zo%q+~OGkR5<_ z$fG6-LehO`7m(2keq$)He5e5V3Se8UY;AMZ zX^@$Jyft{=Z&G_&7`~Z6ytv@S_HhdxD~Lc@>8%$lx8uWz$wcyKU+50y*x% z```nLS{$P&$V64c|M_SsOz+I)wCe;Wi3Q zK*oo#J0nr%$;U*Hpr}abxt3tv0=1m{V@9V7Az2#=UTaC|H?e*=_T6ahv&54-W9k_) zvAWwDN0LAVL%+|u=qbM+FJqhKxKtUl{$hp{sHQ`9EMTBv0Npl7jRQSY(W8Tv6C?nW z6RDx#(H4hc2kxou?N|$2XGB-{vH~s`za+iHm^0xqUcEeg4Hxhl8*$gpr^lHr)7f{o zZoNM)e!^hcFpn>r^F>JDb&d%<8myNf)IGkDA$7%-kACGML`Vv7DRxFw??RzFqr3eK zCDA0GX>xUn*9n9S?)kuJZr_p@V_q(H17tSGSyTN!V9mIrsGUH>U_-1)aEvuws|wHw znWEf3GYdXgcEfhjuu{XT7>KM|`L>{(HDit8&_aj?J4sNtj;jSHZ*_U~Zlv~2OWOw$ z%2pzjpdna+7Ud;7BrMW~a3|-?M%c^^iK2+r0bFH#K*Ss1jJFfX9?wL!M;e2km{~=qRz(B9YEMgQp)TzLR z#K59~y(l=&Frl}Ht`$#Ua`bV#t2y%u3z(gDOU))jOv&lm&OA501lf!p{=-x9Uy;mB5PB!)UM&0s>-N8Wg*OhvqC-FjWNmaXHXC3ovZ!TmP&w zf8h&%Q+?o_Ou+!vH{GwR#*?b4dcLoCMXvo|*((qx)i#9z4_sNPsPz)=dLq~1RF%+F z=Z;O43{6G($5b5*7V7hYIzCpXSJvPOxB2P~5@Qq#x~+xl;eS94@5z0LI?hNN7Q?5H(DgBg>EHa`uxiPyL> zr8+?|xmzfy;Udu76V&b}RGn14clBxJtVa_QI#wQ={{l&)*N1Gk3rrPo;sLfpPv#pf z4TFGJxWiHWu0Dj!EjL`W%<3U7N*ce3)1YN#KF&yLGOj>tT+ZWTUFkUaUi`rYToY$8 zlj?Gp$C`z$Sw;AOax3(UIV?|Yw7gc!8W+ZQM>-ueTb~@FV^2aryzr*?;o%$LP`!s} zx_4a6V^1RDIVNg(Lv69-=7VT`;aDb}7m4HDH8Fgrt@E8pdqW@lPU$nI0T_qwAa*Ri z+AbS9$@m%SjV~V8n@#BG)fdnY*AyL#lNOTp`?yD4wG#+7bixx6?r`cbOHHz6eCvFQjmUM>H|$+?sI zvO2SG{bR?B5{-jn`}>V%XQjEv`S8ohn{k?oIapwv0+y_dtKW2&_biK#8unNngs0$x z!9^RRD3lHW4mCvoo&yH|luc!!_en4eZxSIc;Lm63Cn@0#x3L363Av~X>aX(;dMmd5j8-=Pzs*tLHP@Sni&wsI4~c_af?(`d51{F zN@H+lt!Uux^5^F5wW8<*dteJVyK2{p9%+e31-lJ&bw2C>ZbDTs`cM9kbS|2km>Is` zw(A1=4=0e86p}9M@tyWKisM-b#dCk4h=Id*$=JmjF=Jc4B2I)}s@mxl>T(f=|0R`V zeXSbXAi9!3#)}Z}XaHw4ZMW0nXhXJ`dgROfTbG*|cGBG5#dH_leJfbJ=vVW*V*TSh32-j>u^z!B++C<5Echg{l(Je|};f z{6bwG$M8*d>(<+&w%tG;EZcXV&KpNC%ZGf3K{yb;SF9)kgZ;JWADsQe@yJy*tlm$E zh0Q2ndu=Z~wz{YBS-$&I#N+;xq}evL#udz&pV+O-+^4V!Xs&uy;p!C|(yjB+Eifxl zK1c_LEWqJHxtw0%uX5Ivl*ZXGj5>Jv3X&&pU!r|wZJh&TlaMc-3aqC>AjS_BgO za^mB5;t~N53s>~D=eP7(mu^0LGU=mV-tFW?Qa7M+iG~Suzm*VAI7d{gACIHuv>4Uly5^D&@I4lT9qCapv4SQxZX)8A|S}Q%{ z%{3pe?F*+dnIK=V?R|)W*YJ~M-AqH%A+Wkd6;dKla z1;ah`hh&X)KadQDSi)_sj> z$me@OQn!98LbmVP3s#Fcw+pY=!xHl0x~C4A(+e0Vt@oAfV8v0hW*GtHtq_>LcIx6% zyJvAUN|HzrweP;?;&5-jL;twO+^1FDK^2^sA)A#Qa@qBn)PeIP#E(!)NT6;rp3%S1 zPcuq&)Y=Q(1wqmo(*ngKlbx$PP;GxA$mBmXZJ4tAN;lmF!+Igq*cYz6gTq|ZL61vX zCotzvh^J*iuEFABcA3Ax>s7-8urkkZe>l9Le%$C1x?XlNe4!4l{Vnk!njC<%O`pdv zAHD&m`yk3M2AbzEpFqGsbM?%N@=PJ>`h1IW$KeH>b)}SihomsAw6^APP0g60u_=sg z$S<|V&7 z$*Ii1ytN?X_0q`4U7zy&OSJ=4*H<%{dqO}Y%fRMO{bVyUZ(~rnx9_IY1=V9TqyaRn zx@`RVC=GGw=qi+$T+C~HYPvrRrI8X+?*Jo#{DwAAH7`viO#SM3Y?HYf<1#SEu;1c` zf8t9FjTc(`Q4%&^VHNp=K`tuj&VWGB!!{7}A;^KN|A}9>cueNme({L1r@|GKE=70& zOnLDS)i@Txx!9_4|BY?gaDO}Q?Mrk6K4~f_R;C~X*7$SYc4xJkv*mN0)+Dd(bG}tN zjE_BrgE$Jse7_hQwf5<8oCV`=k=cm{?x92ssAM159vowQ zyG^`c**sV6l0~uGxnQA+D?A(|<$B5r4f)}hCT>N1IYes( zVy%-o^oMT3`vqsdq6~*}3b0*b3F;z8Z5JNo8VHsD2s%EQYn2-e+ueyKMpLnGH0)WxWP{f_ zPtn>#=MZ9RuMZ>zYtIFOI~!^{1boP$_2fZVzeqE1#^~o>5bq2de5!XC2{)$$1Pd$l z4p9c2n*$|NsPT?FBtk>iVXarpT+*`Y8>dX2U=K@yCi*A+0@vraO&^tw3K%L5L39;ZNIwhROz*m%2 z{A8-WWcUiBbqSyXUxW0xPNU=(c5;>OG{*JjGWc9kIHkHcigmQlg2}z43ooOD0uA;*9cb68 zujqt$+h!S#J6|)X<`g<5$Jo!Au8uP7Tw}EAvw)nITJa`~J{kGp*6idXz;W+|W4R%Xm+eAn3h zhfi{4{l3yF7LM->C(T~h*`SghLhdnlT}6k$88F(_`v^z&tGk`Tn!GhVO3KTxnadg< zA1){9Wg3QxtM<4@T+L=rXnlEEk9%QQ-&hutikIJqEl}0^)Y7+=P4I|aP1F(y!53wWZWHeS*`Fk}x{*jt5W5Z94y zwbJ|>ze&6eoP@os7?t*v2YH^%EHC>*Y71%mA0$1!jw@Z7t7HhJN5Sjg9w!bTm&gc+ zsW(qC9pPVu&Or>u{wI!(=!D);j}k@ncOD6F;MoN05)zkoXs1ys=_tfBmYAvc#;OAJ zzDW)Za<#l-6AyDG6}HT zGhs`H%Td-Axf(Y~%CGAzC45l0@%H(Mtyj>t+9jrE2?^2BxLTDhUK&85BE(>u0@`w0 zHxnUGYE0+ZeowrG3yQV`=b(DoKiu#s6!wrNgBX|p>>rGKi^xt#Z_hwK7hiYeq-@Qo zhs;-lwB#+$C0(K*KHI#~w-x}^_ZmZ7VI$x2;T(138UCq>`}Uqt;_0`){sO2Y>b_ux zBNZJ)_5o$B{-CUoEe!jM*|%9teR=$P0d6=+`EA<(Ykl9|fvI@}1~1;bEN5oLr>#RK7K$ZZrO94L?qAe+mpeIHH! zSkkJ8=!B8Gj}-T|tKSdwrt<0~j1rye%`YSoDUtTJ5uNhz$=fk~v-{jsV`v6UVDNxu z@Z^r|)~@JqG`~XG;@EcCyY|hdBr|fxWM!a$t*>LwQ-bo)%w1a^9iFtw9X_>swhJW{40nICS^6KM?Qb$dFe4e4B*GR-$LA*#2s6IPtZpPoRGwL%ViR99~N{ zD=rS^Rn8R}*yq*@Aj^36c35iBzpQfZX5UYFRPC%e0Qf25rRhpY`~i=3(9zIUT&{T2 z$3j;@NZ)zL=*d?Wf8?#C+nZ{xy{n}r znX~F#Sq(j!0M>p4xb{--)v=?LJqO8V|DUp{gF`2={YidpQb&=yGZpV^tMCGMC%K4# z3ZFA8YoGoPQ@UPKXY$rfR*G zyGLjKOD4fp=o+XJ?(wsxvv+g^_dGMaqg&Z06vJ$~l%-zuNsOK9yS4eZFagtht4wcz zX7C~m-7}`7B+Iv}otb~LN!_wUk$C82f^5{E1!hDrPz}6;PJeO|CrspB%bEBTn}R!; z;k}vyz3KV$V%mF5Qs2qa4w5->s7M!@CghwvJ`=V$IDTs?>oPCxg01)`rUzW$ zqhAc0j?E%@Ik3aNgB__ZKv4{bF00HKCq}^S(H6C=ZBx;c8j|a5G1qo`q{v&TPwj5v zztr^C+INf$)Y{*8I4ZKax_-Se^n6}tcRc!3oB@FK^^qk~E){xyhPr`eJKZVq_P0e3 z?D#S>f;SA$Pas%vek?=^){zY~pJ@89xjjiuF=^FJn%fC)B#TjN9=hAz%S-Z$FDLKO zEMT@)atJnm`SB@qg(_lyEJ#N_-ZK&X=a&6b3TMODi7q!Y6ZV-~ylxy|CX@-dr~HW; zDtQXh#u$X6(e9<|ja~Z7MQh`OTFfyn^rJ@PrPe>zKYinMde}Qfo@7EbST`}n!#Em& z9cVsnF!wFTGy|ow z`1kJ7D69Py#+=surud%P-JHyQ^>oHvK=fE~L>ar|b#SI zF*x9ueqfsrUO;*B;-OXv-JVqR#50Pm=)IVM!n&g2am!uDXaN$(f2ai|WwgF*JgFhO zF2HpG+YkO?j9w7p7{wzTGkyisiv-0v&eCLEN9>ea`KUj8SToODs+*_e#PDX2emL7F zeonjQV{WZWq?#7T=-l0kxgW%~<)rIR>muIXfT6py`>(sbkR4n4l=(Cf=SuMm=`-_< zSqad3TV?P)M5xhJAh^z_Qj`u-;CtCMCs;BFkB% z#_Ias)zNYCyJud(ud!Da*I=W=)MMXg*19n*GCYRvW2b$EW}D$&vjl=kB+5EE)$ z41&V&-&!w^$)BdZuytGJ%!*6x3OHlN$tiQV1Hp%wc|JIZ2m zBJw5@a1=2|fzwD`E07WZ1oGgeHXRI`e4*?n=We9qp3v7l=Ne3oJ+{$3@;7@A#@kfNCSKfpZPLsB6=ZB3YXPwjdlY~sz7ES2yRyL8+VIndj~8xV z_TZ$`-rMRG2O(#x+~@rJSNY3xe%WlH3E1Yvh~=ER>?roUA`GgsO)Uu(bqvMJNX2fT z2(sec>7C%@m0z$|m7x=|RVdczULT0Q+4L>cJuxQSGRgb9TSDhhj@*2?X-$lNnXS>; zK-v|mO#WAwRym^IEWPPl?)KK#lix`SP!h}OnMvHgw(*60 zpM&e0olT9867QF?Oftfl!pX6G_O3PAX_`HNT#!WRhLd_2XP^_Vk&0g<`cXK(bE*)a|IOBa1T4#S#ANR3S@W zf_^4=>pxOA|L|Br+1Y3#&$|iiJvlGPr`uFP2_R8d|C3m({)=;oPdEPYwlcOY>BpN| z;Cd6o89tB^OtqgpDNqq>par!|0bTfPaDR|xgSwopXW^m9k6mf0BrU5nBnaVTi)Rd$ zMNS}Lgfb6=RGMEm8oWEljguZr#Aw<|1BW4quPWZNC8rtd+lL z^M^MlL^Ikc2Q>f8f`|-Z!D!(#wXiz5S?h(gXL``mu;<_rE2b*Xf^f*j-r7 z@;#Ep+PMQfWF4fagPfKM&4*!TJ-!>2yN)&3%d~2HYNJ|LW!u>jRB~7)Rf`e8HsvkwG1G#fM5cy=f)S*gpJJHM8^kw636Nz#>^f z2wVL*BC;aqPI2n)RoM8URkpB1Bz$>v=Ov0fLai|DGg`bS{Oau<)yzxkp3dBidG}7~ z(Ru!x2AP>n@q8DTo-vqNAI%hPk%$(VQ#9_Zf^w}%er1!KmS9`=?B&Zsv2utVC(d26 zl<0ve+E1Z7y^N#HJ5KqA=f>Wbd(QUng^e|J2(%9A@mwL+ zNU=w%BNRb?Km03eb}m-vYiHgd8rULkmex_HkJ+uP-d z+B)_tb8&+PEiytajyPDBxFQC?bB^Ius zz4da5`?18y(JQ8g6c?AeEtTw&{5Npo)KUI1Qgt8l?yomn#30uu*Q<&`tsIArrc8v? z(HfX=RIy-x3q`|Db=1A=ICm|ppwON1;)a_=p zM)}t!3N&a-Y2(i;>+K9`2-#rhx1fxmcgJTGz5eo`?F{Q}GG_vw4GLgyp*th378_(7 zRKXYNDnd=Oj3`0SX*!$B_8(xyf+HtA$-3^bBApCX5>cU@@TuJ~a?ApwSaTRZ$VROH zV|uK?7WrKi@xeouA{v}nD=l^#H6bajzyHRkoW%)_m!HXUPLi5JtsdEDI@$h%{N7qs zMAfyG9=DL_ank#{ZYZ=a)?c6Rl^+eK&*A|AQFXUk*k5=91>QFnz$6?|lqiru&xPnt zVea@6xzd7z>C9PBf*1X>1pmK??yUuiIy(-@Cw~iy&@%-3XIqItm$0RG-gEWb?{C9C z5fBhk*9d?F#ebmpjSc`s6cK?U-WH*#sN>ekVyAvzN+wtOUDqmLTJR2Q7=Ib>T(N7? zHDAY*Z{WYVG?@j^oAdmhAo-nxSnx@ZPBb!JQ-15B*!DU?3Ur_fp*=Qx2E79L4_d2+YiYp&TdFw*$OAGFPZN z&w`{R7_Inf_@Qn23&~F31ynz`4es;#4K)Ibj&4tUb0yfD_vA&PSS2uyW-;r3>Y);l z?>gR1NB}4vqrTSYJ(E84Bm^XM1i?_?YDbjE@)!KDPT$ww{viLB00ogA$dC=PH|#69 z``hIv360`{#(FJ+vd1WvLlO1(!@o)b+G#NCSA!g6W!$=OdEn?Vfy@naVfFD}{hLri zl|ZfR+N&W`K|(_h!8rRm&f6W{^)LQEh_nR}X|NCJNut5d0v_;pRE^0d0@#8^IKn?0 z;gI5q5;yRmeGP!)Wm(Adn*m}Cf#Sr+t;!~^pf=s2GjK)&Y6(}UknRYB{4bUyP=4vE z-ALP(4tfI)zG?s*#-;d{y%O&PcXDv5^;s~^A4QG-J&r$6<%yii2kk(P}ANo|@eEa0vPC#xZQCz@S>Vd9pEpl&P@J2_|cy2>9^F+K7qjSjQ@ ze*q2%67f+%dIJQl(tc}Oxh9`!J%t+vZVw@(9!g~RPmyK0qrEN6VuU2p2}B8?(n9$B z0I-yrwh`7hBmlW}=Dkw?O_t+ts_sDK{%9tHB)25s?U0u4G!8Sl+`AfQ&w<9!4ei4F zI{%wAhj{)=NK&g}lClMpdm<~aP6LbN7)u$?@*hk@lO+l{=s9((=_oHn`W_9Os8U~b zv-7l|xAG%+KDa#e;*yL>Anku@qkTw=M5rO#HKe5gOKM5V?FVK%6pwS>(=GzM{i&bw z&7sTjzoZ`o9d5XGJ)<8B!Ci>IhzB}nK-Y)(EszA6ADzFqihiG-hE5=LPL|~_(t#Rx z64=}_7b|8YPo`p@k8(Ftf$~Y`KnL}PE4PT@e^QzOCLZ|Z5kr15su&A0tk1$XPRjnI z6j3aAg6*vW=>4rU$>zW4-5d{+zcTW4V#DH7z3&8>=My9s#pJ)JwglsesqX0Aox8~ zY*1B9LIiKnv?vF9hzCkpy`w6L8cn)P^=x(JLico(K$cB*LG`S34i?N zUodGe9`N;|FcWeZA#F>Dugz3;#mSYl)JYU8f~r;`HWnT^UK;iGUeI$C9G&%E=4h<% z3bicuj=FBsu#w=MW=7Tdb;9`3#)t32=hLSuG>o4#YbN0*(H$w=q2%$82VGK-Ph6f) zK{i%5zYuyp($sWaWHl>@zJSHz$=(JL`4v6Ki=$rTNp#YDXWW{0UOV;Q)^KQqjSBJ5~->E1E>g^LKanM#lynN-tuNCPYm;_H)`|el8 zs4}iWe&vxP-G|}UEtR#83ON%>^+~%N&K#q**Vm@3c56Dr1u*s$yO*18%!9qq|4C{* zuwGU|oEX2EEH+i>&Zqr=fq9MmO2i7+gA1)!+(9HbhmGxGNMpVG-a8UuD7`~Pj$apg zsxHJ`Uh9MEKn#aP$a5#d|1?zulc0I5Oy8m&sb*_v{aqKMgX`}wr||{>y8bFn?&Z+hI3z0nbbcP>=4mY&`t_}AL&y_=@Nxu zqr+n;cE5WMo|DzUFzODa8VFws@FpTW6%G1)pHfFO)n1U>hzFrH#h~F+$r$Q1PBJdaP_|rgFSzr+E^b52*v5nsUl>UVCO;n(mR`gQe zOJ_K>WIG3OxK*x*886c)hz28da0=CBhk|z2wr+AyAdm{|ubK|DFwkB-gSL4{V~{Yz z{J{;{&v=Of`vjaQR#pjD_}|-$I*PraYWikpov*FR?yO zHGt*STo9c%Nle^Fx*YsyaH5#{XlKq`iBH2Zq4g%0qUiLe5*c?f$Sb*8NZqB^zA!%A zl3!Z(koKnd3z-h5MOb3nz1=8U6VLxmTSNxNd;|ucq~nb->hXeFy}r0r|BWE_Q!({C zG?kG8B-;+TuWz_?T3=f9A168Sf1%Frk2s+5B}~kamZOchPA)bH{-MIUVgAlC@554Oo{LJG-3^oUO_-((ZqZ%sem`mw05*T;^S zpti}oziFtu-s=6LWfIiszjE`%oWhT3;?(lDs$PuV`hT7%~Ckqc9M5}=Frbz_ONJhm*h^=!1d|BVjY_1ev!{4YOG>=XVoRr#pnT_fmX0@BT}& z{#@#(OrWcjG4l%&hmQ>1cUJw0^;LCvzt+^GcdlGc-f7a1G#hw>{Rn>`ikUh$CoQ7%t8rm5Yvx{!Y#KNOlzUlkpT03b((tClMaN`;kj3r#cj?m z6-ej)oc*#nHH&{;Dzkg|n}(#MBa-F*0xdLcFbR|dv_pM^f>2qtQl~Jl^y27~Gx_D| ziqt_uKPd~WUuUvQ(M+sWz29r}PN-J6mi&f0A>8;!-`-$ss_g0iqCd^mb0FbN{Y{Yp znMZ7nsOrxN@ZS*btHph_ZnLJR$>6Y((YNz4>`Joj=!#2yf4rvJEu<;>J0j!sz&&Dr zu#Egh&85ZCQO5jbF?cA{YFCW#;&L(BO$0tMV~K8O`Fl&X$2cz`q#eNu1!oWzW=F_ z@&3c0YWL%e|E9Dm&|BuFwX46Ab9rnu49>&Y8>M_R6L(i|Ih3;7AGh=4RpC1hy0Q*1J$7$Q_h`<7;Gkc#m{J0;Wxe2XdLXE zZ*A0D+FlEpwyk5a4L#QJ{x=0E26+KJQVmF?3YaXe2=qE}oT{}1r;wC9{aJmz|J%Sn zE{xhXytYUs`8Qf1pu;R47b@x^Nl=f46=+u2KltN8+r%wOF7%ub%GNJdMo)7S*~}^v zOZ!0^IdW0|@(=_ndF;jM;m9R5k$>az zgDtNIM||*-AVEBZPDs1ko>#1YGTkh_I^+j-o1kf5ton9$>vC(`wJ{1cjk`*9W8513 ztQr3u-=qSMM`=)J7wJ>65f>?p5|F*y58*VEF5Yz!wMyS7sA6R@&l>m}4N>$!w1dGi z2fkfELV67+kBZhwB${t`?wVO-cpFx|hXh z9_d_p!(&$orYuK$uajiT#;)>LY%-M}+qj4V=hy==NZGSjeCR<2U?!wR4?Yo6wrrPD z%vZkZSeE}k?R|M%jO+WqGbUQd5^c5|5!z^1s&VY?L}(|K)+z1l7>At0M6_?pp_0@T zO@)>rg;umTtx~B>t7@8Q`QFp%nWo6&e15Oj@2~OK^YTpha^2VZzOU@|}YESTnlKUGuR-kROrg#l7r*3GXd+T^+bJcz_B32!+j1m;CVyOu6 zr3D7u%WG8Y40==Bm8YuoZ6@s z5EzK^Li#ELXzJc8&qHkp~WZpnUE);0EuXEcIgO8@s^5@hN#GbC4E7+z-h z?td9Yw3@ypImRgL*S*&MWG^$XRDmPmwi4Pafz36!vFS;pPv(bQg3oPKk|j8XBv>Q2 zk`fN&xcy5|yCM|NnApUCeVgi^<#E zpI5@~2pJEwF!;9fzic0yLFAquWoULS&i}(&Vyj{)&j^_(-9%S{69R^<#UTsQq|m;< zD zm>9ggy7LJl`7Tz^WXPEFCsh6S6=jSyz9O>Wnl&LJNo@bP4l=n}j^ucWCqfs@KJ@9YX8V zO~hJk$Y^eRO*S8N`>?PQbJ$HUM8wYUCE&zw^<+E3i%9tfub$-0>{0Q&4g=LuDlC{Z zw_g>3P3}RClTBupty`uIIDK@c5h!^QU%K!*7;0p0Ir>#L;B63gkfZgo3X%yK&A=U1 zlBxIVvFmHzP7L=boq7O{0oLzeH&g?v{5GVbD)}fD6TF$Vt-(VH@>c1i7NoKjOASRz z`x>{SlJLWaG7U&@oMGAn7(i@l!uZ7+J~~Py zDAbtg9$8FqGtPAwrh7_t7p64Rqos3g1eKM|GPeol+6Fei6UaMsKL=h;SJr|r#a~zA zotYIYhg2aQiZu9ov2(m@lWYSqI;H*3ED?K8-IzHkxT*ic2|?JSo|gvn#eg0JYKB&$Yw>93 zp5ZM4IgRq7jcGAeOJ1mf!1AR%?$rMMR;@%h93Voyi`vwrrj{0 zAVnTzf$LHYEA<;WKO6Riug$%kW|Jwp+;gPDDFv_cg>x$kx+&hj}kI6q7`Gkquk#c9q`(=HC)M7YZ6Vf>f(icR@@y5_JBwmHH zq|l(ebDhqV@j;_%bpJD}G;rDH9I7fTY&@0P!$>SxNY&5p$Je%&Qa%u(Qd86yEpmKn z#L!aa3>s1Qf@G2Qr>B8KEtiMp3NOqFST>knzt8)-&U4xn#?!9F#QnD_QX=D zql3>qq0S`bEicuX-$l6NW(> zmyR6{?%Q4r?-f};VA!li7P*<_8}|zm0-!9nzih0b!BK2YD(|239;@aVy3!pfdG4%D zim{LvLA%$py1P%z>y6T2{-(eujaIW|dJsFzr}gPT02D<~8X$y@pWf8!ASpxFckC92 zP@iH^wC%$$Uq9Te$SKP5-fxFaI*D($;~0a*J1(3Cc?PYe5HRxoG4ajEu_;;7g!#I> zW0POd9?Ohd!V~-DmzbDi*e18D=Va!!KRz&jb>FE-9B4%6ZL1V-3{gUG+UoDt9WNG4 zNOYx{4ZZHBov?YNrdk)-tR|_Q_^~{S>TYRV+*x8JQ2`^&fJST}a^dGM2S3hyr7Zh2(uxI#g;(-F4Bs|y_;gFyUu!MXL195@J> zqOxiJ+sEr+*2(3M#;%M1EWb52yMm&IKj}DPO3gcYvph>Pxvb-3DQQq}@|~fD6uoU7 zwC^83=+GMqWyFAR8 zAf+*uIi(R>#-k;Zu2@?QSw|?YgT$HJFo;WL9gKs@&G`(#ysCgM&Ro^*N?* z*&Z>nofl+Dem}7CT_gJ#2=hUX^)XlFC9DTlJ*Gt=Lt|tvE2;E;QOkJF&iOcEd81+9&L(%$YZ{J@83<>FgzQoxjde#tNk-80g>T~ZV zyp^A;eXfiC#QDo*ZuSZ7W3GZPNLtIKdu5u#i_2V%6_qk<-2@?weBq5Kf?igASE}fJ z1?h4Na$`UigJs;OQ~pWaSM`*91V_A9yZavZ>q|V9wKh$^$lXw4%G5963I4r!XwGK* zhrY+-7f->nJu-u&^{`+ELp4yI5t}>t_*zb<5pO;l^k}$Vrm|?!@v@@__MH&uQ>TpS zi{>VM;4=GGY&Ag^kSxhEyua*ey7D_sj8d^b;{svfUDbm$J+#-93sfXRxr(}b+FQ+c z$Wu6pH_NG)Vte`htq7IfUR7@L4*Lx!ciH($v~66TuGGide7mNDuR}R;4F2?~Acz_s zJ$((LT^0MdrbJUZ=tD%4#x_bDhdBQ6$A=ori}z4sDS-oJ z^IsJ|>G2v-y)OFoKL7IjV~7Y0;(zym(*+hRiWU#qa+D>4W2{Lm5Tj_8`1D|XnEU>E zZ`VRc4%%JPAH{o=l;X=(%P7S-6RUfjH@&iT%gU}P8s~qTA;ZGj6?F_arvI!<5{C?> zK5{k1$E+7a)bDWUlSJl*n<$I4$#P6+#mE`=nmM7yUxGG~0<<6-huzC3EX#@;d7q}A zgvZ=C!MUt;JqC5b18sDKDP9vAV^?uqJpcHV9BxV1aMCC&=2@g-qoOO}8WI&K8h82d zc~D3F{6=zoWp|*n6DcZCJEU7p8h%gnnh)&Sh`n;@mOA0qdhLSyiQnENej^CH@t0+f zdg@qFiE$|0Cva+`|F}D+k5ieS2=$g_-U-+{=p{SA&HP)tV{2w83tEeCEH>G4(tB$g z=M8+kEnAG1WyO0+885_DLCz|+q121=a~0}KA9|);x3^y;z;JUd2b^G z&(OYnooMy{_J~n?upmdo?gtFP?WhUi4E!%Z5$P$76(&WX(Q5r+3}W4r%B9jO5d*Eb z@@;#)?L&(#@or_~^6zR@%>+5~bPK;F4ysUj6r|NBJV&Tr6XI6%YVn2D0{zwe&bp>r z2#%Gi3c5M2S{30Ia#+xjPbPKh8BWq7>WPQ9XiWK$R7!$qxSW^An4w!0&c@tOq^({+ z{%uxQM>%MxeXD?D?>4WoI|brsbNIRGEmh(gE9u?ur`NI=d*w!`6qyE6Ov6TYYk(YD zZ>mgpSLcvaS)E~r#Ms{Dn7)Vsb%Bc5tXpR}y%o(Hx9uD7&ll4^3AfEl#9Cmp-(;9| zL)KReC3JRRoq3&AfJ*|K82TupicK!=HB+u$i6z(0j;A|QWG)3~h5*dFUQUjNfccWeMGGYnh>L4<7`&g^{ zNG?GVmYN{<8&q9A&6I-V;snCu(?H!0$}xe!V{21RBl3COZH*07`Cb41g>2nI;DV?W z8tF$xuFv7-=wCS43*f65-#9k?Uob>hKxDD_G zLji&gP!8PFSNiySJJ#?A>ZF~Q> z$HVV7);wK<Z~)T43o3w1#8ftrYS)$L*$bg-0B-7L3juJDh?~#OHnpTY z(gtJ%kAldCzR-lfF&WRwWTCL|J2M7t>pM(E=dcjA1|J`JxTfof2vTvag4&U6U6Vk_ z+gw;acH5HQ zSAZ6Vf7T1a4;M$jYK3mA7Ga-p2(P5w)^L5>uuWzasEpdX8UObbzZ=&hSX3Ta2rfZX zs-txw1lcqe!BLpIGj4*6D%R!2<|2*n*1Y1P>X#_Ot_obmF()YehA?8^yEQO3j^2sc z6p855^1roiU$h#0+9tU72VIn2k!L@oV|t99)b{lbVM@A<`^KY}&yl=B{aueeIzh_l zVXo7pJHN2Fd7qSs<^Vl$4{w`tD9nu(KX(q1ze-_G+}D@@ikqMA9W`ct_#tG6)nm&* z6C8aQ9PTU`K>Qkhrrlm3!8tEtExXu>mTz5WK)-}98-F2dGAj&R5w|gBdY_28{4~J= z`@%xw&vX^`6Kg2G5zoEKsHa5b5=7v&P!!NYZ4L4#p~W5Anz{!?BkiE#rnNSSm$FYH z-0!t&CRyDGO5cFQh9Tm!4_~xFfTU_83gtaT@E!>mdA+DXysadDvy92nmN!-%8~2c6 z9##f1aKHr*1S<#@Cp&Ts`Zf#p7`w0zEqsll&l8H<;bW=4bBIay=X-Y%qq?Rp!$U(W z`Tqd{`AWBgkh_n4h2W_6RU~f+NW13{sh=e7pjZFGt8DPFb!MUxJn2fn>dcyy2@KNh z47fz@JWfShL@9pc4Qfmw3Rq2CifX+@P>*$jA8r|axZ16LyNr-!W`qr*`5 ze*TB3SD%)~r*{QL;icZM4nAoB+LF+&Z55jAsJH9B$uI#6Jv^ju+QV&_vFS80UXA(~ zy13TuwcU0fZ-hv~XBy}(@N6JB1-ji?XfUv9;z0x{e|Gh|4q%Zk+G<8L^K8ezh7%JX z5S2bwC0~85-kTf`r=n0!pn9dLA;bnUdAn4wdyU2i5~+Pst{Y8;cW%TI3o9u&TqkH8 zV}%^v2^Hzi=ek)%AoN{NJ^IHnZZs>pWJ{REY_7cTCxvKM6Tt*sqFgI-ey|EwJ}#k< z840uVnJzD)Ea*chOIRQZ5ug383*&H8B*QpJH{q{ zs>Lbun)q`b+TMS^(%T*J3BUdPr3yaK2s%@tC(+wD;D! z<*{rkVci?=22g627{z^pI}&j4HX!5GWJ@fG5o7a;1`sLv$6q#E8k(mCpY~}%sA~CW zzf+LgJfOK4f6`+?Mapo6u(H7H;=|t~89%*2{{&2TVrv+VCi$GVXT|Xra$>h5$6pw; zfT|tWf5LtMOkI%cBXa{ky`4?*F=DnV5{QR|H3hM zB1mqzm)T|s1imMP>Q{PFxg4|*b9B`%*0*p+bjCgCG;aXq&hC--p&ONj7Ymw)Xs_ve zQ@4|`ZDp>{U7u>o?o_1qFP0ARG;X?hgNG6-Ht`zyT3g?cW40IEtbon(-oLo17(E$% zz$+F3?Fq}xwwk|LvSKO^M{>);CGtu3Ck8p3Kz!;m@!qj>;ei-JBW7$$D+rw(@Ha#w zhp@tUT8Zy9TywcX+Kb?lY$0q{VE(2kN1M$9Y~)L@Jw6s;_t80BDD&Vk#@>zC!QPe5 zH(5QTBnrrXIk((tWShf9J!%iGMp($#7oDUJTh|QA0#^gH;I?g$6oWfEL{j(cnk1G` zf^h4~(H7FfLjA#y2jm5M*j{_R&w!G77r>msY46epBYv=te#ICO^wc6!j(22P76wX> znDlt}_MK?KNYRF*Cina{pc$QV{vt$r27`b1fcg!Liimv32K^LE;1ZU}HL=(u*5)gq zlf%JP?9>U2as6BLQIw&^SMuA}a_ujaz0NI8%D3IPDm*$pD;cN!Fg! zXPi-Cb;gkcw>Yk0oKmDCp$nY(OwTk2_=a4pdw%3>IycBu7nbqOZy1in(Ja5dpb9js z!s5-d_p}XzLH(fomW3Grn6z;ataPHSq0na$_7GC;T47(LD@L-dT7V?~k!nqf`s}A| znXTNG{Hisn-AVbo6aP*Pm<|KaDoNgH_S>4+Vs@}n!gy=n`3BjxOSrB+gGq{g1jeKg zV=?aasx^UtyXOWm%)%3(n!&(6Hw+jcOKdjkk2ZCy!9oGIroFvk$1s||jaGATp*(`|R<>{ykwGQ+=xIZD~yQM8eK)$@z?t#(db{C9X zo3y~Da?0t#n$^oMUQ(p2he^mF) z(a1NUgJT%!+<<|T+FYC7a_^Xh2EHRSbhN1;tt9I{6SJ8jIBskI`Kjhztx}u5z^U?D zo1t3i82p_yufkwy*LSD&&vI3W_{Ibd{E2%ihf&!QBM;iazU$PL&JB0G_^PGZpfy_p z{`LfX4*-Nl6r$XLJ+rjc+Y+EN#{JBGpLd_5C3NI-m+_N{3jp+T!|tK$uaR$-pz}Wu_$w2~0(NjuTVBPz zun7N{t^hIluYbZmAA_4ms|B?Q5EbcPrrr!;vFCu|DUds%^bu3OG zto5^o59?-6YkETzCL9EA4~$n0zR1H zxM`=11$;0G3=8;Rj^kLs2b14HS0XIngE@|40Uu0$$Fve*0Ut>A@hMkwEc>EGe*3~O zHI$a$&z^~yL#9hQ4gWcfcNfpf>*X7}f1#FmuKHpxu$yo0n0$dBzmop#9_8!3GN_Bf zoVCv#eaW6)`65iOfUZ*+DgUg literal 0 HcmV?d00001 From 0603e0b73a9517d71e11f1ebdbb8e17a048a5c3f Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Sat, 1 Dec 2018 11:39:36 -0800 Subject: [PATCH 29/40] Bump version from 0.5.3 to 0.6.0. (#3420) --- python/ray/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/__init__.py b/python/ray/__init__.py index 6d4e0ba9a6b9..ed024a107aa5 100644 --- a/python/ray/__init__.py +++ b/python/ray/__init__.py @@ -65,7 +65,7 @@ from ray.actor import method # noqa: E402 # Ray version string. -__version__ = "0.5.3" +__version__ = "0.6.0" __all__ = [ "error_info", "init", "connect", "disconnect", "get", "put", "wait", From abd37df41e15d08797b348967d155bde324a53a3 Mon Sep 17 00:00:00 2001 From: Hao Chen Date: Sun, 2 Dec 2018 08:11:09 +0800 Subject: [PATCH 30/40] Add stress test for Java worker (#3424) --- .../org/ray/runtime/AbstractRayRuntime.java | 2 +- .../ray/runtime/raylet/RayletClientImpl.java | 4 +- .../java/org/ray/api/test/StressTest.java | 98 +++++++++++++++++++ 3 files changed, 101 insertions(+), 3 deletions(-) create mode 100644 java/test/src/main/java/org/ray/api/test/StressTest.java diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index d8de9a086407..10dc172fd4d9 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -75,7 +75,7 @@ public RayObject put(T obj) { public void put(UniqueId objectId, T obj) { UniqueId taskId = workerContext.getCurrentTask().taskId; - RayLog.core.info("Putting object {}, for task {} ", objectId, taskId); + RayLog.core.debug("Putting object {}, for task {} ", objectId, taskId); objectStoreProxy.put(objectId, obj, null); } diff --git a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java index 0cf7e3d0df7b..91937ba14b1e 100644 --- a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java +++ b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java @@ -90,8 +90,8 @@ public TaskSpec getTask() { @Override public void fetchOrReconstruct(List objectIds, boolean fetchOnly, UniqueId currentTaskId) { - if (RayLog.core.isInfoEnabled()) { - RayLog.core.info("Blocked on objects for task {}, object IDs are {}", + if (RayLog.core.isDebugEnabled()) { + RayLog.core.debug("Blocked on objects for task {}, object IDs are {}", UniqueIdUtil.computeTaskId(objectIds.get(0)), objectIds); } nativeFetchOrReconstruct(client, UniqueIdUtil.getIdBytes(objectIds), diff --git a/java/test/src/main/java/org/ray/api/test/StressTest.java b/java/test/src/main/java/org/ray/api/test/StressTest.java new file mode 100644 index 000000000000..4fab74aed199 --- /dev/null +++ b/java/test/src/main/java/org/ray/api/test/StressTest.java @@ -0,0 +1,98 @@ +package org.ray.api.test; + +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.List; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.ray.api.Ray; +import org.ray.api.RayActor; +import org.ray.api.RayObject; +import org.ray.api.id.UniqueId; + +@RunWith(MyRunner.class) +public class StressTest { + + public static int echo(int x) { + return x; + } + + @Test + public void testSubmittingTasks() { + for (int numIterations : ImmutableList.of(1, 10, 100, 1000)) { + int numTasks = 1000 / numIterations; + for (int i = 0; i < numIterations; i++) { + List resultIds = new ArrayList<>(); + for (int j = 0; j < numTasks; j++) { + resultIds.add(Ray.call(StressTest::echo, 1).getId()); + } + for (Integer result : Ray.get(resultIds)) { + Assert.assertEquals(result, Integer.valueOf(1)); + } + } + } + } + + @Test + public void testDependency() { + RayObject x = Ray.call(StressTest::echo, 1); + for (int i = 0; i < 1000; i++) { + x = Ray.call(StressTest::echo, x); + } + Assert.assertEquals(x.get(), Integer.valueOf(1)); + } + + public static class Actor { + + public int ping() { + return 1; + } + } + + public static class Worker { + + private RayActor actor; + + public Worker(RayActor actor) { + this.actor = actor; + } + + public int ping(int n) { + List objectIds = new ArrayList<>(); + for (int i = 0; i < n; i++) { + objectIds.add(Ray.call(Actor::ping, actor).getId()); + } + int sum = 0; + for (Integer result : Ray.get(objectIds)) { + sum += result; + } + return sum; + } + } + + @Test + public void testSubmittingManyTasksToOneActor() { + RayActor actor = Ray.createActor(Actor::new); + List objectIds = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + RayActor worker = Ray.createActor(Worker::new, actor); + objectIds.add(Ray.call(Worker::ping, worker, 100).getId()); + } + for (Integer result : Ray.get(objectIds)) { + Assert.assertEquals(result, Integer.valueOf(100)); + } + } + + @Test + public void testPuttingAndGettingManyObjects() { + Integer objectToPut = 1; + List> objects = new ArrayList<>(); + for (int i = 0; i < 100_000; i++) { + objects.add(Ray.put(objectToPut)); + } + for (RayObject object : objects) { + Assert.assertEquals(object.get(), objectToPut); + } + } +} From c5b5cdae33ad96f5539da231c87660675bcaaef6 Mon Sep 17 00:00:00 2001 From: Philipp Moritz Date: Sun, 2 Dec 2018 01:15:09 +0100 Subject: [PATCH 31/40] Upgrade Arrow to include Plasma TensorFlow Op release fix (#3448) This includes a fix so the TensorFlow op releases memory properly (https://github.com/apache/arrow/pull/3061) and the possibility to store arrow data structures in plasma (https://github.com/apache/arrow/pull/2832). https://github.com/ray-project/ray/issues/3404 --- cmake/Modules/ArrowExternalProject.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/Modules/ArrowExternalProject.cmake b/cmake/Modules/ArrowExternalProject.cmake index 572e12b27b43..5a054afc74f1 100644 --- a/cmake/Modules/ArrowExternalProject.cmake +++ b/cmake/Modules/ArrowExternalProject.cmake @@ -15,10 +15,10 @@ # - PLASMA_SHARED_LIB set(arrow_URL https://github.com/apache/arrow.git) -# The PR for this commit is https://github.com/apache/arrow/pull/2826. We +# The PR for this commit is https://github.com/apache/arrow/pull/3061. We # include the link here to make it easier to find the right commit because # Arrow often rewrites git history and invalidates certain commits. -set(arrow_TAG b4f7ed6d6ed5cdb6dd136bac3181a438f35c8ea0) +set(arrow_TAG a667fca3b71772886bb2595986266d2039823dcc) set(ARROW_INSTALL_PREFIX ${CMAKE_CURRENT_BINARY_DIR}/external/arrow-install) set(ARROW_HOME ${ARROW_INSTALL_PREFIX}) From 13c8ce4d84a08b0995b8d3135550b3c4f1236ff6 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sat, 1 Dec 2018 19:16:45 -0800 Subject: [PATCH 32/40] Update README.rst with 0.6.0 version number. (#3453) --- README.rst | 2 +- doc/source/installation.rst | 20 ++++++++++---------- python/ray/autoscaler/aws/example-full.yaml | 6 +++--- python/ray/autoscaler/gcp/example-full.yaml | 6 +++--- test/stress_tests/stress_testing_config.yaml | 2 +- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/README.rst b/README.rst index c6c0d024174e..7e50123d9bf6 100644 --- a/README.rst +++ b/README.rst @@ -8,7 +8,7 @@ .. image:: https://readthedocs.org/projects/ray/badge/?version=latest :target: http://ray.readthedocs.io/en/latest/?badge=latest -.. image:: https://img.shields.io/badge/pypi-0.5.3-blue.svg +.. image:: https://img.shields.io/badge/pypi-0.6.0-blue.svg :target: https://pypi.org/project/ray/ | diff --git a/doc/source/installation.rst b/doc/source/installation.rst index f3ad649d683d..68bd37ae96f5 100644 --- a/doc/source/installation.rst +++ b/doc/source/installation.rst @@ -37,16 +37,16 @@ Here are links to the latest wheels (which are built off of master). To install =================== =================== -.. _`Linux Python 3.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp37-cp37m-manylinux1_x86_64.whl -.. _`Linux Python 3.6`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp36-cp36m-manylinux1_x86_64.whl -.. _`Linux Python 3.5`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp35-cp35m-manylinux1_x86_64.whl -.. _`Linux Python 3.4`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp34-cp34m-manylinux1_x86_64.whl -.. _`Linux Python 2.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp27-cp27mu-manylinux1_x86_64.whl -.. _`MacOS Python 3.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp37-cp37m-macosx_10_6_intel.whl -.. _`MacOS Python 3.6`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp36-cp36m-macosx_10_6_intel.whl -.. _`MacOS Python 3.5`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp35-cp35m-macosx_10_6_intel.whl -.. _`MacOS Python 3.4`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp34-cp34m-macosx_10_6_intel.whl -.. _`MacOS Python 2.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp27-cp27m-macosx_10_6_intel.whl +.. _`Linux Python 3.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp37-cp37m-manylinux1_x86_64.whl +.. _`Linux Python 3.6`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp36-cp36m-manylinux1_x86_64.whl +.. _`Linux Python 3.5`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp35-cp35m-manylinux1_x86_64.whl +.. _`Linux Python 3.4`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp34-cp34m-manylinux1_x86_64.whl +.. _`Linux Python 2.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp27-cp27mu-manylinux1_x86_64.whl +.. _`MacOS Python 3.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp37-cp37m-macosx_10_6_intel.whl +.. _`MacOS Python 3.6`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp36-cp36m-macosx_10_6_intel.whl +.. _`MacOS Python 3.5`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp35-cp35m-macosx_10_6_intel.whl +.. _`MacOS Python 3.4`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp34-cp34m-macosx_10_6_intel.whl +.. _`MacOS Python 2.7`: https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp27-cp27m-macosx_10_6_intel.whl Building Ray from source diff --git a/python/ray/autoscaler/aws/example-full.yaml b/python/ray/autoscaler/aws/example-full.yaml index 0d04e0dceaee..d74d45823c21 100644 --- a/python/ray/autoscaler/aws/example-full.yaml +++ b/python/ray/autoscaler/aws/example-full.yaml @@ -89,9 +89,9 @@ setup_commands: # has your Ray repo pre-cloned. Then, you can replace the pip installs # below with a git checkout (and possibly a recompile). - echo 'export PATH="$HOME/anaconda3/envs/tensorflow_p36/bin:$PATH"' >> ~/.bashrc - # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp27-cp27mu-manylinux1_x86_64.whl - # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp35-cp35m-manylinux1_x86_64.whl - - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp36-cp36m-manylinux1_x86_64.whl + # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp27-cp27mu-manylinux1_x86_64.whl + # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp35-cp35m-manylinux1_x86_64.whl + - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp36-cp36m-manylinux1_x86_64.whl # Consider uncommenting these if you also want to run apt-get commands during setup # - sudo pkill -9 apt-get || true # - sudo pkill -9 dpkg || true diff --git a/python/ray/autoscaler/gcp/example-full.yaml b/python/ray/autoscaler/gcp/example-full.yaml index a3df6ad612c2..6afbb464fa6a 100644 --- a/python/ray/autoscaler/gcp/example-full.yaml +++ b/python/ray/autoscaler/gcp/example-full.yaml @@ -124,9 +124,9 @@ setup_commands: pip install google-api-python-client==1.6.7 cython==0.27.3 - # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp27-cp27mu-manylinux1_x86_64.whl - # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp35-cp35m-manylinux1_x86_64.whl - # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp36-cp36m-manylinux1_x86_64.whl + # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp27-cp27mu-manylinux1_x86_64.whl + # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp35-cp35m-manylinux1_x86_64.whl + # - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp36-cp36m-manylinux1_x86_64.whl - >- cd ~ && git clone https://github.com/ray-project/ray || true diff --git a/test/stress_tests/stress_testing_config.yaml b/test/stress_tests/stress_testing_config.yaml index 7126366b53fd..6ef5285035e8 100644 --- a/test/stress_tests/stress_testing_config.yaml +++ b/test/stress_tests/stress_testing_config.yaml @@ -96,7 +96,7 @@ setup_commands: # - git clone https://github.com/ray-project/ray || true - pip install boto3==1.4.8 cython==0.27.3 # - cd ray/python; git checkout master; git pull; pip install -e . --verbose - - pip install https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.5.3-cp36-cp36m-manylinux1_x86_64.whl + - pip install https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-0.6.0-cp36-cp36m-manylinux1_x86_64.whl # Custom commands that will be run on the head node after common setup. head_setup_commands: [] From 4abafd7e62774eb8fb44c944e83b9a329d4fb4ff Mon Sep 17 00:00:00 2001 From: Stephanie Wang Date: Sat, 1 Dec 2018 19:40:33 -0800 Subject: [PATCH 33/40] Fix bug in ray.wait (#3445) ray.wait depends on callbacks from the GCS to decide when an object has appeared in the cluster. The raylet crashes if a callback is received for a wait request that has already completed, but this actually can happen, depending on the order of calls. More precisely: 1. Objects A and B are put in the cluster. 2. Client calls ray.wait([A, B], num_returns=1). 3. Client subscribes to locations for A and B. Locations are cached for both, so callbacks are posted for each. 4. Callback for A fires. The wait completes and the request is removed. 5. Callback for B fires. The wait request no longer exists and raylet crashes. --- src/ray/object_manager/object_manager.cc | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 959235c95794..a3cc87c7f17c 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -628,9 +628,13 @@ void ObjectManager::SubscribeRemainingWaitObjects(const UniqueID &wait_id) { const ObjectID &subscribe_object_id) { if (!client_ids.empty()) { auto object_id_wait_state = active_wait_requests_.find(wait_id); - // We never expect to handle a subscription notification for a wait that has - // already completed. - RAY_CHECK(object_id_wait_state != active_wait_requests_.end()); + if (object_id_wait_state == active_wait_requests_.end()) { + // Depending on the timing of calls to the object directory, we + // may get a subscription notification after the wait call has + // already completed. If so, then don't process the + // notification. + return; + } auto &wait_state = object_id_wait_state->second; RAY_CHECK(wait_state.remaining.erase(subscribe_object_id)); wait_state.found.insert(subscribe_object_id); From 7abfbfd2f7c5878aaa7bc878b55ef00d31cf3ebc Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 3 Dec 2018 01:24:36 -0800 Subject: [PATCH 34/40] [rllib] Better error message for unsupported non-atari image observation sizes (#3444) --- python/ray/rllib/models/visionnet.py | 17 ++++++++++------- python/ray/rllib/test/test_catalog.py | 4 ++-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/python/ray/rllib/models/visionnet.py b/python/ray/rllib/models/visionnet.py index 4105af7dd367..1d856e42cec4 100644 --- a/python/ray/rllib/models/visionnet.py +++ b/python/ray/rllib/models/visionnet.py @@ -16,7 +16,7 @@ def _build_layers_v2(self, input_dict, num_outputs, options): inputs = input_dict["obs"] filters = options.get("conv_filters") if not filters: - filters = get_filter_config(options) + filters = get_filter_config(inputs) activation = get_activation_fn(options.get("conv_activation")) @@ -47,7 +47,7 @@ def _build_layers_v2(self, input_dict, num_outputs, options): return flatten(fc2), flatten(fc1) -def get_filter_config(options): +def get_filter_config(inputs): filters_84x84 = [ [16, [8, 8], 4], [32, [4, 4], 2], @@ -58,12 +58,15 @@ def get_filter_config(options): [32, [4, 4], 2], [256, [11, 11], 1], ] - dim = options.get("dim") - if dim == 84: + shape = inputs.shape.as_list()[1:] + if len(shape) == 3 and shape[:2] == [84, 84]: return filters_84x84 - elif dim == 42: + elif len(shape) == 3 and shape[:2] == [42, 42]: return filters_42x42 else: raise ValueError( - "No default configuration for image size={}".format(dim) + - ", you must specify `conv_filters` manually as a model option.") + "No default configuration for obs input {}".format(inputs) + + ", you must specify `conv_filters` manually as a model option. " + "Default configurations are only available for inputs of size " + "[?, 42, 42, K] and [?, 84, 84, K]. You may alternatively want " + "to use a custom model or preprocessor.") diff --git a/python/ray/rllib/test/test_catalog.py b/python/ray/rllib/test/test_catalog.py index 852a02fc4d1e..efa1aba0e2f0 100644 --- a/python/ray/rllib/test/test_catalog.py +++ b/python/ray/rllib/test/test_catalog.py @@ -72,13 +72,13 @@ def testDefaultModels(self): with tf.variable_scope("test1"): p1 = ModelCatalog.get_model({ - "obs": np.zeros((10, 3), dtype=np.float32) + "obs": tf.zeros((10, 3), dtype=tf.float32) }, Box(0, 1, shape=(3, ), dtype=np.float32), 5, {}) self.assertEqual(type(p1), FullyConnectedNetwork) with tf.variable_scope("test2"): p2 = ModelCatalog.get_model({ - "obs": np.zeros((10, 84, 84, 3), dtype=np.float32) + "obs": tf.zeros((10, 84, 84, 3), dtype=tf.float32) }, Box(0, 1, shape=(84, 84, 3), dtype=np.float32), 5, {}) self.assertEqual(type(p2), VisionNetwork) From d8205976e8e5c98e177c0d1d03fefef21a69d5d9 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 3 Dec 2018 19:55:25 -0800 Subject: [PATCH 35/40] [rllib] Auto clip actions to Box space range; deprecate squash_to_range (#3426) * fix clip * tweak wording * remove squash entirely * Update rllib-models.rst * fix argument order * Apply suggestions from code review Co-Authored-By: ericl --- doc/source/rllib-models.rst | 2 +- python/ray/rllib/agents/agent.py | 3 ++ .../ray/rllib/evaluation/policy_evaluator.py | 9 +++- python/ray/rllib/evaluation/sampler.py | 53 ++++++++++++++++--- python/ray/rllib/models/action_dist.py | 44 +-------------- python/ray/rllib/models/catalog.py | 10 ++-- .../ray/rllib/test/test_supported_spaces.py | 26 +++++---- python/ray/rllib/tuned_examples/pong-ppo.yaml | 33 +++++++----- 8 files changed, 101 insertions(+), 79 deletions(-) diff --git a/doc/source/rllib-models.rst b/doc/source/rllib-models.rst index 09c49e2751bf..9e7070b66c48 100644 --- a/doc/source/rllib-models.rst +++ b/doc/source/rllib-models.rst @@ -233,7 +233,7 @@ In this example we overrode existing methods of the existing DDPG policy graph, Variable-length / Parametric Action Spaces ------------------------------------------ -Custom models can be used to work with environments where (1) the set of valid actions varies per step, and/or (2) the number of valid actions is very large, as in `OpenAI Five `__ and `Horizon `__. The general idea is that the meaning of actions can be completely conditioned on the observation, that is, the ``a`` in ``Q(s, a)`` is just a token in ``[0, MAX_AVAIL_ACTIONS)`` that only has meaning in the context of ``s``. This works with algorithms in the `DQN and policy-gradient families `__ and can be implemented as follows: +Custom models can be used to work with environments where (1) the set of valid actions varies per step, and/or (2) the number of valid actions is very large, as in `OpenAI Five `__ and `Horizon `__. The general idea is that the meaning of actions can be completely conditioned on the observation, i.e., the ``a`` in ``Q(s, a)`` becomes just a token in ``[0, MAX_AVAIL_ACTIONS)`` that only has meaning in the context of ``s``. This works with algorithms in the `DQN and policy-gradient families `__ and can be implemented as follows: 1. The environment should return a mask and/or list of valid action embeddings as part of the observation for each step. To enable batching, the number of actions can be allowed to vary from 1 to some max number: diff --git a/python/ray/rllib/agents/agent.py b/python/ray/rllib/agents/agent.py index f0d9510756b9..9dbc267cf5bb 100644 --- a/python/ray/rllib/agents/agent.py +++ b/python/ray/rllib/agents/agent.py @@ -61,6 +61,8 @@ # Whether to clip rewards prior to experience postprocessing. Setting to # None means clip for Atari only. "clip_rewards": None, + # Whether to np.clip() actions to the action space low/high range spec. + "clip_actions": True, # Whether to use rllib or deepmind preprocessors by default "preprocessor_pref": "deepmind", @@ -226,6 +228,7 @@ def session_creator(): num_envs=config["num_envs_per_worker"], observation_filter=config["observation_filter"], clip_rewards=config["clip_rewards"], + clip_actions=config["clip_actions"], env_config=config["env_config"], model_config=config["model"], policy_config=config, diff --git a/python/ray/rllib/evaluation/policy_evaluator.py b/python/ray/rllib/evaluation/policy_evaluator.py index db5f7ee887b2..33d5ee219ca1 100644 --- a/python/ray/rllib/evaluation/policy_evaluator.py +++ b/python/ray/rllib/evaluation/policy_evaluator.py @@ -100,6 +100,7 @@ def __init__(self, num_envs=1, observation_filter="NoFilter", clip_rewards=None, + clip_actions=True, env_config=None, model_config=None, policy_config=None, @@ -155,6 +156,8 @@ def __init__(self, clip_rewards (bool): Whether to clip rewards to [-1, 1] prior to experience postprocessing. Setting to None means clip for Atari only. + clip_actions (bool): Whether to clip action values to the range + specified by the policy action space. env_config (dict): Config to pass to the env creator. model_config (dict): Config to use when creating the policy model. policy_config (dict): Config to pass to the policy. In the @@ -289,7 +292,8 @@ def make_env(vector_index): self.callbacks, horizon=episode_horizon, pack=pack_episodes, - tf_sess=self.tf_sess) + tf_sess=self.tf_sess, + clip_actions=clip_actions) self.sampler.start() else: self.sampler = SyncSampler( @@ -302,7 +306,8 @@ def make_env(vector_index): self.callbacks, horizon=episode_horizon, pack=pack_episodes, - tf_sess=self.tf_sess) + tf_sess=self.tf_sess, + clip_actions=clip_actions) logger.debug("Created evaluator with env {} ({}), policies {}".format( self.async_env, self.env, self.policy_map)) diff --git a/python/ray/rllib/evaluation/sampler.py b/python/ray/rllib/evaluation/sampler.py index 2c6411f33510..0bda18bc0361 100644 --- a/python/ray/rllib/evaluation/sampler.py +++ b/python/ray/rllib/evaluation/sampler.py @@ -2,6 +2,7 @@ from __future__ import division from __future__ import print_function +import gym from collections import defaultdict, namedtuple import logging import numpy as np @@ -47,7 +48,8 @@ def __init__(self, callbacks, horizon=None, pack=False, - tf_sess=None): + tf_sess=None, + clip_actions=True): self.async_vector_env = AsyncVectorEnv.wrap_async(env) self.unroll_length = unroll_length self.horizon = horizon @@ -58,7 +60,8 @@ def __init__(self, self.rollout_provider = _env_runner( self.async_vector_env, self.extra_batches.put, self.policies, self.policy_mapping_fn, self.unroll_length, self.horizon, - self._obs_filters, clip_rewards, pack, callbacks, tf_sess) + self._obs_filters, clip_rewards, clip_actions, pack, callbacks, + tf_sess) self.metrics_queue = queue.Queue() def get_data(self): @@ -104,7 +107,8 @@ def __init__(self, callbacks, horizon=None, pack=False, - tf_sess=None): + tf_sess=None, + clip_actions=True): for _, f in obs_filters.items(): assert getattr(f, "is_concurrent", False), \ "Observation Filter must support concurrent updates." @@ -123,6 +127,7 @@ def __init__(self, self.pack = pack self.tf_sess = tf_sess self.callbacks = callbacks + self.clip_actions = clip_actions def run(self): try: @@ -135,8 +140,8 @@ def _run(self): rollout_provider = _env_runner( self.async_vector_env, self.extra_batches.put, self.policies, self.policy_mapping_fn, self.unroll_length, self.horizon, - self._obs_filters, self.clip_rewards, self.pack, self.callbacks, - self.tf_sess) + self._obs_filters, self.clip_rewards, self.clip_actions, self.pack, + self.callbacks, self.tf_sess) while True: # The timeout variable exists because apparently, if one worker # dies, the other workers won't die with it, unless the timeout is @@ -197,6 +202,7 @@ def _env_runner(async_vector_env, horizon, obs_filters, clip_rewards, + clip_actions, pack, callbacks, tf_sess=None): @@ -217,6 +223,7 @@ def _env_runner(async_vector_env, clip_rewards (bool): Whether to clip rewards before postprocessing. pack (bool): Whether to pack multiple episodes into each batch. This guarantees batches will be exactly `unroll_length` in size. + clip_actions (bool): Whether to clip actions to the space range. callbacks (dict): User callbacks to run on episode events. tf_sess (Session|None): Optional tensorflow session to use for batching TF policy evaluations. @@ -272,7 +279,7 @@ def new_episode(): # Do batched policy eval eval_results = _do_policy_eval(tf_sess, to_eval, policies, - active_episodes) + active_episodes, clip_actions) # Process results and update episode state actions_to_send = _process_policy_eval_results( @@ -413,7 +420,7 @@ def _process_observations(async_vector_env, policies, batch_builder_pool, return active_envs, to_eval, outputs -def _do_policy_eval(tf_sess, to_eval, policies, active_episodes): +def _do_policy_eval(tf_sess, to_eval, policies, active_episodes, clip_actions): """Call compute actions on observation batches to get next actions. Returns: @@ -448,6 +455,13 @@ def _do_policy_eval(tf_sess, to_eval, policies, active_episodes): for k, v in pending_fetches.items(): eval_results[k] = builder.get(v) + if clip_actions: + for policy_id, results in eval_results.items(): + policy = _get_or_raise(policies, policy_id) + actions, rnn_out_cols, pi_info_cols = results + eval_results[policy_id] = (_clip_actions( + actions, policy.action_space), rnn_out_cols, pi_info_cols) + return eval_results @@ -516,6 +530,31 @@ def _fetch_atari_metrics(async_vector_env): return atari_out +def _clip_actions(actions, space): + """Called to clip actions to the specified range of this policy. + + Arguments: + actions: Batch of actions or TupleActions. + space: Action space the actions should be present in. + + Returns: + Clipped batch of actions. + """ + + if isinstance(space, gym.spaces.Box): + return np.clip(actions, space.low, space.high) + elif isinstance(space, gym.spaces.Tuple): + if not isinstance(actions, TupleActions): + raise ValueError("Expected tuple space for actions {}: {}".format( + actions, space)) + out = [] + for a, s in zip(actions.batches, space.spaces): + out.append(_clip_actions(a, s)) + return TupleActions(out) + else: + return actions + + def _unbatch_tuple_actions(action_batch): # convert list of batches -> batch of lists if isinstance(action_batch, TupleActions): diff --git a/python/ray/rllib/models/action_dist.py b/python/ray/rllib/models/action_dist.py index 75a43deeb789..76d45e244151 100644 --- a/python/ray/rllib/models/action_dist.py +++ b/python/ray/rllib/models/action_dist.py @@ -95,19 +95,10 @@ class DiagGaussian(ActionDistribution): second half the gaussian standard deviations. """ - def __init__(self, inputs, low=None, high=None): + def __init__(self, inputs): ActionDistribution.__init__(self, inputs) mean, log_std = tf.split(inputs, 2, axis=1) self.mean = mean - self.low = low - self.high = high - - # Squash to range if specified. We use a sigmoid here this to avoid the - # mean drifting too far past the bounds and causing nan outputs. - # https://github.com/ray-project/ray/issues/1862 - if low is not None: - self.mean = low + tf.sigmoid(self.mean) * (high - low) - self.log_std = log_std self.std = tf.exp(log_std) @@ -131,10 +122,7 @@ def entropy(self): reduction_indices=[1]) def sample(self): - out = self.mean + self.std * tf.random_normal(tf.shape(self.mean)) - if self.low is not None: - out = tf.clip_by_value(out, self.low, self.high) - return out + return self.mean + self.std * tf.random_normal(tf.shape(self.mean)) class Deterministic(ActionDistribution): @@ -147,34 +135,6 @@ def sample(self): return self.inputs -def squash_to_range(dist_cls, low, high): - """Squashes an action distribution to a range in (low, high). - - Arguments: - dist_cls (class): ActionDistribution class to wrap. - low (float|array): Scalar value or array of values. - high (float|array): Scalar value or array of values. - """ - - class SquashToRangeWrapper(dist_cls): - def __init__(self, inputs): - dist_cls.__init__(self, inputs, low=low, high=high) - - def logp(self, x): - return dist_cls.logp(self, x) - - def kl(self, other): - return dist_cls.kl(self, other) - - def entropy(self): - return dist_cls.entropy(self) - - def sample(self): - return dist_cls.sample(self) - - return SquashToRangeWrapper - - class MultiActionDistribution(ActionDistribution): """Action distribution that operates for list of actions. diff --git a/python/ray/rllib/models/catalog.py b/python/ray/rllib/models/catalog.py index 63a7e73890cc..f9e8af2829f8 100644 --- a/python/ray/rllib/models/catalog.py +++ b/python/ray/rllib/models/catalog.py @@ -15,8 +15,7 @@ from ray.rllib.env.external_env import ExternalEnv from ray.rllib.env.vector_env import VectorEnv from ray.rllib.models.action_dist import ( - Categorical, Deterministic, DiagGaussian, MultiActionDistribution, - squash_to_range) + Categorical, Deterministic, DiagGaussian, MultiActionDistribution) from ray.rllib.models.preprocessors import get_preprocessor from ray.rllib.models.fcnet import FullyConnectedNetwork from ray.rllib.models.visionnet import VisionNetwork @@ -38,7 +37,7 @@ "fcnet_hiddens": [256, 256], # For control envs, documented in ray.rllib.models.Model "free_log_std": False, - # Whether to squash the action output to space range + # (deprecated) Whether to use sigmoid to squash actions to space range "squash_to_range": False, # == LSTM == @@ -114,8 +113,9 @@ def get_action_dist(action_space, config, dist_type=None): if dist_type is None: dist = DiagGaussian if config.get("squash_to_range"): - dist = squash_to_range(dist, action_space.low, - action_space.high) + raise ValueError( + "The squash_to_range option is deprecated. See the " + "clip_actions agent option instead.") return dist, action_space.shape[0] * 2 elif dist_type == "deterministic": return Deterministic, action_space.shape[0] diff --git a/python/ray/rllib/test/test_supported_spaces.py b/python/ray/rllib/test/test_supported_spaces.py index b98a006bca3b..7a5e45ef3aa4 100644 --- a/python/ray/rllib/test/test_supported_spaces.py +++ b/python/ray/rllib/test/test_supported_spaces.py @@ -120,12 +120,15 @@ def testAll(self): stats, check_bounds=True) check_support("DQN", {"timesteps_per_iteration": 1}, stats) - check_support("A3C", { - "num_workers": 1, - "optimizer": { - "grads_per_step": 1 - } - }, stats) + check_support( + "A3C", { + "num_workers": 1, + "optimizer": { + "grads_per_step": 1 + } + }, + stats, + check_bounds=True) check_support( "PPO", { "num_workers": 1, @@ -133,9 +136,6 @@ def testAll(self): "train_batch_size": 10, "sample_batch_size": 10, "sgd_minibatch_size": 1, - "model": { - "squash_to_range": True - }, }, stats, check_bounds=True) @@ -153,7 +153,13 @@ def testAll(self): "num_rollouts": 1, "rollouts_used": 1 }, stats) - check_support("PG", {"num_workers": 1, "optimizer": {}}, stats) + check_support( + "PG", { + "num_workers": 1, + "optimizer": {} + }, + stats, + check_bounds=True) num_unexpected_errors = 0 for (alg, a_name, o_name), stat in sorted(stats.items()): if stat not in ["ok", "unsupported"]: diff --git a/python/ray/rllib/tuned_examples/pong-ppo.yaml b/python/ray/rllib/tuned_examples/pong-ppo.yaml index 1447481643fe..d7e273cc6e2b 100644 --- a/python/ray/rllib/tuned_examples/pong-ppo.yaml +++ b/python/ray/rllib/tuned_examples/pong-ppo.yaml @@ -1,17 +1,26 @@ -# On a Tesla K80 GPU, this achieves the maximum reward in about 1-1.5 hours. +# On a single GPU, this achieves maximum reward in ~15-20 minutes. # -# $ python train.py -f tuned_examples/pong-ppo.yaml --ray-num-gpus=1 +# $ python train.py -f tuned_examples/pong-ppo.yaml # -# - PPO_PongDeterministic-v4_0: TERMINATED [pid=16387], 4984 s, 1117981 ts, 21 rew -# - PPO_PongDeterministic-v4_0: TERMINATED [pid=83606], 4592 s, 1068671 ts, 21 rew -# -pong-deterministic-ppo: - env: PongDeterministic-v4 +pong-ppo: + env: PongNoFrameskip-v4 run: PPO - stop: - episode_reward_mean: 21 config: - gamma: 0.99 - num_workers: 4 - num_sgd_iter: 20 + lambda: 0.95 + kl_coeff: 0.5 + clip_rewards: True + clip_param: 0.1 + vf_clip_param: 10.0 + entropy_coeff: 0.01 + train_batch_size: 5000 + sample_batch_size: 20 + sgd_minibatch_size: 500 + num_sgd_iter: 10 + num_workers: 32 + num_envs_per_worker: 5 + batch_mode: truncate_episodes + observation_filter: NoFilter + vf_share_layers: true num_gpus: 1 + model: + dim: 42 From be6567e6fd8fdaab0bf1f72e2255890e94a4f2b3 Mon Sep 17 00:00:00 2001 From: Kristian Hartikainen Date: Mon, 3 Dec 2018 21:39:43 -0800 Subject: [PATCH 36/40] Tweak/exec attach info (#3447) * Add custom cluster name to exec info * Update submit info to match exec info --- python/ray/scripts/scripts.py | 37 +++++++++++++++++++++++++++++------ 1 file changed, 31 insertions(+), 6 deletions(-) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 13fa63efbd40..60b05e4226d2 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -591,9 +591,21 @@ def submit(cluster_config_file, screen, tmux, stop, start, cluster_name, cmd = " ".join(["python", target] + list(script_args)) exec_cluster(cluster_config_file, cmd, screen, tmux, stop, False, cluster_name, port_forward) - if tmux: - logger.info("Use `ray attach {} --tmux` " - "to check on command status.".format(cluster_config_file)) + + if tmux or screen: + attach_command_parts = ["ray attach", cluster_config_file] + if cluster_name is not None: + attach_command_parts.append( + "--cluster-name={}".format(cluster_name)) + if tmux: + attach_command_parts.append("--tmux") + elif screen: + attach_command_parts.append("--screen") + + attach_command = " ".join(attach_command_parts) + attach_info = "Use `{}` to check on command status.".format( + attach_command) + logger.info(attach_info) @cli.command() @@ -627,11 +639,24 @@ def submit(cluster_config_file, screen, tmux, stop, start, cluster_name, def exec_cmd(cluster_config_file, cmd, screen, tmux, stop, start, cluster_name, port_forward): assert not (screen and tmux), "Can specify only one of `screen` or `tmux`." + exec_cluster(cluster_config_file, cmd, screen, tmux, stop, start, cluster_name, port_forward) - if tmux: - logger.info("Use `ray attach {} --tmux` " - "to check on command status.".format(cluster_config_file)) + + if tmux or screen: + attach_command_parts = ["ray attach", cluster_config_file] + if cluster_name is not None: + attach_command_parts.append( + "--cluster-name={}".format(cluster_name)) + if tmux: + attach_command_parts.append("--tmux") + elif screen: + attach_command_parts.append("--screen") + + attach_command = " ".join(attach_command_parts) + attach_info = "Use `{}` to check on command status.".format( + attach_command) + logger.info(attach_info) @cli.command() From ce355d13d4b50481ba1a86b63555fef204e25f9f Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 3 Dec 2018 23:15:43 -0800 Subject: [PATCH 37/40] [rllib] Allow envs to be auto-registered; add on_train_result callback 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 --- .travis.yml | 30 ++-- doc/source/rllib-env.rst | 28 +++- doc/source/rllib-training.rst | 141 ++++++++++++++---- python/ray/rllib/agents/agent.py | 27 +++- .../ray/rllib/examples/carla/a3c_lane_keep.py | 51 ------- .../ray/rllib/examples/carla/dqn_lane_keep.py | 53 ------- .../ray/rllib/examples/carla/ppo_lane_keep.py | 63 -------- python/ray/rllib/examples/carla/train_a3c.py | 6 +- python/ray/rllib/examples/carla/train_dqn.py | 6 +- python/ray/rllib/examples/carla/train_ppo.py | 6 +- python/ray/rllib/examples/custom_env.py | 7 +- .../examples/custom_metrics_and_callbacks.py | 9 ++ python/ray/tune/trial.py | 6 +- 13 files changed, 191 insertions(+), 242 deletions(-) delete mode 100644 python/ray/rllib/examples/carla/a3c_lane_keep.py delete mode 100644 python/ray/rllib/examples/carla/dqn_lane_keep.py delete mode 100644 python/ray/rllib/examples/carla/ppo_lane_keep.py diff --git a/.travis.yml b/.travis.yml index b48089d52ecd..5f45138cbdda 100644 --- a/.travis.yml +++ b/.travis.yml @@ -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 @@ -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 diff --git a/doc/source/rllib-env.rst b/doc/source/rllib-env.rst index ca36186e1a5f..37ea011a0b5c 100644 --- a/doc/source/rllib-env.rst +++ b/doc/source/rllib-env.rst @@ -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 `__, 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 `__. 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 ------------------------ diff --git a/doc/source/rllib-training.rst b/doc/source/rllib-training.rst index dc37d22943ba..e647b0a2791f 100644 --- a/doc/source/rllib-training.rst +++ b/doc/source/rllib-training.rst @@ -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/ 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 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~ @@ -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": { @@ -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), }, }, } @@ -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 `__. 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/ 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 -------- diff --git a/python/ray/rllib/agents/agent.py b/python/ray/rllib/agents/agent.py index 9dbc267cf5bb..b84154f5bbb1 100644 --- a/python/ray/rllib/agents/agent.py +++ b/python/ray/rllib/agents/agent.py @@ -2,12 +2,13 @@ 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 @@ -15,7 +16,7 @@ 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 @@ -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 === @@ -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: @@ -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 @@ -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.""" diff --git a/python/ray/rllib/examples/carla/a3c_lane_keep.py b/python/ray/rllib/examples/carla/a3c_lane_keep.py deleted file mode 100644 index 9629808ba4c7..000000000000 --- a/python/ray/rllib/examples/carla/a3c_lane_keep.py +++ /dev/null @@ -1,51 +0,0 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -import ray -from ray.tune import register_env, run_experiments - -from env import CarlaEnv, ENV_CONFIG -from models import register_carla_model -from scenarios import LANE_KEEP - -env_name = "carla_env" -env_config = ENV_CONFIG.copy() -env_config.update({ - "verbose": False, - "x_res": 80, - "y_res": 80, - "use_depth_camera": False, - "discrete_actions": False, - "server_map": "/Game/Maps/Town02", - "reward_function": "lane_keep", - "enable_planner": False, - "scenarios": [LANE_KEEP], -}) - -register_env(env_name, lambda env_config: CarlaEnv(env_config)) -register_carla_model() - -ray.init() -run_experiments({ - "carla-a3c": { - "run": "A3C", - "env": "carla_env", - "config": { - "env_config": env_config, - "model": { - "custom_model": "carla", - "custom_options": { - "image_shape": [80, 80, 6], - }, - "conv_filters": [ - [16, [8, 8], 4], - [32, [4, 4], 2], - [512, [10, 10], 1], - ], - }, - "gamma": 0.8, - "num_workers": 1, - }, - }, -}) diff --git a/python/ray/rllib/examples/carla/dqn_lane_keep.py b/python/ray/rllib/examples/carla/dqn_lane_keep.py deleted file mode 100644 index 84fed98cd5f9..000000000000 --- a/python/ray/rllib/examples/carla/dqn_lane_keep.py +++ /dev/null @@ -1,53 +0,0 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -import ray -from ray.tune import register_env, run_experiments - -from env import CarlaEnv, ENV_CONFIG -from models import register_carla_model -from scenarios import LANE_KEEP - -env_name = "carla_env" -env_config = ENV_CONFIG.copy() -env_config.update({ - "verbose": False, - "x_res": 80, - "y_res": 80, - "use_depth_camera": False, - "discrete_actions": True, - "server_map": "/Game/Maps/Town02", - "reward_function": "lane_keep", - "enable_planner": False, - "scenarios": [LANE_KEEP], -}) - -register_env(env_name, lambda env_config: CarlaEnv(env_config)) -register_carla_model() - -ray.init() -run_experiments({ - "carla-dqn": { - "run": "DQN", - "env": "carla_env", - "config": { - "env_config": env_config, - "model": { - "custom_model": "carla", - "custom_options": { - "image_shape": [80, 80, 6], - }, - "conv_filters": [ - [16, [8, 8], 4], - [32, [4, 4], 2], - [512, [10, 10], 1], - ], - }, - "timesteps_per_iteration": 100, - "learning_starts": 1000, - "schedule_max_timesteps": 100000, - "gamma": 0.8, - }, - }, -}) diff --git a/python/ray/rllib/examples/carla/ppo_lane_keep.py b/python/ray/rllib/examples/carla/ppo_lane_keep.py deleted file mode 100644 index ac0f6ff8aff0..000000000000 --- a/python/ray/rllib/examples/carla/ppo_lane_keep.py +++ /dev/null @@ -1,63 +0,0 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -import ray -from ray.tune import register_env, run_experiments - -from env import CarlaEnv, ENV_CONFIG -from models import register_carla_model -from scenarios import LANE_KEEP - -env_name = "carla_env" -env_config = ENV_CONFIG.copy() -env_config.update({ - "verbose": False, - "x_res": 80, - "y_res": 80, - "use_depth_camera": False, - "discrete_actions": False, - "server_map": "/Game/Maps/Town02", - "reward_function": "lane_keep", - "enable_planner": False, - "scenarios": [LANE_KEEP], -}) - -register_env(env_name, lambda env_config: CarlaEnv(env_config)) -register_carla_model() - -ray.init() -run_experiments({ - "carla-ppo": { - "run": "PPO", - "env": "carla_env", - "config": { - "env_config": env_config, - "model": { - "custom_model": "carla", - "custom_options": { - "image_shape": [80, 80, 6], - }, - "conv_filters": [ - [16, [8, 8], 4], - [32, [4, 4], 2], - [512, [10, 10], 1], - ], - }, - "num_workers": 1, - "timesteps_per_batch": 2000, - "min_steps_per_task": 100, - "lambda": 0.95, - "clip_param": 0.2, - "num_sgd_iter": 20, - "sgd_stepsize": 0.0001, - "sgd_batchsize": 32, - "devices": ["/gpu:0"], - "tf_session_args": { - "gpu_options": { - "allow_growth": True - } - } - }, - }, -}) diff --git a/python/ray/rllib/examples/carla/train_a3c.py b/python/ray/rllib/examples/carla/train_a3c.py index 2c12cd8245cf..8fbcfbc576d1 100644 --- a/python/ray/rllib/examples/carla/train_a3c.py +++ b/python/ray/rllib/examples/carla/train_a3c.py @@ -3,13 +3,12 @@ from __future__ import print_function import ray -from ray.tune import grid_search, register_env, run_experiments +from ray.tune import grid_search, run_experiments from env import CarlaEnv, ENV_CONFIG from models import register_carla_model from scenarios import TOWN2_STRAIGHT -env_name = "carla_env" env_config = ENV_CONFIG.copy() env_config.update({ "verbose": False, @@ -23,7 +22,6 @@ "scenarios": TOWN2_STRAIGHT, }) -register_env(env_name, lambda env_config: CarlaEnv(env_config)) register_carla_model() redis_address = ray.services.get_node_ip_address() + ":6379" @@ -31,7 +29,7 @@ run_experiments({ "carla-a3c": { "run": "A3C", - "env": "carla_env", + "env": CarlaEnv, "config": { "env_config": env_config, "use_gpu_for_workers": True, diff --git a/python/ray/rllib/examples/carla/train_dqn.py b/python/ray/rllib/examples/carla/train_dqn.py index fa2dba1053aa..27aa65444d38 100644 --- a/python/ray/rllib/examples/carla/train_dqn.py +++ b/python/ray/rllib/examples/carla/train_dqn.py @@ -3,13 +3,12 @@ from __future__ import print_function import ray -from ray.tune import register_env, run_experiments +from ray.tune import run_experiments from env import CarlaEnv, ENV_CONFIG from models import register_carla_model from scenarios import TOWN2_ONE_CURVE -env_name = "carla_env" env_config = ENV_CONFIG.copy() env_config.update({ "verbose": False, @@ -21,7 +20,6 @@ "scenarios": TOWN2_ONE_CURVE, }) -register_env(env_name, lambda env_config: CarlaEnv(env_config)) register_carla_model() ray.init() @@ -35,7 +33,7 @@ def shape_out(spec): run_experiments({ "carla-dqn": { "run": "DQN", - "env": "carla_env", + "env": CarlaEnv, "config": { "env_config": env_config, "model": { diff --git a/python/ray/rllib/examples/carla/train_ppo.py b/python/ray/rllib/examples/carla/train_ppo.py index a9339ca79481..6c49240142c2 100644 --- a/python/ray/rllib/examples/carla/train_ppo.py +++ b/python/ray/rllib/examples/carla/train_ppo.py @@ -3,13 +3,12 @@ from __future__ import print_function import ray -from ray.tune import register_env, run_experiments +from ray.tune import run_experiments from env import CarlaEnv, ENV_CONFIG from models import register_carla_model from scenarios import TOWN2_STRAIGHT -env_name = "carla_env" env_config = ENV_CONFIG.copy() env_config.update({ "verbose": False, @@ -20,14 +19,13 @@ "server_map": "/Game/Maps/Town02", "scenarios": TOWN2_STRAIGHT, }) -register_env(env_name, lambda env_config: CarlaEnv(env_config)) register_carla_model() ray.init(redirect_output=True) run_experiments({ "carla": { "run": "PPO", - "env": "carla_env", + "env": CarlaEnv, "config": { "env_config": env_config, "model": { diff --git a/python/ray/rllib/examples/custom_env.py b/python/ray/rllib/examples/custom_env.py index 66c0288081f9..0d96eef6acb6 100644 --- a/python/ray/rllib/examples/custom_env.py +++ b/python/ray/rllib/examples/custom_env.py @@ -11,7 +11,6 @@ import ray from ray.tune import run_experiments -from ray.tune.registry import register_env class SimpleCorridor(gym.Env): @@ -42,13 +41,13 @@ def step(self, action): if __name__ == "__main__": - env_creator_name = "corridor" - register_env(env_creator_name, lambda config: SimpleCorridor(config)) + # Can also register the env creator function explicitly with: + # register_env("corridor", lambda config: SimpleCorridor(config)) ray.init() run_experiments({ "demo": { "run": "PPO", - "env": "corridor", + "env": SimpleCorridor, # or "corridor" if registered above "config": { "env_config": { "corridor_length": 5, diff --git a/python/ray/rllib/examples/custom_metrics_and_callbacks.py b/python/ray/rllib/examples/custom_metrics_and_callbacks.py index eec7bffb571f..c92ae8783748 100644 --- a/python/ray/rllib/examples/custom_metrics_and_callbacks.py +++ b/python/ray/rllib/examples/custom_metrics_and_callbacks.py @@ -35,6 +35,13 @@ def on_sample_end(info): print("returned sample batch of size {}".format(info["samples"].count)) +def on_train_result(info): + print("agent.train() result: {} -> {} episodes".format( + info["agent"], info["result"]["episodes_this_iter"])) + # you can mutate the result dict to add new fields to return + info["result"]["callback_ok"] = True + + if __name__ == "__main__": parser = argparse.ArgumentParser() parser.add_argument("--num-iters", type=int, default=2000) @@ -54,6 +61,7 @@ def on_sample_end(info): "on_episode_step": tune.function(on_episode_step), "on_episode_end": tune.function(on_episode_end), "on_sample_end": tune.function(on_sample_end), + "on_train_result": tune.function(on_train_result), }, }, } @@ -64,3 +72,4 @@ def on_sample_end(info): print(custom_metrics) assert "mean_pole_angle" in custom_metrics assert type(custom_metrics["mean_pole_angle"]) is float + assert "callback_ok" in trials[0].last_result diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 65683eeb53c7..2f7dd175b483 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -314,8 +314,10 @@ def __repr__(self): def __str__(self): """Combines ``env`` with ``trainable_name`` and ``experiment_tag``.""" if "env" in self.config: - identifier = "{}_{}".format(self.trainable_name, - self.config["env"]) + env = self.config["env"] + if isinstance(env, type): + env = env.__name__ + identifier = "{}_{}".format(self.trainable_name, env) else: identifier = self.trainable_name if self.experiment_tag: From 9d0bd50e78fa937f7c3a66d94365e5d6cdef47c1 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 14:47:31 -0800 Subject: [PATCH 38/40] [tune] Component notification on node failure + Tests (#3414) Changes include: - Notify Components on Requeue - Slight refactoring of Node Failure handling - Better tests --- .travis/install-dependencies.sh | 4 +- python/ray/tune/ray_trial_executor.py | 17 +++- python/ray/tune/test/cluster_tests.py | 76 +++++++----------- .../ray/tune/test/ray_trial_executor_test.py | 9 ++- python/ray/tune/test/trial_runner_test.py | 80 ++++++++++++++++--- python/ray/tune/trial.py | 18 +++-- python/ray/tune/trial_executor.py | 26 +----- python/ray/tune/trial_runner.py | 64 +++++++++++---- 8 files changed, 180 insertions(+), 114 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index 0fb597d4686f..5bae4ba87f8d 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -25,7 +25,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update sudo apt-get install -y cmake pkg-config python-dev python-numpy build-essential autoconf curl libtool unzip @@ -51,7 +51,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed which -s brew diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 7b725e05f342..6b107b17c82f 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -110,19 +110,27 @@ def _stop_trial(self, trial, error=False, error_msg=None, if stop_logger: trial.close_logger() - def start_trial(self, trial, checkpoint_obj=None): - """Starts the trial.""" + def start_trial(self, trial, checkpoint=None): + """Starts the trial. + + Will not return resources if trial repeatedly fails on start. + + Args: + trial (Trial): Trial to be started. + checkpoint (Checkpoint): A Python object or path storing the state + of trial. + """ self._commit_resources(trial.resources) try: - self._start_trial(trial, checkpoint_obj) + self._start_trial(trial, checkpoint) except Exception: logger.exception("Error stopping runner - retrying...") error_msg = traceback.format_exc() time.sleep(2) self._stop_trial(trial, error=True, error_msg=error_msg) try: - self._start_trial(trial) + self._start_trial(trial, checkpoint) except Exception: logger.exception("Error starting runner, aborting!") error_msg = traceback.format_exc() @@ -140,6 +148,7 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): self._stop_trial( trial, error=error, error_msg=error_msg, stop_logger=stop_logger) if prior_status == Trial.RUNNING: + logger.debug("Returning resources for this trial.") self._return_resources(trial.resources) out = self._find_item(self._running, trial) for result_id in out: diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index f9425cc3e301..59f12181b8ff 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -3,45 +3,22 @@ from __future__ import print_function import json -import time import pytest try: import pytest_timeout except ImportError: pytest_timeout = None -from ray.test.cluster_utils import Cluster import ray -from ray import tune +from ray.rllib import _register_all +from ray.test.cluster_utils import Cluster from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner from ray.tune.suggest import BasicVariantGenerator -def register_test_trainable(): - class _Train(tune.Trainable): - def _setup(self, config): - self.state = {"hi": 1} - - def _train(self): - self.state["hi"] += 1 - time.sleep(0.5) - return {} - - def _save(self, path): - return self.state - - def _restore(self, state): - self.state = state - - tune.register_trainable("test", _Train) - - -@pytest.fixture -def start_connected_cluster(): - # Start the Ray processes. - +def _start_new_cluster(): cluster = Cluster( initialize_head=True, connect=True, @@ -51,7 +28,15 @@ def start_connected_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() + # Pytest doesn't play nicely with imports + _register_all() + return cluster + + +@pytest.fixture +def start_connected_cluster(): + # Start the Ray processes. + cluster = _start_new_cluster() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -71,39 +56,36 @@ def start_connected_emptyhead_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() + # Pytest doesn't play nicely with imports + _register_all() yield cluster # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() -@pytest.mark.skipif( - pytest_timeout is None, - reason="Timeout package not installed; skipping test.") -@pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster - assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] - nodes += [cluster.add_node(resources=dict(CPU=1))] - assert cluster.wait_for_nodes() - assert ray.global_state.cluster_resources()["CPU"] == 2 - + assert ray.global_state.cluster_resources()["CPU"] == 1 runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) runner.step() # run 1 + nodes += [cluster.add_node(resources=dict(CPU=1))] + assert cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 2 cluster.remove_node(nodes.pop()) assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 runner.step() # run 2 + assert sum(t.status == Trial.RUNNING for t in runner.get_trials()) == 1 for i in range(5): nodes += [cluster.add_node(resources=dict(CPU=1))] @@ -111,12 +93,7 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 6 runner.step() # 1 result - - for i in range(5): - node = nodes.pop() - cluster.remove_node(node) - assert cluster.wait_for_nodes() - assert ray.global_state.cluster_resources()["CPU"] == 1 + assert sum(t.status == Trial.RUNNING for t in runner.get_trials()) == 2 @pytest.mark.skip("Add this test once reconstruction is fixed") @@ -133,7 +110,7 @@ def test_remove_node_before_result(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 3}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) @@ -179,7 +156,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): } # Test recovery of trial that hasn't been checkpointed - t = Trial("test", **kwargs) + t = Trial("__fake", **kwargs) runner.add_trial(t) runner.step() # start runner.step() # 1 result @@ -199,7 +176,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t.status == Trial.TERMINATED # Test recovery of trial that has been checkpointed - t2 = Trial("test", **kwargs) + t2 = Trial("__fake", **kwargs) runner.add_trial(t2) runner.step() # start runner.step() # 1 result @@ -216,7 +193,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t2.status == Trial.TERMINATED # Test recovery of trial that won't be checkpointed - t3 = Trial("test", **{"stopping_criterion": {"training_iteration": 3}}) + t3 = Trial("__fake", **{"stopping_criterion": {"training_iteration": 3}}) runner.add_trial(t3) runner.step() # start runner.step() # 1 result @@ -238,6 +215,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): """Removing a node in full cluster causes Trial to be requeued.""" cluster = start_connected_emptyhead_cluster node = cluster.add_node(resources=dict(CPU=1)) + assert cluster.wait_for_nodes() runner = TrialRunner(BasicVariantGenerator()) kwargs = { @@ -248,7 +226,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): "max_failures": 1 } - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) diff --git a/python/ray/tune/test/ray_trial_executor_test.py b/python/ray/tune/test/ray_trial_executor_test.py index 35c413e717bb..86c4bb189595 100644 --- a/python/ray/tune/test/ray_trial_executor_test.py +++ b/python/ray/tune/test/ray_trial_executor_test.py @@ -9,8 +9,9 @@ from ray.rllib import _register_all from ray.tune import Trainable from ray.tune.ray_trial_executor import RayTrialExecutor +from ray.tune.registry import _global_registry, TRAINABLE_CLASS from ray.tune.suggest import BasicVariantGenerator -from ray.tune.trial import Trial, Checkpoint +from ray.tune.trial import Trial, Checkpoint, Resources class RayTrialExecutorTest(unittest.TestCase): @@ -50,6 +51,12 @@ def testPauseResume(self): self.trial_executor.stop_trial(trial) self.assertEqual(Trial.TERMINATED, trial.status) + def testStartFailure(self): + _global_registry.register(TRAINABLE_CLASS, "asdf", None) + trial = Trial("asdf", resources=Resources(1, 0)) + self.trial_executor.start_trial(trial) + self.assertEqual(Trial.ERROR, trial.status) + def testPauseResume2(self): """Tests that pausing works for trials being processed.""" trial = Trial("__fake") diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 6b142d354ec7..8e4aa2cea148 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -3,6 +3,7 @@ from __future__ import print_function import os +import sys import time import unittest @@ -25,6 +26,11 @@ SuggestionAlgorithm) from ray.tune.suggest.variant_generator import RecursiveDependencyError +if sys.version_info >= (3, 3): + from unittest.mock import patch +else: + from mock import patch + class TrainableFunctionApiTest(unittest.TestCase): def setUp(self): @@ -845,6 +851,25 @@ def testMaxConcurrentSuggestions(self): self.assertEqual(len(searcher.next_trials()), 0) +def create_mock_components(): + class _MockScheduler(FIFOScheduler): + errored_trials = [] + + def on_trial_error(self, trial_runner, trial): + self.errored_trials += [trial] + + class _MockSearchAlg(BasicVariantGenerator): + errored_trials = [] + + def on_trial_complete(self, trial_id, error=False, **kwargs): + if error: + self.errored_trials += [trial_id] + + searchalg = _MockSearchAlg() + scheduler = _MockScheduler() + return searchalg, scheduler + + class TrialRunnerTest(unittest.TestCase): def tearDown(self): ray.shutdown() @@ -889,16 +914,6 @@ def train(config, reporter): self.assertLessEqual(len(trial.logdir), 200) trial_executor.stop_trial(trial) - def testTrialErrorOnStart(self): - ray.init() - trial_executor = RayTrialExecutor() - _global_registry.register(TRAINABLE_CLASS, "asdf", None) - trial = Trial("asdf", resources=Resources(1, 0)) - try: - trial_executor.start_trial(trial) - except Exception as e: - self.assertIn("a class", str(e)) - def testExtraResources(self): ray.init(num_cpus=4, num_gpus=2) runner = TrialRunner(BasicVariantGenerator()) @@ -1055,7 +1070,9 @@ def testThrowOnOverstep(self): def testFailureRecoveryDisabled(self): ray.init(num_cpus=1, num_gpus=1) - runner = TrialRunner(BasicVariantGenerator()) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) kwargs = { "resources": Resources(cpu=1, gpu=1), "checkpoint_freq": 1, @@ -1074,10 +1091,15 @@ def testFailureRecoveryDisabled(self): runner.step() self.assertEqual(trials[0].status, Trial.ERROR) self.assertEqual(trials[0].num_failures, 1) + self.assertEqual(len(searchalg.errored_trials), 1) + self.assertEqual(len(scheduler.errored_trials), 1) def testFailureRecoveryEnabled(self): ray.init(num_cpus=1, num_gpus=1) - runner = TrialRunner(BasicVariantGenerator()) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) + kwargs = { "resources": Resources(cpu=1, gpu=1), "checkpoint_freq": 1, @@ -1098,6 +1120,40 @@ def testFailureRecoveryEnabled(self): self.assertEqual(trials[0].num_failures, 1) runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) + self.assertEqual(len(searchalg.errored_trials), 0) + self.assertEqual(len(scheduler.errored_trials), 0) + + def testFailureRecoveryNodeRemoval(self): + ray.init(num_cpus=1, num_gpus=1) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) + + kwargs = { + "resources": Resources(cpu=1, gpu=1), + "checkpoint_freq": 1, + "max_failures": 1, + "config": { + "mock_error": True, + }, + } + runner.add_trial(Trial("__fake", **kwargs)) + trials = runner.get_trials() + + with patch('ray.global_state.cluster_resources') as resource_mock: + resource_mock.return_value = {"CPU": 1, "GPU": 1} + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + + # Mimic a node failure + resource_mock.return_value = {"CPU": 0, "GPU": 0} + runner.step() + self.assertEqual(trials[0].status, Trial.PENDING) + self.assertEqual(trials[0].num_failures, 1) + self.assertEqual(len(searchalg.errored_trials), 0) + self.assertEqual(len(scheduler.errored_trials), 1) def testFailureRecoveryMaxFailures(self): ray.init(num_cpus=1, num_gpus=1) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 2f7dd175b483..f60fd25f2dba 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -216,17 +216,19 @@ def should_stop(self, result): return False - def should_checkpoint(self, result): + def should_checkpoint(self): """Whether this trial is due for checkpointing.""" + result = self.last_result or {} if result.get(DONE) and self.checkpoint_at_end: return True - if not self.checkpoint_freq: + if self.checkpoint_freq: + return result.get(TRAINING_ITERATION, + 0) % self.checkpoint_freq == 0 + else: return False - return self.last_result[TRAINING_ITERATION] % self.checkpoint_freq == 0 - def progress_string(self): """Returns a progress message for printing out to the console.""" @@ -281,10 +283,12 @@ def has_checkpoint(self): def should_recover(self): """Returns whether the trial qualifies for restoring. - This is if a checkpoint frequency is set, which includes settings - where there may not yet be a checkpoint. + This is if a checkpoint frequency is set and has not failed more than + max_failures. This may return true even when there may not yet + be a checkpoint. """ - return self.checkpoint_freq > 0 + return (self.checkpoint_freq > 0 + and self.num_failures < self.max_failures) def update_last_result(self, result, terminate=False): if terminate: diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index e0b541218bf1..063129780b47 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -32,12 +32,10 @@ def has_resources(self, resources): "has_resources() method") def start_trial(self, trial, checkpoint=None): - """Starts the trial restoring from checkpoint if checkpoint != None. - - If an error is encountered when starting the trial, an exception will - be thrown. + """Starts the trial restoring from checkpoint if checkpoint is provided. Args: + trial (Trial): Trial to be started. checkpoint(Checkpoint): A Python object or path storing the state of trial. """ @@ -59,26 +57,6 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): raise NotImplementedError("Subclasses of TrialExecutor must provide " "stop_trial() method") - def restart_trial(self, trial, error_msg=None): - """Restarts or requeues the trial. - - The state of the trial should restore from the last checkpoint. Trial - is requeued if the cluster no longer has resources to accomodate it. - - Args: - error_msg (str): Optional error message. - """ - self.stop_trial( - trial, - error=error_msg is not None, - error_msg=error_msg, - stop_logger=False) - trial.result_logger.flush() - if self.has_resources(trial.resources): - self.start_trial(trial) - else: - trial.status = Trial.PENDING - def continue_training(self, trial): """Continues the training of this trial.""" pass diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index d89b3dda7ee1..84457ff8d9e9 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -12,7 +12,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor from ray.tune.result import TIME_THIS_ITER_S -from ray.tune.trial import Trial +from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -279,17 +279,14 @@ def _process_events(self): result, terminate=(decision == TrialScheduler.STOP)) if decision == TrialScheduler.CONTINUE: - if trial.should_checkpoint(result): - # TODO(rliaw): This is a blocking call - self.trial_executor.save(trial) + self._checkpoint_if_needed(trial) self.trial_executor.continue_training(trial) elif decision == TrialScheduler.PAUSE: self.trial_executor.pause_trial(trial) elif decision == TrialScheduler.STOP: # Checkpoint before ending the trial # if checkpoint_at_end experiment option is set to True - if trial.should_checkpoint(result): - self.trial_executor.save(trial) + self._checkpoint_if_needed(trial) self.trial_executor.stop_trial(trial) else: assert False, "Invalid scheduling decision: {}".format( @@ -298,24 +295,61 @@ def _process_events(self): logger.exception("Error processing event.") error_msg = traceback.format_exc() if trial.status == Trial.RUNNING: - if trial.should_recover() and \ - trial.num_failures < trial.max_failures: + if trial.should_recover(): self._try_recover(trial, error_msg) else: self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( trial.trial_id, error=True) - self.trial_executor.stop_trial(trial, True, error_msg) + self.trial_executor.stop_trial( + trial, error=True, error_msg=error_msg) + + def _checkpoint_if_needed(self, trial): + """Checkpoints trial based off trial.last_result.""" + if trial.should_checkpoint(): + # Save trial runtime if possible + if hasattr(trial, "runner") and trial.runner: + self.trial_executor.save(trial, storage=Checkpoint.DISK) def _try_recover(self, trial, error_msg): + """Tries to recover trial. + + Notifies SearchAlgorithm and Scheduler if failure to recover. + + Args: + trial (Trial): Trial to recover. + error_msg (str): Error message from prior to invoking this method. + """ try: - logger.info("Attempting to recover" - " trial state from last checkpoint.") - self.trial_executor.restart_trial(trial, error_msg) + self.trial_executor.stop_trial( + trial, + error=error_msg is not None, + error_msg=error_msg, + stop_logger=False) + trial.result_logger.flush() + if self.trial_executor.has_resources(trial.resources): + logger.info("Attempting to recover" + " trial state from last checkpoint.") + self.trial_executor.start_trial(trial) + if trial.status == Trial.ERROR: + raise RuntimeError("Trial did not start correctly.") + else: + logger.debug("Notifying Scheduler and requeueing trial.") + self._requeue_trial(trial) except Exception: - error_msg = traceback.format_exc() - logger.warning("Error recovering trial from checkpoint, abort.") - self.trial_executor.stop_trial(trial, True, error_msg=error_msg) + logger.exception("Error recovering trial from checkpoint, abort.") + self._scheduler_alg.on_trial_error(self, trial) + self._search_alg.on_trial_complete(trial.trial_id, error=True) + + def _requeue_trial(self, trial): + """Notification to TrialScheduler and requeue trial. + + This does not notify the SearchAlgorithm because + the function evaluation is still in progress. + """ + self._scheduler_alg.on_trial_error(self, trial) + trial.status = Trial.PENDING + self._scheduler_alg.on_trial_add(self, trial) def _update_trial_queue(self, blocking=False, timeout=600): """Adds next trials to queue if possible. From 3d0a2e32c63a6b63d9e46066e378072b29931770 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 30 Nov 2018 12:27:38 -0800 Subject: [PATCH 39/40] try recover --- python/ray/tune/ray_trial_executor.py | 2 +- python/ray/tune/trial_runner.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 19cff512144c..451f5638da6c 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -135,7 +135,7 @@ def start_trial(self, trial, checkpoint=None, raise_on_failure=False): self._stop_trial(trial, error=True, error_msg=error_msg) try: self._start_trial(trial, checkpoint) - except Exception as exc: + except Exception: logger.exception("Error starting runner, aborting!") error_msg = traceback.format_exc() self._stop_trial(trial, error=True, error_msg=error_msg) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 9cf597e3b4b8..3adcde22da79 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -359,7 +359,7 @@ def _process_events(self): error_msg = traceback.format_exc() if trial.status == Trial.RUNNING: if trial.should_recover(): - self.try_recover(trial, error_msg) + self._try_recover(trial, error_msg) else: self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( @@ -375,7 +375,7 @@ def _checkpoint_if_needed(self, trial): self.trial_executor.save(trial, storage=Checkpoint.DISK) self.trial_executor.try_checkpoint_metadata(trial) - def try_recover(self, trial, error_msg): + def _try_recover(self, trial, error_msg): """Tries to recover trial. Notifies SearchAlgorithm and Scheduler if failure to recover. From 4bb938f4522e00b379aaaba1ac3f1c0ef9e2d45e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 15:18:13 -0800 Subject: [PATCH 40/40] merge --- .travis/install-dependencies.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index d7b68ecc6dae..5bae4ba87f8d 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -24,7 +24,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then wget https://repo.continuum.io/miniconda/Miniconda2-4.5.4-Linux-x86_64.sh -O miniconda.sh -nv bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" - pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update @@ -50,7 +50,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then wget https://repo.continuum.io/miniconda/Miniconda2-4.5.4-MacOSX-x86_64.sh -O miniconda.sh -nv bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" - pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed