From e6c3f8590693aff521c3015235c14cfe6c3053ec Mon Sep 17 00:00:00 2001 From: Ruiyang Wang <56065503+rynewang@users.noreply.github.com> Date: Wed, 22 May 2024 23:03:19 -0700 Subject: [PATCH 01/65] [core] Deflake test_get_locations on windows. (#45506) Signed-off-by: Ruiyang Wang --- python/ray/tests/test_get_locations.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/tests/test_get_locations.py b/python/ray/tests/test_get_locations.py index 74e1e149fea9..c833f07cbdad 100644 --- a/python/ray/tests/test_get_locations.py +++ b/python/ray/tests/test_get_locations.py @@ -151,8 +151,7 @@ def task(): class BigObject: def __init__(self): - # 100 MiB of memory used... - self.data = np.zeros((100 * 1024 * 1024), dtype=np.uint8) + self.data = np.zeros((BIG_OBJ_SIZE,), dtype=np.uint8) @ray.remote From d091bb5df7b0a8ec10d11ed53f44129f621c0c7d Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Thu, 23 May 2024 00:30:57 -0700 Subject: [PATCH 02/65] [Data] Allow user to configure timeout for actor pool (#45508) If your Dataset waits more than 10 minutes to launch GPU actors, your program fails. This PR allows you to configure this behavior so that you can wait longer. Signed-off-by: Balaji Veeramani --- .../execution/operators/actor_pool_map_operator.py | 7 ++----- python/ray/data/context.py | 7 +++++++ python/ray/data/tests/test_actor_pool_map_operator.py | 7 ++----- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py b/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py index 64a37d495a05..908a938abe80 100644 --- a/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py +++ b/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py @@ -28,10 +28,6 @@ # fairly high since streaming backpressure prevents us from overloading actors. DEFAULT_MAX_TASKS_IN_FLIGHT = 4 -# The default time to wait for minimum requested actors -# to start before raising a timeout, in seconds. -DEFAULT_WAIT_FOR_MIN_ACTORS_SEC = 60 * 10 - class ActorPoolMapOperator(MapOperator): """A MapOperator implementation that executes tasks on an actor pool. @@ -132,7 +128,8 @@ def start(self, options: ExecutionOptions): # upstream operators, leading to a spike in memory usage prior to steady state. logger.debug(f"{self._name}: Waiting for {len(refs)} pool actors to start...") try: - ray.get(refs, timeout=DEFAULT_WAIT_FOR_MIN_ACTORS_SEC) + timeout = DataContext.get_current().wait_for_min_actors_s + ray.get(refs, timeout=timeout) except ray.exceptions.GetTimeoutError: raise ray.exceptions.GetTimeoutError( "Timed out while starting actors. " diff --git a/python/ray/data/context.py b/python/ray/data/context.py index 6c68595cf024..a550c17eb7cb 100644 --- a/python/ray/data/context.py +++ b/python/ray/data/context.py @@ -128,6 +128,10 @@ # calls if the URI is an S3 URI. DEFAULT_S3_TRY_CREATE_DIR = False +DEFAULT_WAIT_FOR_MIN_ACTORS_S = env_integer( + "RAY_DATA_DEFAULT_WAIT_FOR_MIN_ACTORS_S", 60 * 10 +) + def _execution_options_factory() -> "ExecutionOptions": # Lazily import to avoid circular dependencies. @@ -224,6 +228,8 @@ class DataContext: execution starts. s3_try_create_dir: If ``True``, try to create directories on S3 when a write call is made with a S3 URI. + wait_for_min_actors_s: The default time to wait for minimum requested + actors to start before raising a timeout, in seconds. """ target_max_block_size: int = DEFAULT_TARGET_MAX_BLOCK_SIZE @@ -269,6 +275,7 @@ class DataContext: ) print_on_execution_start: bool = True s3_try_create_dir: bool = DEFAULT_S3_TRY_CREATE_DIR + wait_for_min_actors_s: int = DEFAULT_WAIT_FOR_MIN_ACTORS_S def __post_init__(self): # The additonal ray remote args that should be added to diff --git a/python/ray/data/tests/test_actor_pool_map_operator.py b/python/ray/data/tests/test_actor_pool_map_operator.py index 07eebfe0dccb..ee162024d4d9 100644 --- a/python/ray/data/tests/test_actor_pool_map_operator.py +++ b/python/ray/data/tests/test_actor_pool_map_operator.py @@ -515,7 +515,7 @@ def test_locality_manager_busyness_ranking(self): assert res3 is None -def test_start_actor_timeout(ray_start_regular_shared): +def test_start_actor_timeout(ray_start_regular_shared, restore_data_context): """Tests that ActorPoolMapOperator raises an exception on timeout while waiting for actors.""" @@ -523,11 +523,9 @@ class UDFClass: def __call__(self, x): return x - from ray.data._internal.execution.operators import actor_pool_map_operator from ray.exceptions import GetTimeoutError - original_timeout = actor_pool_map_operator.DEFAULT_WAIT_FOR_MIN_ACTORS_SEC - actor_pool_map_operator.DEFAULT_WAIT_FOR_MIN_ACTORS_SEC = 1 + ray.data.DataContext.get_current().wait_for_min_actors_s = 1 with pytest.raises( GetTimeoutError, @@ -544,7 +542,6 @@ def __call__(self, x): compute=ray.data.ActorPoolStrategy(size=5), num_gpus=100, ).take_all() - actor_pool_map_operator.DEFAULT_WAIT_FOR_MIN_ACTORS_SEC = original_timeout if __name__ == "__main__": From 2bd35d76175e22a129566ba1677f08d80eb2b94c Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Thu, 23 May 2024 08:43:27 -0700 Subject: [PATCH 03/65] Revert "[ci][microcheck] include step id from all step job flavors" (#45514) Reverts ray-project/ray#45403. This seems like not a good idea. Run between different test flavors do not provide a lot of signals, but there are a lot of those so it is notably more expensive. Test: - https://buildkite.com/ray-project/microcheck/builds/488 Signed-off-by: can --- .buildkite/serve.rayci.yml | 2 +- release/ray_release/test.py | 15 +++------------ release/ray_release/tests/test_test.py | 15 ++++++--------- 3 files changed, 10 insertions(+), 22 deletions(-) diff --git a/.buildkite/serve.rayci.yml b/.buildkite/serve.rayci.yml index bc545ed0c5d2..97c7cd14a421 100644 --- a/.buildkite/serve.rayci.yml +++ b/.buildkite/serve.rayci.yml @@ -57,7 +57,7 @@ steps: depends_on: servepydantic1build - label: ":ray-serve: serve: python {{matrix.python}} tests ({{matrix.worker_id}})" - if: build.pull_request.labels includes "continuous-build" || pipeline.id == "0189e759-8c96-4302-b6b5-b4274406bf89" + if: build.pull_request.labels includes "continuous-build" || pipeline.id == "0189e759-8c96-4302-b6b5-b4274406bf89" || pipeline.id == "018f4f1e-1b73-4906-9802-92422e3badaa" tags: - serve - python diff --git a/release/ray_release/test.py b/release/ray_release/test.py index 343e02dda7f1..868adfac0cb5 100644 --- a/release/ray_release/test.py +++ b/release/ray_release/test.py @@ -204,19 +204,10 @@ def gen_high_impact_tests(cls, prefix: str) -> Dict[str, List]: ] step_id_to_tests = {} for test in high_impact_tests: - recent_results = test.get_test_results() - if not recent_results: + step_id = test.get_test_results(limit=1)[0].rayci_step_id + if not step_id: continue - recent_commit = recent_results[0].commit - for result in recent_results: - # consider all results with the same recent commit; this is to make sure - # we will include different job flavors of the same test - if result.commit != recent_commit: - continue - step_id = result.rayci_step_id - if not step_id: - continue - step_id_to_tests[step_id] = step_id_to_tests.get(step_id, []) + [test] + step_id_to_tests[step_id] = step_id_to_tests.get(step_id, []) + [test] return step_id_to_tests diff --git a/release/ray_release/tests/test_test.py b/release/ray_release/tests/test_test.py index 229bb8b4f108..b9a756b5b3ed 100644 --- a/release/ray_release/tests/test_test.py +++ b/release/ray_release/tests/test_test.py @@ -61,11 +61,11 @@ def _stub_test(val: dict) -> Test: def _stub_test_result( - status: ResultStatus = ResultStatus.SUCCESS, rayci_step_id="123", commit="456" + status: ResultStatus = ResultStatus.SUCCESS, rayci_step_id="123" ) -> TestResult: return TestResult( status=status.value, - commit=commit, + commit="1234567890", branch="master", url="url", timestamp=0, @@ -340,7 +340,7 @@ def gen_high_impact_tests(mock_gen_from_s3) -> None: "name": "core_test", Test.KEY_IS_HIGH_IMPACT: "false", "test_results": [ - _stub_test_result(rayci_step_id="corebuild", commit="123"), + _stub_test_result(rayci_step_id="corebuild"), ], } ) @@ -349,9 +349,7 @@ def gen_high_impact_tests(mock_gen_from_s3) -> None: "name": "data_test_01", Test.KEY_IS_HIGH_IMPACT: "true", "test_results": [ - _stub_test_result(rayci_step_id="databuild", commit="123"), - _stub_test_result(rayci_step_id="data15build", commit="123"), - _stub_test_result(rayci_step_id="data12build", commit="456"), + _stub_test_result(rayci_step_id="databuild"), ], } ) @@ -360,7 +358,7 @@ def gen_high_impact_tests(mock_gen_from_s3) -> None: "name": "data_test_02", Test.KEY_IS_HIGH_IMPACT: "true", "test_results": [ - _stub_test_result(rayci_step_id="databuild", commit="789"), + _stub_test_result(rayci_step_id="databuild"), ], } ) @@ -368,8 +366,7 @@ def gen_high_impact_tests(mock_gen_from_s3) -> None: mock_gen_from_s3.return_value = [core_test, data_test_01, data_test_02] assert Test.gen_high_impact_tests("linux") == { - "databuild": [data_test_01, data_test_02], - "data15build": [data_test_01], + "databuild": [data_test_01, data_test_02] } From 4ebbcbf8d884d9b8b839ff96f119061b41c91989 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Thu, 23 May 2024 18:33:19 +0200 Subject: [PATCH 04/65] [RLlib] Fix Single/MultiAgentEnvRunner missing env-to-module connector call in `_sample_episodes()`. (#45517) --- rllib/algorithms/callbacks.py | 22 ++++++++++--- rllib/env/multi_agent_env_runner.py | 48 ++++++++++++++++++++-------- rllib/env/single_agent_env_runner.py | 27 ++++++++++++++-- 3 files changed, 75 insertions(+), 22 deletions(-) diff --git a/rllib/algorithms/callbacks.py b/rllib/algorithms/callbacks.py index 6e8b61c1c8eb..e0d595be3c17 100644 --- a/rllib/algorithms/callbacks.py +++ b/rllib/algorithms/callbacks.py @@ -384,12 +384,24 @@ def on_episode_end( The exact time of the call of this callback is after `env.step([action])` and also after the results of this step (observation, reward, terminated, truncated, infos) have been logged to the given `episode` object, where either terminated - or truncated were True. + or truncated were True: - Note that on the new API stack, this callback is always preceeded by an - `on_episode_step` call, which comes before the call to this method, but is - provided with the non-finalized episode object (meaning the data has NOT - been converted to numpy arrays yet). + - The env is stepped: `final_obs, rewards, ... = env.step([action])` + + - The step results are logged `episode.add_env_step(final_obs, rewards)` + + - Callback `on_episode_step` is fired. + + - Another env-to-module connector call is made (even though we won't need any + RLModule forward pass anymore). We make this additional call to ensure that in + case users use the connector pipeline to process observations (and write them + back into the episode), the episode object has all observations - even the + terminal one - properly processed. + + - ---> This callback `on_episode_end()` is fired. <--- + + - The episode is finalized (i.e. lists of obs/rewards/actions/etc.. are + converted into numpy arrays). Args: episode: The terminated/truncated SingleAgent- or MultiAgentEpisode object diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index e60300c11a4b..3e76e8b0025b 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -320,6 +320,10 @@ def _sample_timesteps( extra_model_outputs=extra_model_outputs, ) + # Make the `on_episode_step` callback (before finalizing the episode + # object). + self._make_on_episode_callback("on_episode_step") + # Episode is done for all agents. Wrap up the old one and create a new # one (and reset it) to continue. if self._episode.is_done: @@ -329,17 +333,20 @@ def _sample_timesteps( # a call and in case the structure of the observations change # sufficiently, the following `finalize()` call on the episode will # fail. - self._env_to_module( - episodes=[self._episode], - explore=explore, - rl_module=self.module, - shared_data=self._shared_data, - ) + if self.module is not None: + self._env_to_module( + episodes=[self._episode], + explore=explore, + rl_module=self.module, + shared_data=self._shared_data, + ) - # Make the `on_episode_step` and `on_episode_end` callbacks (before - # finalizing the episode object). - self._make_on_episode_callback("on_episode_step") + # Make the `on_episode_end` callback (before finalizing the episode, + # but after(!) the last env-to-module connector call has been made. + # -> All obs (even the terminal one) should have been processed now (by + # the connector, if applicable). self._make_on_episode_callback("on_episode_end") + # Finalize (numpy'ize) the episode. self._episode.finalize(drop_zero_len_single_agent_episodes=True) done_episodes_to_return.append(self._episode) @@ -356,10 +363,6 @@ def _sample_timesteps( # Make the `on_episode_start` callback. self._make_on_episode_callback("on_episode_start") - else: - # Make the `on_episode_step` callback. - self._make_on_episode_callback("on_episode_step") - # Already perform env-to-module connector call for next call to # `_sample_timesteps()`. See comment in c'tor for `self._cached_to_module`. if self.module is not None: @@ -531,7 +534,24 @@ def _sample_episodes( # Increase episode count. eps += 1 - # Make `on_episode_end` callback before finalizing the episode. + # We have to perform an extra env-to-module pass here, just in case + # the user's connector pipeline performs (permanent) transforms + # on each observation (including this final one here). Without such + # a call and in case the structure of the observations change + # sufficiently, the following `finalize()` call on the episode will + # fail. + if self.module is not None: + self._env_to_module( + episodes=[_episode], + explore=explore, + rl_module=self.module, + shared_data=_shared_data, + ) + + # Make the `on_episode_end` callback (before finalizing the episode, + # but after(!) the last env-to-module connector call has been made. + # -> All obs (even the terminal one) should have been processed now (by + # the connector, if applicable). self._make_on_episode_callback("on_episode_end") # Finish the episode. diff --git a/rllib/env/single_agent_env_runner.py b/rllib/env/single_agent_env_runner.py index 1482da133bd0..065e70781ae5 100644 --- a/rllib/env/single_agent_env_runner.py +++ b/rllib/env/single_agent_env_runner.py @@ -334,6 +334,10 @@ def _sample_timesteps( truncated=truncateds[env_index], extra_model_outputs=extra_model_output, ) + # Make the `on_episode_step` and `on_episode_end` callbacks (before + # finalizing the episode object). + self._make_on_episode_callback("on_episode_step", env_index) + # We have to perform an extra env-to-module pass here, just in case # the user's connector pipeline performs (permanent) transforms # on each observation (including this final one here). Without such @@ -347,9 +351,7 @@ def _sample_timesteps( rl_module=self.module, shared_data=self._shared_data, ) - # Make the `on_episode_step` and `on_episode_end` callbacks (before - # finalizing the episode object). - self._make_on_episode_callback("on_episode_step", env_index) + self._make_on_episode_callback("on_episode_end", env_index) # Then finalize (numpy'ize) the episode. @@ -524,6 +526,25 @@ def _sample_episodes( self._make_on_episode_callback( "on_episode_step", env_index, episodes ) + + # We have to perform an extra env-to-module pass here, just in case + # the user's connector pipeline performs (permanent) transforms + # on each observation (including this final one here). Without such + # a call and in case the structure of the observations change + # sufficiently, the following `finalize()` call on the episode will + # fail. + if self.module is not None: + self._env_to_module( + episodes=[episodes[env_index]], + explore=explore, + rl_module=self.module, + shared_data=_shared_data, + ) + + # Make the `on_episode_end` callback (before finalizing the episode, + # but after(!) the last env-to-module connector call has been made. + # -> All obs (even the terminal one) should have been processed now + # (by the connector, if applicable). self._make_on_episode_callback( "on_episode_end", env_index, episodes ) From d8f5aeb16ff502c2eccb29def5802eccc00313d3 Mon Sep 17 00:00:00 2001 From: Gene Der Su Date: Thu, 23 May 2024 10:44:16 -0700 Subject: [PATCH 05/65] [Serve] keep serve alive with bad deployment on `serve run --reload cli` (#45483) ## Why are these changes needed? Put the file watching code inside a try-except block to prevent bad deployment shutdown serve. Also added a test to ensure this behavior. ## Related issue number Closes https://github.com/ray-project/ray/issues/45204 ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Gene Su --- python/ray/serve/scripts.py | 28 +++++++++++++++++----------- python/ray/serve/tests/test_cli_2.py | 25 +++++++++++++++++++++---- 2 files changed, 38 insertions(+), 15 deletions(-) diff --git a/python/ray/serve/scripts.py b/python/ray/serve/scripts.py index 4132c9d12fcd..dbad99756546 100644 --- a/python/ray/serve/scripts.py +++ b/python/ray/serve/scripts.py @@ -563,17 +563,23 @@ def run( yield_on_timeout=True, ): if changes: - cli_logger.info( - f"Detected file change in path {watch_dir}. Redeploying app." - ) - # The module needs to be reloaded with `importlib` in order to pick - # up any changes. - app = _private_api.call_app_builder_with_args_if_necessary( - import_attr(import_path, reload_module=True), args_dict - ) - serve.run( - target=app, blocking=True, name=name, route_prefix=route_prefix - ) + try: + # The module needs to be reloaded with `importlib` in order to + # pick up any changes. + app = _private_api.call_app_builder_with_args_if_necessary( + import_attr(import_path, reload_module=True), args_dict + ) + serve.run( + target=app, + blocking=False, + name=name, + route_prefix=route_prefix, + ) + except Exception: + traceback.print_exc() + cli_logger.error( + "Deploying the latest version of the application failed." + ) except KeyboardInterrupt: cli_logger.info("Got KeyboardInterrupt, shutting down...") diff --git a/python/ray/serve/tests/test_cli_2.py b/python/ray/serve/tests/test_cli_2.py index c5f842400b31..3d45ea2ea31a 100644 --- a/python/ray/serve/tests/test_cli_2.py +++ b/python/ray/serve/tests/test_cli_2.py @@ -49,13 +49,17 @@ def ping_endpoint(endpoint: str, params: str = ""): return CONNECTION_ERROR_MSG -def check_app_running(app_name: str): +def check_app_status(app_name: str, expected_status: str): status_response = subprocess.check_output(["serve", "status"]) status = yaml.safe_load(status_response)["applications"] - assert status[app_name]["status"] == "RUNNING" + assert status[app_name]["status"] == expected_status return True +def check_app_running(app_name: str): + return check_app_status(app_name, "RUNNING") + + @pytest.mark.skipif(sys.platform == "win32", reason="File path incorrect on Windows.") def test_status_multi_app(ray_start_stop): """Deploys a multi-app config file and checks their status.""" @@ -813,6 +817,7 @@ def test_run_reload_basic(ray_start_stop, tmp_path): @serve.deployment class MessageDeployment: def __init__(self, msg): + {invalid_suffix} self.msg = msg def __call__(self): @@ -822,9 +827,9 @@ def __call__(self): msg_app = MessageDeployment.bind("Hello {message}!") """ - def write_file(message: str): + def write_file(message: str, invalid_suffix: str = ""): with open(os.path.join(tmp_path, "reload_serve.py"), "w") as f: - code = code_template.format(message=message) + code = code_template.format(invalid_suffix=invalid_suffix, message=message) print(f"Writing updated code:\n{code}") f.write(code) f.flush() @@ -851,6 +856,18 @@ def write_file(message: str): write_file("Updated") wait_for_condition(lambda: ping_endpoint("") == "Hello Updated!", timeout=10) + # Ensure a bad change doesn't shut down serve and serve reports deploy failed. + write_file(message="update1", invalid_suffix="foobar") + wait_for_condition( + condition_predictor=check_app_status, + app_name="default", + expected_status="DEPLOY_FAILED", + ) + + # Ensure the following reload happens as expected. + write_file("Updated2") + wait_for_condition(lambda: ping_endpoint("") == "Hello Updated2!", timeout=10) + p.send_signal(signal.SIGINT) p.wait() assert ping_endpoint("") == CONNECTION_ERROR_MSG From 4fb670b84dc8b52a50a268bf05d3593189568b38 Mon Sep 17 00:00:00 2001 From: Tim Paine <3105306+timkpaine@users.noreply.github.com> Date: Thu, 23 May 2024 23:29:38 +0200 Subject: [PATCH 06/65] Unpin setproctitle, fixes #37727 (#40289) Unpin setproctitle, fixes #37727 Signed-off-by: Tim Paine <3105306+timkpaine@users.noreply.github.com> --- ci/env/install-dependencies.sh | 2 +- python/setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/env/install-dependencies.sh b/ci/env/install-dependencies.sh index 67e7d05bb826..784800bcc2c7 100755 --- a/ci/env/install-dependencies.sh +++ b/ci/env/install-dependencies.sh @@ -514,7 +514,7 @@ install_thirdparty_packages() { fi mkdir -p "${WORKSPACE_DIR}/python/ray/thirdparty_files" RAY_THIRDPARTY_FILES="$(realpath "${WORKSPACE_DIR}/python/ray/thirdparty_files")" - CC=gcc python -m pip install psutil==5.9.6 setproctitle==1.2.2 colorama==0.4.6 --target="${RAY_THIRDPARTY_FILES}" + CC=gcc python -m pip install psutil==5.9.6 "setproctitle>=1.2.2,<1.4" colorama==0.4.6 --target="${RAY_THIRDPARTY_FILES}" } install_dependencies() { diff --git a/python/setup.py b/python/setup.py index 8c1d573f5109..c402875e0f44 100644 --- a/python/setup.py +++ b/python/setup.py @@ -516,7 +516,7 @@ def build(build_python, build_java, build_cpp): # that certain flags will not be passed along such as --user or sudo. # TODO(rkn): Fix this. if not os.getenv("SKIP_THIRDPARTY_INSTALL"): - pip_packages = ["psutil", "setproctitle==1.2.2", "colorama"] + pip_packages = ["psutil", "setproctitle>=1.2.2,<1.4", "colorama"] subprocess.check_call( [ sys.executable, From 48e571157716dcae3584d73b95be6b2208aaf329 Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Thu, 23 May 2024 19:14:18 -0700 Subject: [PATCH 07/65] [Data] Clarify that `num_rows_per_file` isn't strict (#45529) The parameter name num_rows_per_file suggests that the resulting files should have exactly the specified number of rows, but that isn't the case. This PR clarifies that the value is more of a hint. Signed-off-by: Balaji Veeramani --- python/ray/data/dataset.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 960cba0a61e9..76fe82f01022 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -2795,6 +2795,8 @@ def write_parquet( arguments for each dataset block. num_rows_per_file: The target number of rows to write to each file. If ``None``, Ray Data writes a system-chosen number of rows to each file. + The specified value is a hint, not a strict limit. Ray Data might write + more or fewer rows to each file. ray_remote_args: Kwargs passed to :meth:`~ray.remote` in the write tasks. concurrency: The maximum number of Ray tasks to run concurrently. Set this to control number of tasks to run concurrently. This doesn't change the @@ -2903,6 +2905,8 @@ def write_json( arguments for each dataset block. num_rows_per_file: The target number of rows to write to each file. If ``None``, Ray Data writes a system-chosen number of rows to each file. + The specified value is a hint, not a strict limit. Ray Data might write + more or fewer rows to each file. ray_remote_args: kwargs passed to :meth:`~ray.remote` in the write tasks. concurrency: The maximum number of Ray tasks to run concurrently. Set this to control number of tasks to run concurrently. This doesn't change the @@ -3083,6 +3087,8 @@ def write_csv( write arguments for each dataset block. num_rows_per_file: The target number of rows to write to each file. If ``None``, Ray Data writes a system-chosen number of rows to each file. + The specified value is a hint, not a strict limit. Ray Data might write + more or fewer rows to each file. ray_remote_args: kwargs passed to :meth:`~ray.remote` in the write tasks. concurrency: The maximum number of Ray tasks to run concurrently. Set this to control number of tasks to run concurrently. This doesn't change the @@ -3183,6 +3189,8 @@ def write_tfrecords( look like. num_rows_per_file: The target number of rows to write to each file. If ``None``, Ray Data writes a system-chosen number of rows to each file. + The specified value is a hint, not a strict limit. Ray Data might write + more or fewer rows to each file. ray_remote_args: kwargs passed to :meth:`~ray.remote` in the write tasks. concurrency: The maximum number of Ray tasks to run concurrently. Set this to control number of tasks to run concurrently. This doesn't change the @@ -3267,6 +3275,8 @@ def write_webdataset( implementation to write each dataset block to a custom output path. num_rows_per_file: The target number of rows to write to each file. If ``None``, Ray Data writes a system-chosen number of rows to each file. + The specified value is a hint, not a strict limit. Ray Data might write + more or fewer rows to each file. ray_remote_args: Kwargs passed to ``ray.remote`` in the write tasks. concurrency: The maximum number of Ray tasks to run concurrently. Set this to control number of tasks to run concurrently. This doesn't change the @@ -3354,6 +3364,8 @@ def write_numpy( look like. num_rows_per_file: The target number of rows to write to each file. If ``None``, Ray Data writes a system-chosen number of rows to each file. + The specified value is a hint, not a strict limit. Ray Data might write + more or fewer rows to each file. ray_remote_args: kwargs passed to :meth:`~ray.remote` in the write tasks. concurrency: The maximum number of Ray tasks to run concurrently. Set this to control number of tasks to run concurrently. This doesn't change the From 4851df7019112160badf2b6a1e384ab3a6909681 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Thu, 23 May 2024 21:38:40 -0700 Subject: [PATCH 08/65] [core] log dedup should not dedup number only lines (#45485) Signed-off-by: hongchaodeng --- python/ray/_private/ray_logging.py | 9 +++-- python/ray/tests/test_log_dedup.py | 54 ++++++++++++++++++++++++++++++ 2 files changed, 61 insertions(+), 2 deletions(-) diff --git a/python/ray/_private/ray_logging.py b/python/ray/_private/ray_logging.py index e9ee12195340..5699251e31ce 100644 --- a/python/ray/_private/ray_logging.py +++ b/python/ray/_private/ray_logging.py @@ -284,11 +284,16 @@ def deduplicate(self, batch: LogBatch) -> List[LogBatch]: continue dedup_key = _canonicalise_log_line(line) + if dedup_key == "": + # Don't dedup messages that are empty after canonicalization. + # Because that's all the information users want to see. + output[0]["lines"].append(line) + continue + if dedup_key in self.recent: sources = self.recent[dedup_key].sources sources.add(source) - # We deduplicate the warnings/errorm essages from - # raylet by default. + # We deduplicate the warnings/error messages from raylet by default. if len(sources) > 1 or batch["pid"] == "raylet": state = self.recent[dedup_key] self.recent[dedup_key] = DedupState( diff --git a/python/ray/tests/test_log_dedup.py b/python/ray/tests/test_log_dedup.py index c1353d7c9458..833af274bea0 100644 --- a/python/ray/tests/test_log_dedup.py +++ b/python/ray/tests/test_log_dedup.py @@ -19,6 +19,60 @@ def test_nodedup_logs_single_process(): assert out1 == [batch1] +def test_nodedup_logs_buffer_only_lines(): + now = 142300000.0 + + def gettime(): + return now + + dedup = LogDeduplicator(5, None, None, _timesource=gettime) + batch1 = { + "ip": "node1", + "pid": 100, + # numbers are canonicalised, so this would lead to empty dedup_key + "lines": ["1"], + } + + # Immediately prints always. + out1 = dedup.deduplicate(batch1) + assert out1 == [batch1] + + now += 1.0 + + # Should print new lines even if it is number only again + batch2 = { + "ip": "node2", + "pid": 200, + "lines": ["2"], + } + out2 = dedup.deduplicate(batch2) + assert out2 == [ + { + "ip": "node2", + "pid": 200, + "lines": ["2"], + } + ] + + now += 3.0 + + # Should print new lines even if it is same number + batch3 = { + "ip": "node3", + "pid": 300, + "lines": ["2"], + } + # Should buffer duplicates. + out3 = dedup.deduplicate(batch3) + assert out3 == [ + { + "ip": "node3", + "pid": 300, + "lines": ["2"], + } + ] + + def test_dedup_logs_multiple_processes(): now = 142300000.0 From 7fb0ce1edc3b53a82257d57a715b042d07045ad9 Mon Sep 17 00:00:00 2001 From: simonsays1980 Date: Fri, 24 May 2024 10:32:49 +0200 Subject: [PATCH 09/65] [RLlib] Add support for multi-agent off-policy algorithms in the new API stack. (#45182) --- .bazelrc | 1 + rllib/BUILD | 19 ++ rllib/algorithms/dqn/dqn.py | 5 +- rllib/algorithms/dqn/dqn_rainbow_learner.py | 6 +- .../dqn/torch/dqn_rainbow_torch_learner.py | 2 +- rllib/algorithms/sac/sac.py | 1 + rllib/algorithms/sac/sac_learner.py | 1 - .../algorithms/sac/torch/sac_torch_learner.py | 9 +- .../common/agent_to_module_mapping.py | 3 - .../common/batch_individual_items.py | 5 +- rllib/env/multi_agent_env_runner.py | 9 +- rllib/env/multi_agent_episode.py | 24 ++ .../sac/multi_agent_pendulum_sac.py | 79 +++++ rllib/utils/metrics/__init__.py | 1 + .../multi_agent_episode_replay_buffer.py | 272 ++++++++---------- .../prioritized_episode_replay_buffer.py | 1 + .../test_multi_agent_episode_replay_buffer.py | 64 ++--- rllib/utils/replay_buffers/utils.py | 5 +- 18 files changed, 299 insertions(+), 208 deletions(-) create mode 100644 rllib/tuned_examples/sac/multi_agent_pendulum_sac.py diff --git a/.bazelrc b/.bazelrc index 1637085f31f5..307e9e38dcfd 100644 --- a/.bazelrc +++ b/.bazelrc @@ -167,6 +167,7 @@ test:ci --flaky_test_attempts=3 test:ci --nocache_test_results test:ci --spawn_strategy=local test:ci --test_output=errors +test:ci --experimental_ui_max_stdouterr_bytes=-1 test:ci --test_verbose_timeout_warnings test:ci-debug -c dbg test:ci-debug --copt="-g" diff --git a/rllib/BUILD b/rllib/BUILD index 9333e7b1adeb..5af156d68c77 100644 --- a/rllib/BUILD +++ b/rllib/BUILD @@ -463,6 +463,25 @@ py_test( args = ["--dir=tuned_examples/sac"] ) +# TODO (simon): These tests are not learning, yet. +# py_test( +# name = "learning_tests_multi_agent_pendulum_sac", +# main = "tuned_examples/sac/multi_agent_pendulum_sac.py", +# tags = ["team:rllib", "exclusive", "learning_tests", "torch_only", "learning_tests_pendulum", "learning_tests_continuous"], +# size = "large", +# srcs = ["tuned_examples/sac/multi_agent_pendulum_sac.py"], +# args = ["--enable-new-api-stack", "--num-agents=2"] +# ) + +# py_test( +# name = "learning_tests_multi_agent_pendulum_sac_multi_gpu", +# main = "tuned_examples/sac/multi_agent_pendulum_sac.py", +# tags = ["team:rllib", "exclusive", "learning_tests", "torch_only", "learning_tests_pendulum", "learning_tests_continuous", "multi_gpu"], +# size = "large", +# srcs = ["tuned_examples/sac/multi_agent_pendulum_sac.py"], +# args = ["--enable-new-api-stack", "--num-agents=2", "--num-gpus=2"] +# ) + # -------------------------------------------------------------------- # Algorithms (Compilation, Losses, simple functionality tests) # rllib/algorithms/ diff --git a/rllib/algorithms/dqn/dqn.py b/rllib/algorithms/dqn/dqn.py index 78502085ff51..2368b733376f 100644 --- a/rllib/algorithms/dqn/dqn.py +++ b/rllib/algorithms/dqn/dqn.py @@ -16,7 +16,6 @@ from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.algorithms.algorithm_config import AlgorithmConfig, NotProvided -from ray.rllib.algorithms.dqn.dqn_rainbow_learner import TD_ERROR_KEY from ray.rllib.algorithms.dqn.dqn_tf_policy import DQNTFPolicy from ray.rllib.algorithms.dqn.dqn_torch_policy import DQNTorchPolicy from ray.rllib.core.learner import Learner @@ -64,6 +63,7 @@ REPLAY_BUFFER_UPDATE_PRIOS_TIMER, SAMPLE_TIMER, SYNCH_WORKER_WEIGHTS_TIMER, + TD_ERROR_KEY, TIMERS, ) from ray.rllib.utils.deprecation import DEPRECATED_VALUE @@ -662,7 +662,7 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: num_items=self.config.train_batch_size, n_step=self.config.n_step, gamma=self.config.gamma, - beta=self.config.replay_buffer_config["beta"], + beta=self.config.replay_buffer_config.get("beta"), ) # Perform an update on the buffer-sampled train batch. @@ -700,6 +700,7 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: }, reduce="sum", ) + # TODO (sven): Uncomment this once agent steps are available in the # Learner stats. # self.metrics.log_dict(self.metrics.peek( diff --git a/rllib/algorithms/dqn/dqn_rainbow_learner.py b/rllib/algorithms/dqn/dqn_rainbow_learner.py index 1aba7f757008..abc73ada8413 100644 --- a/rllib/algorithms/dqn/dqn_rainbow_learner.py +++ b/rllib/algorithms/dqn/dqn_rainbow_learner.py @@ -13,7 +13,10 @@ override, OverrideToImplementCustomLogic_CallToSuperRecommended, ) -from ray.rllib.utils.metrics import LAST_TARGET_UPDATE_TS, NUM_TARGET_UPDATES +from ray.rllib.utils.metrics import ( + LAST_TARGET_UPDATE_TS, + NUM_TARGET_UPDATES, +) from ray.rllib.utils.typing import ModuleID if TYPE_CHECKING: @@ -32,7 +35,6 @@ QF_TARGET_NEXT_PROBS = "qf_target_next_probs" QF_PREDS = "qf_preds" QF_PROBS = "qf_probs" -TD_ERROR_KEY = "td_error" TD_ERROR_MEAN_KEY = "td_error_mean" diff --git a/rllib/algorithms/dqn/torch/dqn_rainbow_torch_learner.py b/rllib/algorithms/dqn/torch/dqn_rainbow_torch_learner.py index 0a887908b114..7ec354ba61f9 100644 --- a/rllib/algorithms/dqn/torch/dqn_rainbow_torch_learner.py +++ b/rllib/algorithms/dqn/torch/dqn_rainbow_torch_learner.py @@ -14,13 +14,13 @@ QF_TARGET_NEXT_PROBS, QF_PREDS, QF_PROBS, - TD_ERROR_KEY, TD_ERROR_MEAN_KEY, ) from ray.rllib.core.columns import Columns from ray.rllib.core.learner.torch.torch_learner import TorchLearner from ray.rllib.utils.annotations import override from ray.rllib.utils.framework import try_import_torch +from ray.rllib.utils.metrics import TD_ERROR_KEY from ray.rllib.utils.nested_dict import NestedDict from ray.rllib.utils.typing import ModuleID, TensorType diff --git a/rllib/algorithms/sac/sac.py b/rllib/algorithms/sac/sac.py index e249e2dae7a3..8ab0c89cf3ad 100644 --- a/rllib/algorithms/sac/sac.py +++ b/rllib/algorithms/sac/sac.py @@ -352,6 +352,7 @@ def validate(self) -> None: ] not in [ "EpisodeReplayBuffer", "PrioritizedEpisodeReplayBuffer", + "MultiAgentEpisodeReplayBuffer", ]: raise ValueError( "When using the new `EnvRunner API` the replay buffer must be of type " diff --git a/rllib/algorithms/sac/sac_learner.py b/rllib/algorithms/sac/sac_learner.py index 94a2e907de96..cdbc44ee749b 100644 --- a/rllib/algorithms/sac/sac_learner.py +++ b/rllib/algorithms/sac/sac_learner.py @@ -20,7 +20,6 @@ QF_TWIN_LOSS_KEY = "qf_twin_loss" QF_TWIN_PREDS = "qf_twin_preds" TD_ERROR_MEAN_KEY = "td_error_mean" -TD_ERROR_KEY = "td_error" class SACLearner(DQNRainbowLearner): diff --git a/rllib/algorithms/sac/torch/sac_torch_learner.py b/rllib/algorithms/sac/torch/sac_torch_learner.py index 0565e950136b..229b8cc4549f 100644 --- a/rllib/algorithms/sac/torch/sac_torch_learner.py +++ b/rllib/algorithms/sac/torch/sac_torch_learner.py @@ -15,17 +15,15 @@ QF_TWIN_LOSS_KEY, QF_TWIN_PREDS, TD_ERROR_MEAN_KEY, - TD_ERROR_KEY, SACLearner, ) -from ray.rllib.core import DEFAULT_MODULE_ID from ray.rllib.core.columns import Columns from ray.rllib.core.learner.learner import ( POLICY_LOSS_KEY, ) from ray.rllib.utils.annotations import override from ray.rllib.utils.framework import try_import_torch -from ray.rllib.utils.metrics import ALL_MODULES +from ray.rllib.utils.metrics import ALL_MODULES, TD_ERROR_KEY from ray.rllib.utils.nested_dict import NestedDict from ray.rllib.utils.typing import ModuleID, ParamDict, TensorType @@ -221,8 +219,6 @@ def compute_loss_for_module( # Note further, we use here the Huber loss instead of the mean squared error # as it improves training performance. critic_loss = torch.mean( - # TODO (simon): Introduce priority weights when episode buffer is ready. - # batch[PRIO_WEIGHTS] * batch["weights"] * torch.nn.HuberLoss(reduction="none", delta=1.0)( q_selected, q_selected_target @@ -303,6 +299,7 @@ def compute_loss_for_module( def compute_gradients( self, loss_per_module: Dict[str, TensorType], **kwargs ) -> ParamDict: + # Set all grads to `None`. for optim in self._optimizer_parameters: optim.zero_grad(set_to_none=True) @@ -317,7 +314,7 @@ def compute_gradients( for component in ( ["qf", "policy", "alpha"] + ["qf_twin"] if config.twin_q else [] ): - self.metrics.peek(DEFAULT_MODULE_ID, component + "_loss").backward( + self.metrics.peek(module_id, component + "_loss").backward( retain_graph=True ) grads.update( diff --git a/rllib/connectors/common/agent_to_module_mapping.py b/rllib/connectors/common/agent_to_module_mapping.py index c304fa60a174..b54a20bb050f 100644 --- a/rllib/connectors/common/agent_to_module_mapping.py +++ b/rllib/connectors/common/agent_to_module_mapping.py @@ -133,9 +133,6 @@ def __call__( shared_data: Optional[dict] = None, **kwargs, ) -> Any: - # This Connector should only be used in a multi-agent setting. - assert not episodes or isinstance(episodes[0], MultiAgentEpisode) - # Current agent to module mapping function. # agent_to_module_mapping_fn = shared_data.get("agent_to_module_mapping_fn") # Store in shared data, which module IDs map to which episode/agent, such diff --git a/rllib/connectors/common/batch_individual_items.py b/rllib/connectors/common/batch_individual_items.py index e4a4f2ac8d86..9b5460b4cb49 100644 --- a/rllib/connectors/common/batch_individual_items.py +++ b/rllib/connectors/common/batch_individual_items.py @@ -33,7 +33,10 @@ def __call__( # to a batch structure of: # [module_id] -> [col0] -> [list of items] if is_marl_module and column in rl_module: - assert is_multi_agent + # assert is_multi_agent + # TODO (simon, sven): Check, if we need for other cases this check. + # If MA Off-Policy and independent sampling we need to overcome + # this check. module_data = column_data for col, col_data in module_data.copy().items(): if isinstance(col_data, list) and col != Columns.INFOS: diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index 3e76e8b0025b..043083aaa11a 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -3,7 +3,6 @@ from collections import defaultdict from functools import partial -import numpy as np from typing import DefaultDict, Dict, List, Optional from ray.rllib.algorithms.algorithm_config import AlgorithmConfig @@ -623,9 +622,11 @@ def get_metrics(self) -> ResultDict: module_episode_returns, ) - # If no episodes at all, log NaN stats. - if len(self._done_episodes_for_metrics) == 0: - self._log_episode_metrics(np.nan, np.nan, np.nan) + # TODO (simon): This results in hundreds of warnings in the logs + # b/c reducing over NaNs is not supported. + # # If no episodes at all, log NaN stats. + # if len(self._done_episodes_for_metrics) == 0: + # self._log_episode_metrics(np.nan, np.nan, np.nan) # Log num episodes counter for this iteration. self.metrics.log_value( diff --git a/rllib/env/multi_agent_episode.py b/rllib/env/multi_agent_episode.py index 216aaf5f0f31..a59cec2d7a63 100644 --- a/rllib/env/multi_agent_episode.py +++ b/rllib/env/multi_agent_episode.py @@ -58,6 +58,30 @@ class MultiAgentEpisode: up to here, b/c there is nothing to learn from these "premature" rewards. """ + __slots__ = ( + "id_", + "agent_to_module_mapping_fn", + "_agent_to_module_mapping", + "observation_space", + "action_space", + "env_t_started", + "env_t", + "agent_t_started", + "env_t_to_agent_t", + "_hanging_actions_end", + "_hanging_extra_model_outputs_end", + "_hanging_rewards_end", + "_hanging_actions_begin", + "_hanging_extra_model_outputs_begin", + "_hanging_rewards_begin", + "is_terminated", + "is_truncated", + "agent_episodes", + "_temporary_timestep_data", + "_start_time", + "_last_step_time", + ) + SKIP_ENV_TS_TAG = "S" def __init__( diff --git a/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py b/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py new file mode 100644 index 000000000000..a70ca0b9d62b --- /dev/null +++ b/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py @@ -0,0 +1,79 @@ +from ray.rllib.algorithms.sac import SACConfig +from ray.rllib.examples.envs.classes.multi_agent import MultiAgentPendulum +from ray.rllib.utils.metrics import ( + ENV_RUNNER_RESULTS, + EPISODE_RETURN_MEAN, + NUM_ENV_STEPS_SAMPLED_LIFETIME, +) +from ray.tune.registry import register_env + +from ray.rllib.utils.test_utils import add_rllib_example_script_args + +parser = add_rllib_example_script_args() +# Use `parser` to add your own custom command line options to this script +# and (if needed) use their values to set up `config` below. +args = parser.parse_args() + +register_env( + "multi_agent_pendulum", + lambda _: MultiAgentPendulum({"num_agents": args.num_agents or 2}), +) + +config = ( + SACConfig() + .environment(env="multi_agent_pendulum") + .rl_module( + model_config_dict={ + "fcnet_hiddens": [256, 256], + "fcnet_activation": "relu", + "post_fcnet_hiddens": [], + "post_fcnet_activation": None, + "post_fcnet_weights_initializer": "orthogonal_", + "post_fcnet_weights_initializer_config": {"gain": 0.01}, + } + ) + .api_stack( + enable_rl_module_and_learner=True, + enable_env_runner_and_connector_v2=True, + ) + .env_runners( + rollout_fragment_length=1, + num_env_runners=2, + num_envs_per_env_runner=1, + ) + .training( + initial_alpha=1.001, + lr=3e-4, + target_entropy="auto", + n_step=1, + tau=0.005, + train_batch_size_per_learner=256, + target_network_update_freq=1, + replay_buffer_config={ + "type": "MultiAgentEpisodeReplayBuffer", + "capacity": 100000, + }, + num_steps_sampled_before_learning_starts=256, + ) + .reporting( + metrics_num_episodes_for_smoothing=5, + min_sample_timesteps_per_iteration=1000, + ) +) + +if args.num_agents: + config.multi_agent( + policy_mapping_fn=lambda aid, *arg, **kw: f"p{aid}", + policies={f"p{i}" for i in range(args.num_agents)}, + ) + +stop = { + NUM_ENV_STEPS_SAMPLED_LIFETIME: 500000, + # `episode_return_mean` is the sum of all agents/policies' returns. + f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": -400.0 * (args.num_agents or 2), +} + +if __name__ == "__main__": + from ray.rllib.utils.test_utils import run_rllib_example_script_experiment + + run_rllib_example_script_experiment(config, args, stop=stop) diff --git a/rllib/utils/metrics/__init__.py b/rllib/utils/metrics/__init__.py index 164ec20cf405..39e087da9434 100644 --- a/rllib/utils/metrics/__init__.py +++ b/rllib/utils/metrics/__init__.py @@ -91,3 +91,4 @@ # Learner. LEARNER_STATS_KEY = "learner_stats" ALL_MODULES = "__all_modules__" +TD_ERROR_KEY = "td_error" diff --git a/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py b/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py index c32415ba01a9..9b1af8e86cff 100644 --- a/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py +++ b/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py @@ -3,10 +3,11 @@ from gymnasium.core import ActType, ObsType import numpy as np import scipy -from typing import Any, Dict, List, Optional, Tuple, Union +from typing import Any, Dict, List, Optional, Set, Tuple, Union from ray.rllib.core.columns import Columns from ray.rllib.env.multi_agent_episode import MultiAgentEpisode +from ray.rllib.env.single_agent_episode import SingleAgentEpisode from ray.rllib.utils.replay_buffers.episode_replay_buffer import EpisodeReplayBuffer from ray.rllib.utils import force_list from ray.rllib.utils.annotations import override, DeveloperAPI @@ -130,40 +131,42 @@ def add( """ episodes: List["MultiAgentEpisode"] = force_list(episodes) - new_episode_ids: List[str] = [] - for eps in episodes: - new_episode_ids.append(eps.id_) - self._num_timesteps += eps.env_steps() - self._num_timesteps_added += eps.env_steps() + new_episode_ids: List[str] = {eps.id_ for eps in episodes} + total_env_timesteps = sum([eps.env_steps() for eps in episodes]) + self._num_timesteps += total_env_timesteps + self._num_timesteps_added += total_env_timesteps # Evict old episodes. - eps_evicted: List["MultiAgentEpisode"] = [] - eps_evicted_ids: List[Union[str, int]] = [] - eps_evicted_idxs: List[int] = [] + eps_evicted_ids: Set[Union[str, int]] = set() + eps_evicted_idxs: Set[int] = set() while ( self._num_timesteps > self.capacity and self._num_remaining_episodes(new_episode_ids, eps_evicted_ids) != 1 ): # Evict episode. evicted_episode = self.episodes.popleft() - eps_evicted.append(evicted_episode) - eps_evicted_ids.append(evicted_episode.id_) - eps_evicted_idxs.append(self.episode_id_to_index.pop(evicted_episode.id_)) + eps_evicted_ids.add(evicted_episode.id_) + eps_evicted_idxs.add(self.episode_id_to_index.pop(evicted_episode.id_)) # If this episode has a new chunk in the new episodes added, # we subtract it again. # TODO (sven, simon): Should we just treat such an episode chunk # as a new episode? if evicted_episode.id_ in new_episode_ids: - new_eps_to_evict = episodes[new_episode_ids.index(evicted_episode.id_)] + idx = next( + i + for i, eps in enumerate(episodes) + if eps.id_ == evicted_episode.id_ + ) + new_eps_to_evict = episodes.pop(idx) self._num_timesteps -= new_eps_to_evict.env_steps() self._num_timesteps_added -= new_eps_to_evict.env_steps() - episodes.remove(new_eps_to_evict) # Remove the timesteps of the evicted episode from the counter. self._num_timesteps -= evicted_episode.env_steps() self._num_agent_timesteps -= evicted_episode.agent_steps() self._num_episodes_evicted += 1 # Remove the module timesteps of the evicted episode from the counters. self._evict_module_episodes(evicted_episode) + del evicted_episode # Add agent and module steps. for eps in episodes: @@ -174,41 +177,38 @@ def add( # Remove corresponding indices, if episodes were evicted. if eps_evicted_idxs: - new_indices = [] - # Each index 2-tuple is of the form (ma_episode_idx, timestep) and + # If the episode is not exvicted, we keep the index. + # Note, ach index 2-tuple is of the form (ma_episode_idx, timestep) and # refers to a certain environment timestep in a certain multi-agent # episode. - for idx_tuple in self._indices: - # If episode index is not from an evicted episode, keep it. - if idx_tuple[0] not in eps_evicted_idxs: - new_indices.append(idx_tuple) - # Assign the new list of indices. - self._indices = new_indices + self._indices = [ + idx_tuple + for idx_tuple in self._indices + if idx_tuple[0] not in eps_evicted_idxs + ] # Also remove corresponding module indices. for module_id, module_indices in self._module_to_indices.items(): - new_module_indices = [] # Each index 3-tuple is of the form # (ma_episode_idx, agent_id, timestep) and refers to a certain # agent timestep in a certain multi-agent episode. - for idx_triplet in module_indices: - if idx_triplet[0] not in eps_evicted_idxs: - new_module_indices.append(idx_triplet) - self._module_to_indices[module_id] = new_module_indices + self._module_to_indices[module_id] = [ + idx_triplet + for idx_triplet in module_indices + if idx_triplet[0] not in eps_evicted_idxs + ] for eps in episodes: eps = copy.deepcopy(eps) # If the episode is part of an already existing episode, concatenate. if eps.id_ in self.episode_id_to_index: eps_idx = self.episode_id_to_index[eps.id_] - existing_eps = self.episodes[eps_idx] + existing_eps = self.episodes[eps_idx - self._num_episodes_evicted] existing_len = len(existing_eps) self._indices.extend( [ ( eps_idx, - # Note, we add 1 b/c the first timestep is - # never sampled. - existing_len + i + 1, + existing_len + i, ) for i in range(len(eps)) ] @@ -223,7 +223,7 @@ def add( self.episodes.append(eps) eps_idx = len(self.episodes) - 1 + self._num_episodes_evicted self.episode_id_to_index[eps.id_] = eps_idx - self._indices.extend([(eps_idx, i + 1) for i in range(len(eps))]) + self._indices.extend([(eps_idx, i) for i in range(len(eps))]) # Add new module indices. self._add_new_module_indices(eps, eps_idx, False) @@ -240,6 +240,7 @@ def sample( include_extra_model_outputs: bool = False, replay_mode: str = "independent", modules_to_sample: Optional[List[ModuleID]] = None, + **kwargs, ) -> SampleBatchType: """Samples a batch of multi-agent transitions. @@ -458,46 +459,25 @@ def _sample_independent( gamma: float, include_infos: bool, include_extra_model_outputs: bool, - modules_to_sample: Optional[List[ModuleID]], + modules_to_sample: Optional[Set[ModuleID]], ) -> SampleBatchType: """Samples a batch of independent multi-agent transitions.""" + + actual_n_step = n_step or 1 # Sample the n-step if necessary. - if isinstance(n_step, tuple): - # Use random n-step sampling. - random_n_step = True - else: - actual_n_step = n_step or 1 - random_n_step = False + random_n_step = isinstance(n_step, tuple) - ret = {} + sampled_episodes = [] # TODO (simon): Ensure that the module has data and if not, skip it. # TODO (sven): Should we then error out or skip? I think the Learner # should handle this case when a module has no train data. - for module_id in modules_to_sample or self._module_to_indices.keys(): - # Rows to return. - observations: List[List[ObsType]] = [[] for _ in range(batch_size_B)] - next_observations: List[List[ObsType]] = [[] for _ in range(batch_size_B)] - actions: List[List[ActType]] = [[] for _ in range(batch_size_B)] - rewards: List[List[float]] = [[] for _ in range(batch_size_B)] - is_terminated: List[bool] = [False for _ in range(batch_size_B)] - is_truncated: List[bool] = [False for _ in range(batch_size_B)] - weights: List[float] = [[1.0] for _ in range(batch_size_B)] - n_steps: List[List[int]] = [[] for _ in range(batch_size_B)] - # If `info` should be included, construct also a container for them. - if include_infos: - infos: List[List[Dict[str, Any]]] = [[] for _ in range(batch_size_B)] - # If `extra_model_outputs` should be included, construct a container for - # them. - if include_extra_model_outputs: - extra_model_outputs: List[List[Dict[str, Any]]] = [ - [] for _ in range(batch_size_B) - ] + modules_to_sample = modules_to_sample or set(self._module_to_indices.keys()) + for module_id in modules_to_sample: + module_indices = self._module_to_indices[module_id] B = 0 while B < batch_size_B: # Now sample from the single-agent timesteps. - index_tuple = self._module_to_indices[module_id][ - self.rng.integers(len(self._module_to_indices[module_id])) - ] + index_tuple = module_indices[self.rng.integers(len(module_indices))] # This will be an agent timestep (not env timestep). # TODO (simon, sven): Maybe deprecate sa_episode_idx (_) in the index @@ -507,109 +487,95 @@ def _sample_independent( index_tuple[1], index_tuple[2], ) - # If we cannnot make the n-step, we resample. - if sa_episode_ts - n_step < 0: - continue - # If we use random n-step sampling, draw the n-step for this item. - if random_n_step: - actual_n_step = int(self.rng.integers(n_step[0], n_step[1])) - # If we are at the end of an episode, continue. - # Note, priority sampling got us `o_(t+n)` and we need for the loss - # calculation in addition `o_t`. - # TODO (simon): Maybe introduce a variable `num_retries` until the - # while loop should break when not enough samples have been collected - # to make n-step possible. - if sa_episode_ts - actual_n_step < 0: - continue - else: - n_steps[B] = actual_n_step + # Get the multi-agent episode. ma_episode = self.episodes[ma_episode_idx] # Retrieve the single-agent episode for filtering. sa_episode = ma_episode.agent_episodes[agent_id] - # Ensure that each row contains a tuple of the form: - # (o_t, a_t, sum(r_(t:t+n_step)), o_(t+n_step)) - # TODO (simon): Implement version for sequence sampling when using RNNs. - sa_eps_observation = sa_episode.get_observations( - slice(sa_episode_ts - actual_n_step, sa_episode_ts + 1) - ) - # Note, the reward that is collected by transitioning from `o_t` to - # `o_(t+1)` is stored in the next transition in `SingleAgentEpisode`. - sa_eps_rewards = sa_episode.get_rewards( - slice(sa_episode_ts - actual_n_step, sa_episode_ts) - ) - observations[B] = sa_eps_observation[0] - next_observations[B] = sa_eps_observation[-1] + + # If we use random n-step sampling, draw the n-step for this item. + if random_n_step: + actual_n_step = int(self.rng.integers(n_step[0], n_step[1])) + # If we cannnot make the n-step, we resample. + if sa_episode_ts + actual_n_step > len(sa_episode): + continue # Note, this will be the reward after executing action - # `a_(episode_ts-n_step+1)`. For `n_step>1` this will be the sum of + # `a_(episode_ts)`. For `n_step>1` this will be the sum of # all rewards that were collected over the last n steps. - rewards[B] = scipy.signal.lfilter( - [1], [1, -gamma], sa_eps_rewards[::-1], axis=0 + sa_raw_rewards = sa_episode.get_rewards( + slice(sa_episode_ts, sa_episode_ts + actual_n_step) + ) + sa_rewards = scipy.signal.lfilter( + [1], [1, -gamma], sa_raw_rewards[::-1], axis=0 )[-1] - # Note, `SingleAgentEpisode` stores the action that followed - # `o_t` with `o_(t+1)`, therefore, we need the next one. - # TODO (simon): This gets the wrong action as long as the getters are - # not fixed. - actions[B] = sa_episode.get_actions(sa_episode_ts - actual_n_step) - if include_infos: - # If infos are included we include the ones from the last timestep - # as usually the info contains additional values about the last - # state. - infos[B] = sa_episode.get_infos(sa_episode_ts) - if include_extra_model_outputs: - # If `extra_model_outputs` are included we include the ones from the - # first timestep as usually the `extra_model_outputs` contain - # additional values from the forward pass that produced the action - # at the first timestep. - # Note, we extract them into single row dictionaries similar to the - # infos, in a connector we can then extract these into single batch - # rows. - extra_model_outputs[B] = { - k: sa_episode.get_extra_model_outputs( - k, sa_episode_ts - actual_n_step - ) - for k in sa_episode.extra_model_outputs.keys() - } - # If the sampled time step is the episode's last time step check, if - # the episode is terminated or truncated. - if sa_episode_ts == sa_episode.t: - is_terminated[B] = sa_episode.is_terminated - is_truncated[B] = sa_episode.is_truncated + + sampled_sa_episode = SingleAgentEpisode( + id_=sa_episode.id_, + # Provide the IDs for the learner connector. + agent_id=sa_episode.agent_id, + module_id=sa_episode.module_id, + multi_agent_episode_id=ma_episode.id_, + # Ensure that each episode contains a tuple of the form: + # (o_t, a_t, sum(r_(t:t+n_step)), o_(t+n_step)) + # Two observations (t and t+n). + observations=[ + sa_episode.get_observations(sa_episode_ts), + sa_episode.get_observations(sa_episode_ts + actual_n_step), + ], + observation_space=sa_episode.observation_space, + infos=( + [ + sa_episode.get_infos(sa_episode_ts), + sa_episode.get_infos(sa_episode_ts + actual_n_step), + ] + if include_infos + else None + ), + actions=[sa_episode.get_actions(sa_episode_ts)], + action_space=sa_episode.action_space, + rewards=[sa_rewards], + # If the sampled single-agent episode is the single-agent episode's + # last time step, check, if the single-agent episode is terminated + # or truncated. + terminated=( + sa_episode_ts + actual_n_step >= len(sa_episode) + and sa_episode.is_terminated + ), + truncated=( + sa_episode_ts + actual_n_step >= len(sa_episode) + and sa_episode.is_truncated + ), + extra_model_outputs={ + "weights": [1.0], + "n_step": [actual_n_step], + **( + { + k: [ + sa_episode.get_extra_model_outputs(k, sa_episode_ts) + ] + for k in sa_episode.extra_model_outputs.keys() + } + if include_extra_model_outputs + else {} + ), + }, + # TODO (sven): Support lookback buffers. + len_lookback_buffer=0, + t_started=sa_episode_ts, + ) + # Append single-agent episode to the list of sampled episodes. + sampled_episodes.append(sampled_sa_episode) # Increase counter. B += 1 # Increase the per module timesteps counter. - self.sampled_timesteps_per_module[module_id] += batch_size_B - ret[module_id] = { - # Note, observation and action spaces could be complex. `batch` - # takes care of these. - Columns.OBS: batch(observations), - Columns.ACTIONS: batch(actions), - Columns.REWARDS: np.array(rewards), - Columns.NEXT_OBS: batch(next_observations), - Columns.TERMINATEDS: np.array(is_terminated), - Columns.TRUNCATEDS: np.array(is_truncated), - "weights": np.array(weights), - "n_step": np.array(n_steps), - } - # Include infos if necessary. - if include_infos: - ret[module_id].update( - { - Columns.INFOS: infos, - } - ) - # Include extra model outputs, if necessary. - if include_extra_model_outputs: - ret[module_id].update( - # These could be complex, too. - batch(extra_model_outputs) - ) + self.sampled_timesteps_per_module[module_id] += B + # Increase the counter for environment timesteps. self.sampled_timesteps += batch_size_B # Return multi-agent dictionary. - return ret + return sampled_episodes def _sample_synchonized( self, @@ -899,7 +865,9 @@ def _add_new_module_indices( sa_episode_in_buffer = False if sa_episode_in_buffer: existing_eps_len = len( - self.episodes[episode_idx].agent_episodes[agent_id] + self.episodes[ + episode_idx - self._num_episodes_evicted + ].agent_episodes[agent_id] ) else: existing_eps_len = 0 @@ -910,7 +878,7 @@ def _add_new_module_indices( # Keep the MAE index for sampling episode_idx, agent_id, - existing_eps_len + i + 1, + existing_eps_len + i, ) for i in range(len(module_eps)) ] diff --git a/rllib/utils/replay_buffers/prioritized_episode_replay_buffer.py b/rllib/utils/replay_buffers/prioritized_episode_replay_buffer.py index 38690232351e..b2a656d75f06 100644 --- a/rllib/utils/replay_buffers/prioritized_episode_replay_buffer.py +++ b/rllib/utils/replay_buffers/prioritized_episode_replay_buffer.py @@ -217,6 +217,7 @@ def add( # TODO (sven, simon): Should we just treat such an episode chunk # as a new episode? if eps_evicted_ids[-1] in new_episode_ids: + # TODO (simon): Apply the same logic as in the MA-case. len_to_subtract = len( episodes[new_episode_ids.index(eps_evicted_idxs[-1])] ) diff --git a/rllib/utils/replay_buffers/tests/test_multi_agent_episode_replay_buffer.py b/rllib/utils/replay_buffers/tests/test_multi_agent_episode_replay_buffer.py index 14a3860c5e6c..3844b5a485c6 100644 --- a/rllib/utils/replay_buffers/tests/test_multi_agent_episode_replay_buffer.py +++ b/rllib/utils/replay_buffers/tests/test_multi_agent_episode_replay_buffer.py @@ -5,6 +5,7 @@ from ray.rllib.utils.replay_buffers.multi_agent_episode_replay_buffer import ( MultiAgentEpisodeReplayBuffer, ) +from ray.rllib.utils.test_utils import check class TestMultiAgentEpisodeReplayBuffer(unittest.TestCase): @@ -150,59 +151,58 @@ def test_buffer_independent_sample_logic(self): for i in range(1000): sample = buffer.sample(batch_size_B=16, n_step=1) self.assertTrue(buffer.get_sampled_timesteps() == 16 * (i + 1)) - self.assertTrue("module_1" in sample) - self.assertTrue("module_2" in sample) - for module_id in sample: - self.assertTrue(buffer.get_sampled_timesteps(module_id) == 16 * (i + 1)) + module_ids = {eps.module_id for eps in sample} + self.assertTrue("module_1" in module_ids) + self.assertTrue("module_2" in module_ids) + for eps in sample: + # For both modules, we should have 16 x (i + 1) timesteps sampled. + # Note, this must be the same here as the number of timesteps sampled + # altogether, b/c we sample both modules. + self.assertTrue( + buffer.get_sampled_timesteps("module_1") == 16 * (i + 1) + ) + self.assertTrue( + buffer.get_sampled_timesteps("module_2") == 16 * (i + 1) + ) ( obs, - actions, - rewards, + action, + reward, next_obs, is_terminated, is_truncated, - weights, - n_steps, + weight, + n_step, ) = ( - sample[module_id]["obs"], - sample[module_id]["actions"], - sample[module_id]["rewards"], - sample[module_id]["new_obs"], - sample[module_id]["terminateds"], - sample[module_id]["truncateds"], - sample[module_id]["weights"], - sample[module_id]["n_step"], + eps.get_observations(0), + eps.get_actions(-1), + eps.get_rewards(-1), + eps.get_observations(-1), + eps.is_terminated, + eps.is_truncated, + eps.get_extra_model_outputs("weights", -1), + eps.get_extra_model_outputs("n_step", -1), ) # Make sure terminated and truncated are never both True. - assert not np.any(np.logical_and(is_truncated, is_terminated)) - - # All fields have same shape. - assert ( - obs.shape[:2] - == rewards.shape - == actions.shape - == next_obs.shape - == is_truncated.shape - == is_terminated.shape - ) + assert not (is_truncated and is_terminated) # Note, floating point numbers cannot be compared directly. tolerance = 1e-8 # Assert that actions correspond to the observations. - self.assertTrue(np.all(actions - obs < tolerance)) + check(obs, action, atol=tolerance) # Assert that next observations are correctly one step after # observations. - self.assertTrue(np.all(next_obs - obs - 1 < tolerance)) + check(next_obs, obs + 1, atol=tolerance) # Assert that the reward comes from the next observation. - self.assertTrue(np.all(rewards * 10 - next_obs < tolerance)) + check(reward * 10, next_obs, atol=tolerance) # Furthermore, assert that the importance sampling weights are # one for `beta=0.0`. - self.assertTrue(np.all(weights - 1.0 < tolerance)) + check(weight, 1.0, atol=tolerance) # Assert that all n-steps are 1.0 as passed into `sample`. - self.assertTrue(np.all(n_steps - 1.0 < tolerance)) + check(n_step, 1.0, atol=tolerance) def test_buffer_synchronized_sample_logic(self): """Samples synchronized from the multi-agent buffer.""" diff --git a/rllib/utils/replay_buffers/utils.py b/rllib/utils/replay_buffers/utils.py index 3b1bb6b6924f..c825c64fc3ff 100644 --- a/rllib/utils/replay_buffers/utils.py +++ b/rllib/utils/replay_buffers/utils.py @@ -8,7 +8,7 @@ from ray.rllib.utils.annotations import OldAPIStack from ray.rllib.utils.deprecation import DEPRECATED_VALUE from ray.rllib.utils.from_config import from_config -from ray.rllib.utils.metrics import ALL_MODULES +from ray.rllib.utils.metrics import ALL_MODULES, TD_ERROR_KEY from ray.rllib.utils.metrics.learner_info import LEARNER_STATS_KEY from ray.rllib.utils.replay_buffers import ( EpisodeReplayBuffer, @@ -30,9 +30,6 @@ logger = logging.getLogger(__name__) -# TODO (simon): Move all regular keys to the metric constants file. -TD_ERROR_KEY = "td_error" - @DeveloperAPI def update_priorities_in_episode_replay_buffer( From 5cb7c09af5dc5e6a225526b020ee4b010d5b2748 Mon Sep 17 00:00:00 2001 From: simonsays1980 Date: Fri, 24 May 2024 16:13:23 +0200 Subject: [PATCH 10/65] [RLlib] Cleanup examples folder: Add example restoring 1 of n agents from a checkpoint. (#45462) --- rllib/BUILD | 18 +- .../restore_1_of_n_agents_from_checkpoint.py | 257 +++++++++--------- 2 files changed, 144 insertions(+), 131 deletions(-) diff --git a/rllib/BUILD b/rllib/BUILD index 5af156d68c77..097ad26ca80c 100644 --- a/rllib/BUILD +++ b/rllib/BUILD @@ -2157,15 +2157,6 @@ py_test( srcs = ["examples/checkpoints/onnx_torch.py"], ) -#@OldAPIStack -py_test( - name = "examples/checkpoints/restore_1_of_n_agents_from_checkpoint", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py"], - args = ["--pre-training-iters=1", "--stop-iters=1", "--num-cpus=4"] -) - # subdirectory: connectors/ # .................................... # Framestacking examples only run in smoke-test mode (a few iters only). @@ -2751,6 +2742,15 @@ py_test( # args = ["--enable-new-api-stack", "--num-agents=2", "--as-test", "--framework=torch", "--stop-reward=-100.0", "--num-cpus=4"], # ) +py_test( + name = "examples/checkpoints/restore_1_of_n_agents_from_checkpoint", + main = "examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py", + tags = ["team:rllib", "exclusive", "examples", "examples_use_all_core", "no_main"], + size = "large", + srcs = ["examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py"], + args = ["--enable-new-api-stack", "--num-agents=2", "--framework=torch", "--checkpoint-freq=20", "--checkpoint-at-end", "--num-cpus=4", "--algo=PPO"] +) + py_test( name = "examples/multi_agent/rock_paper_scissors_heuristic_vs_learned", main = "examples/multi_agent/rock_paper_scissors_heuristic_vs_learned.py", diff --git a/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py b/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py index 9c4dc3805613..4338791c71fa 100644 --- a/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py +++ b/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py @@ -1,140 +1,153 @@ -# TODO (sven): Move this example script into the new API stack. - -"""Simple example of how to restore only one of n agents from a trained -multi-agent Algorithm using Ray tune. - -Control the number of agents and policies via --num-agents and --num-policies. +"""An example script showing how to load module weights for 1 of n agents +from checkpoint. + +This example: + - Runs a multi-agent `Pendulum-v1` experiment with >= 2 policies. + - Saves a checkpoint of the `MultiAgentRLModule` used every `--checkpoint-freq` + iterations. + - Stops the experiments after the agents reach a combined return of `-800`. + - Picks the best checkpoint by combined return and restores policy 0 from it. + - Runs a second experiment with the restored `RLModule` for policy 0 and + a fresh `RLModule` for the other policies. + - Stops the second experiment after the agents reach a combined return of `-800`. + +How to run this script +---------------------- +`python [script file name].py --enable-new-api-stack --num-agents=2 +--checkpoint-freq=20 --checkpoint-at-end` + +Control the number of agents and policies (RLModules) via --num-agents and +--num-policies. + +Control the number of checkpoints by setting `--checkpoint-freq` to a value > 0. +Note that the checkpoint frequency is per iteration and this example needs at +least a single checkpoint to load the RLModule weights for policy 0. +If `--checkpoint-at-end` is set, a checkpoint will be saved at the end of the +experiment. + +For debugging, use the following additional command line options +`--no-tune --num-env-runners=0` +which should allow you to set breakpoints anywhere in the RLlib code and +have the execution stop there for inspection and debugging. + +For logging to your WandB account, use: +`--wandb-key=[your WandB API key] --wandb-project=[some project name] +--wandb-run-name=[optional: WandB run name (within the defined project)]` + +Results to expect +----------------- +You should expect a reward of -400.0 eventually being achieved by a simple +single PPO policy (no tuning, just using RLlib's default settings). In the +second run of the experiment, the MARL module weights for policy 0 are +restored from the checkpoint of the first run. The reward for a single agent +should be -400.0 again, but the training time should be shorter (around 30 +iterations instead of 190). """ -import argparse -import gymnasium as gym import os -import random - -import ray -from ray import air, tune from ray.air.constants import TRAINING_ITERATION -from ray.rllib.algorithms.algorithm import Algorithm -from ray.rllib.algorithms.callbacks import DefaultCallbacks -from ray.rllib.algorithms.ppo import PPOConfig -from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole -from ray.rllib.policy.policy import Policy -from ray.rllib.utils.framework import try_import_tf -from ray.rllib.utils.metrics import ( - ENV_RUNNER_RESULTS, - EPISODE_RETURN_MEAN, - NUM_ENV_STEPS_SAMPLED_LIFETIME, +from ray.rllib.core.rl_module.marl_module import MultiAgentRLModuleSpec +from ray.rllib.examples.envs.classes.multi_agent import MultiAgentPendulum +from ray.rllib.utils.metrics import ENV_RUNNER_RESULTS, NUM_ENV_STEPS_SAMPLED_LIFETIME +from ray.rllib.utils.test_utils import ( + add_rllib_example_script_args, + run_rllib_example_script_experiment, ) -from ray.rllib.utils.test_utils import check_learning_achieved +from ray.tune.registry import get_trainable_cls, register_env -tf1, tf, tfv = try_import_tf() - -parser = argparse.ArgumentParser() - -parser.add_argument("--num-agents", type=int, default=4) -parser.add_argument("--num-policies", type=int, default=2) -parser.add_argument("--pre-training-iters", type=int, default=5) -parser.add_argument("--num-cpus", type=int, default=0) -parser.add_argument( - "--framework", - choices=["tf", "tf2", "torch"], - default="torch", - help="The DL framework specifier.", -) -parser.add_argument( - "--as-test", - action="store_true", - help="Whether this script should be run as a test: --stop-reward must " - "be achieved within --stop-timesteps AND --stop-iters.", -) -parser.add_argument( - "--stop-iters", type=int, default=200, help="Number of iterations to train." -) -parser.add_argument( - "--stop-timesteps", type=int, default=100000, help="Number of timesteps to train." -) -parser.add_argument( - "--stop-reward", type=float, default=150.0, help="Reward at which we stop training." +parser = add_rllib_example_script_args( + default_iters=200, + default_timesteps=100000, + default_reward=-400.0, ) +# TODO (sven): This arg is currently ignored (hard-set to 2). +parser.add_argument("--num-policies", type=int, default=2) + if __name__ == "__main__": args = parser.parse_args() - ray.init(num_cpus=args.num_cpus or None) - - # Get obs- and action Spaces. - single_env = gym.make("CartPole-v1") - obs_space = single_env.observation_space - act_space = single_env.action_space - - # Setup PPO with an ensemble of `num_policies` different policies. - policies = { - f"policy_{i}": (None, obs_space, act_space, None) - for i in range(args.num_policies) - } - policy_ids = list(policies.keys()) - - def policy_mapping_fn(agent_id, episode, worker, **kwargs): - pol_id = random.choice(policy_ids) - return pol_id - - config = ( - PPOConfig() - .environment(MultiAgentCartPole, env_config={"num_agents": args.num_agents}) - .framework(args.framework) - .training(num_sgd_iter=10) - .multi_agent(policies=policies, policy_mapping_fn=policy_mapping_fn) - # Use GPUs iff `RLLIB_NUM_GPUS` env var set to > 0. - .resources(num_gpus=int(os.environ.get("RLLIB_NUM_GPUS", "0"))) + # Register our environment with tune. + if args.num_agents > 1: + register_env( + "env", + lambda _: MultiAgentPendulum(config={"num_agents": args.num_agents}), + ) + else: + raise ValueError( + f"`num_agents` must be > 1, but is {args.num_agents}." + "Read the script docstring for more information." + ) + + assert args.checkpoint_freq > 0, ( + "This example requires at least one checkpoint to load the RLModule " + "weights for policy 0." ) - # Do some training and store the checkpoint. - results = tune.Tuner( - "PPO", - param_space=config.to_dict(), - run_config=air.RunConfig( - stop={TRAINING_ITERATION: args.pre_training_iters}, - verbose=1, - checkpoint_config=air.CheckpointConfig( - checkpoint_frequency=1, checkpoint_at_end=True - ), - ), - ).fit() - print("Pre-training done.") - - best_checkpoint = results.get_best_result().checkpoint - print(f".. best checkpoint was: {best_checkpoint}") - - policy_0_checkpoint = os.path.join( - best_checkpoint.to_directory(), "policies/policy_0" + base_config = ( + get_trainable_cls(args.algo) + .get_default_config() + .environment("env") + .training( + train_batch_size_per_learner=512, + mini_batch_size_per_learner=64, + lambda_=0.1, + gamma=0.95, + lr=0.0003, + vf_clip_param=10.0, + ) + .rl_module( + model_config_dict={"fcnet_activation": "relu"}, + ) ) - restored_policy_0 = Policy.from_checkpoint(policy_0_checkpoint) - restored_policy_0_weights = restored_policy_0.get_weights() - print("Starting new tune.Tuner().fit()") - # Start our actual experiment. - stop = { - f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": args.stop_reward, - NUM_ENV_STEPS_SAMPLED_LIFETIME: args.stop_timesteps, - TRAINING_ITERATION: args.stop_iters, + # Add a simple multi-agent setup. + if args.num_agents > 0: + base_config.multi_agent( + policies={f"p{i}" for i in range(args.num_agents)}, + policy_mapping_fn=lambda aid, *a, **kw: f"p{aid}", + ) + + # Augment the base config with further settings and train the agents. + results = run_rllib_example_script_experiment(base_config, args) + + # Create an env instance to get the observation and action spaces. + env = MultiAgentPendulum(config={"num_agents": args.num_agents}) + # Get the default module spec from the algorithm config. + module_spec = base_config.get_default_rl_module_spec() + module_spec.model_config_dict = base_config.model_config | { + "fcnet_activation": "relu", } - - class RestoreWeightsCallback(DefaultCallbacks): - def on_algorithm_init(self, *, algorithm: "Algorithm", **kwargs) -> None: - algorithm.set_weights({"policy_0": restored_policy_0_weights}) - - # Make sure, the non-1st policies are not updated anymore. - config.policies_to_train = [pid for pid in policy_ids if pid != "policy_0"] - config.callbacks(RestoreWeightsCallback) - - results = tune.run( - "PPO", - stop=stop, - config=config.to_dict(), - verbose=1, + module_spec.observation_space = env.envs[0].observation_space + module_spec.action_space = env.envs[0].action_space + # Create the module for each policy, but policy 0. + module_specs = {} + for i in range(1, args.num_agents or 1): + module_specs[f"p{i}"] = module_spec + + # Now swap in the RLModule weights for policy 0. + chkpt_path = results.get_best_result().checkpoint.path + p_0_module_state_path = os.path.join(chkpt_path, "learner", "module_state", "p0") + module_spec.load_state_path = p_0_module_state_path + module_specs["p0"] = module_spec + + # Create the MARL module. + marl_module_spec = MultiAgentRLModuleSpec(module_specs=module_specs) + # Define the MARL module in the base config. + base_config.rl_module(rl_module_spec=marl_module_spec) + # We need to re-register the environment when starting a new run. + register_env( + "env", + lambda _: MultiAgentPendulum(config={"num_agents": args.num_agents}), ) + # Define stopping criteria. + stop = { + # TODO (simon): Change to -800 once the metrics are fixed. Currently + # the combined return is not correctly computed. + f"{ENV_RUNNER_RESULTS}/episode_return_mean": -400, + f"{NUM_ENV_STEPS_SAMPLED_LIFETIME}": 20000, + TRAINING_ITERATION: 30, + } - if args.as_test: - check_learning_achieved(results, args.stop_reward) - - ray.shutdown() + # Run the experiment again with the restored MARL module. + run_rllib_example_script_experiment(base_config, args, stop=stop) From fde52033983ee529ac6ef3d1265086bd45bd23c9 Mon Sep 17 00:00:00 2001 From: simonsays1980 Date: Fri, 24 May 2024 18:00:47 +0200 Subject: [PATCH 11/65] [RLlib] Example Cleanups new API Stack - Autoregressive Action Module. (#45525) --- .../envs/classes/correlated_actions_env.py | 13 +- .../rl_modules/autoregressive_actions_rlm.py | 107 +++++++ .../classes/autoregressive_actions_rlm.py | 295 ++++++++++++++++++ rllib/models/torch/torch_distributions.py | 9 +- 4 files changed, 416 insertions(+), 8 deletions(-) create mode 100644 rllib/examples/rl_modules/autoregressive_actions_rlm.py create mode 100644 rllib/examples/rl_modules/classes/autoregressive_actions_rlm.py diff --git a/rllib/examples/envs/classes/correlated_actions_env.py b/rllib/examples/envs/classes/correlated_actions_env.py index 055a4d75d558..a3db0d7556d9 100644 --- a/rllib/examples/envs/classes/correlated_actions_env.py +++ b/rllib/examples/envs/classes/correlated_actions_env.py @@ -1,5 +1,6 @@ import gymnasium as gym -from gymnasium.spaces import Discrete, Tuple +from gymnasium.spaces import Box, Discrete, Tuple +import numpy as np import random @@ -13,19 +14,19 @@ class CorrelatedActionsEnv(gym.Env): to a1. I.e., +10 at most per step. One way to effectively learn this is through correlated action - distributions, e.g., in examples/autoregressive_action_dist.py + distributions, e.g., in examples/rl_modules/autoregressive_action_rlm.py There are 20 steps. Hence, the best score would be ~200 reward. """ - def __init__(self, _): - self.observation_space = Discrete(2) + def __init__(self, _=None): + self.observation_space = Box(0, 1, shape=(1,), dtype=np.float32) self.action_space = Tuple([Discrete(2), Discrete(2)]) self.last_observation = None def reset(self, *, seed=None, options=None): self.t = 0 - self.last_observation = random.choice([0, 1]) + self.last_observation = np.array([random.choice([0, 1])], dtype=np.float32) return self.last_observation, {} def step(self, action): @@ -39,5 +40,5 @@ def step(self, action): if a1 == a2: reward += 5 done = truncated = self.t > 20 - self.last_observation = random.choice([0, 1]) + self.last_observation = np.array([random.choice([0, 1])], dtype=np.float32) return self.last_observation, reward, done, truncated, {} diff --git a/rllib/examples/rl_modules/autoregressive_actions_rlm.py b/rllib/examples/rl_modules/autoregressive_actions_rlm.py new file mode 100644 index 000000000000..7920ca622738 --- /dev/null +++ b/rllib/examples/rl_modules/autoregressive_actions_rlm.py @@ -0,0 +1,107 @@ +"""An example script showing how to define and load an `RLModule` with +a dependent action space. + +This examples: + - Defines an `RLModule` with autoregressive actions. + - It does so by implementing a prior distribution for the first couple + of actions and then using these actions in a posterior distribution. + - Furthermore, it uses in the `RLModule` our simple base `Catalog` class + to build the distributions. + - Uses this `RLModule` in a PPO training run on a simple environment + that rewards synchronized actions. + - Stops the training after 100k steps or when the mean episode return + exceeds 150 in evaluation, i.e. if the agent has learned to + synchronize its actions. + +How to run this script +---------------------- +`python [script file name].py --enable-new-api-stack --num-env-runners 2` + +Control the number of `EnvRunner`s with the `--num-env-runners` flag. This +will increase the sampling speed. + +For debugging, use the following additional command line options +`--no-tune --num-env-runners=0` +which should allow you to set breakpoints anywhere in the RLlib code and +have the execution stop there for inspection and debugging. + +For logging to your WandB account, use: +`--wandb-key=[your WandB API key] --wandb-project=[some project name] +--wandb-run-name=[optional: WandB run name (within the defined project)]` + +Results to expect +----------------- +You should expect a reward of around 155-160 after ~36,000 timesteps sampled +(trained) being achieved by a simple PPO policy (no tuning, just using RLlib's +default settings). For details take also a closer look into the +`CorrelatedActionsEnv` environment. Rewards are such that to receive a return +over 100, the agent must learn to synchronize its actions. +""" + + +from ray.rllib.algorithms.ppo import PPOConfig +from ray.rllib.core.models.catalog import Catalog +from ray.rllib.core.rl_module.rl_module import SingleAgentRLModuleSpec +from ray.rllib.examples.envs.classes.correlated_actions_env import CorrelatedActionsEnv +from ray.rllib.examples.rl_modules.classes.autoregressive_actions_rlm import ( + AutoregressiveActionTorchRLM, +) +from ray.rllib.utils.metrics import ( + ENV_RUNNER_RESULTS, + EPISODE_RETURN_MEAN, + EVALUATION_RESULTS, + NUM_ENV_STEPS_SAMPLED_LIFETIME, +) +from ray.rllib.utils.test_utils import ( + add_rllib_example_script_args, + run_rllib_example_script_experiment, +) +from ray.tune import register_env + + +register_env("correlated_actions_env", lambda _: CorrelatedActionsEnv(_)) + +parser = add_rllib_example_script_args( + default_iters=200, + default_timesteps=100000, + default_reward=150.0, +) + +if __name__ == "__main__": + args = parser.parse_args() + + if args.algo != "PPO": + raise ValueError("This example only supports PPO. Please use --algo=PPO.") + + base_config = ( + PPOConfig() + .environment(env="correlated_actions_env") + .rl_module( + model_config_dict={ + "post_fcnet_hiddens": [64, 64], + "post_fcnet_activation": "relu", + }, + # We need to explicitly specify here RLModule to use and + # the catalog needed to build it. + rl_module_spec=SingleAgentRLModuleSpec( + module_class=AutoregressiveActionTorchRLM, + catalog_class=Catalog, + ), + ) + .evaluation( + evaluation_num_env_runners=1, + evaluation_interval=1, + # Run evaluation parallel to training to speed up the example. + evaluation_parallel_to_training=True, + ) + ) + + # Let's stop the training after 100k steps or when the mean episode return + # exceeds 150 in evaluation. + stop = { + f"{NUM_ENV_STEPS_SAMPLED_LIFETIME}": 100000, + f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 150.0, + } + + # Run the example (with Tune). + run_rllib_example_script_experiment(base_config, args, stop=stop) diff --git a/rllib/examples/rl_modules/classes/autoregressive_actions_rlm.py b/rllib/examples/rl_modules/classes/autoregressive_actions_rlm.py new file mode 100644 index 000000000000..fa773944976b --- /dev/null +++ b/rllib/examples/rl_modules/classes/autoregressive_actions_rlm.py @@ -0,0 +1,295 @@ +from abc import abstractmethod +from typing import Any, Dict, Type + +from ray.rllib.core import Columns +from ray.rllib.core.models.base import ENCODER_OUT +from ray.rllib.core.models.configs import MLPHeadConfig +from ray.rllib.core.models.specs.specs_dict import SpecDict +from ray.rllib.core.rl_module.rl_module import RLModule +from ray.rllib.core.rl_module.torch.torch_rl_module import TorchRLModule +from ray.rllib.models.distributions import Distribution +from ray.rllib.utils.annotations import ( + override, + OverrideToImplementCustomLogic_CallToSuperRecommended, +) +from ray.rllib.utils.framework import try_import_torch +from ray.rllib.utils.torch_utils import convert_to_torch_tensor +from ray.rllib.utils.typing import TensorType + +torch, nn = try_import_torch() + + +# TODO (simon): Improvements: `inference-only` mode. +class AutoregressiveActionRLM(RLModule): + """An RLModule that implements an autoregressive action distribution. + + This RLModule implements an autoregressive action distribution, where the + action is sampled in two steps. First, the prior action is sampled from a + prior distribution. Then, the posterior action is sampled from a posterior + distribution that depends on the prior action and the input data. The prior + and posterior distributions are implemented as MLPs. + + The following components are implemented: + - ENCODER: An encoder that processes the observations from the environment. + - PI: A Policy head that outputs the actions, the log probabilities of the + actions, and the input to the action distribution. This head is composed + of two sub-heads: + - A prior head that outputs the logits for the prior action distribution. + - A posterior head that outputs the logits for the posterior action + distribution. + - A value function head that outputs the value function. + + Note, this RLModule is implemented for the `PPO` algorithm only. It is not + guaranteed to work with other algorithms. + """ + + @override(RLModule) + @OverrideToImplementCustomLogic_CallToSuperRecommended + def setup(self): + super().setup() + + # Build the encoder. + self.encoder = self.config.get_catalog().build_encoder(framework=self.framework) + + # Build the prior and posterior heads. + # Note, the action space is a Tuple space. + self.action_dist_cls = self.config.get_catalog().get_action_dist_cls( + self.framework + ) + # Note further, we neet to know the required input dimensions for + # the partial distributions. + self.required_output_dims = self.action_dist_cls.required_input_dim( + space=self.config.action_space, + as_list=True, + ) + action_dims = self.config.action_space[0].shape or (1,) + latent_dims = self.config.get_catalog().latent_dims + prior_config = MLPHeadConfig( + # Use the hidden dimension from the encoder output. + input_dims=latent_dims, + # Use configurations from the `model_config_dict`. + hidden_layer_dims=self.config.model_config_dict["post_fcnet_hiddens"], + hidden_layer_activation=self.config.model_config_dict[ + "post_fcnet_activation" + ], + output_layer_dim=self.required_output_dims[0], + output_layer_activation="linear", + ) + # Build the posterior head. + posterior_config = MLPHeadConfig( + input_dims=(latent_dims[0] + action_dims[0],), + hidden_layer_dims=self.config.model_config_dict["post_fcnet_hiddens"], + hidden_layer_activation=self.config.model_config_dict[ + "post_fcnet_activation" + ], + output_layer_dim=self.required_output_dims[1], + output_layer_activation="linear", + ) + + self.prior = prior_config.build(framework=self.framework) + self.posterior = posterior_config.build(framework=self.framework) + + # Build the value function head. + vf_config = MLPHeadConfig( + input_dims=latent_dims, + hidden_layer_dims=self.config.model_config_dict["post_fcnet_hiddens"], + hidden_layer_activation=self.config.model_config_dict[ + "post_fcnet_activation" + ], + output_layer_dim=1, + output_layer_activation="linear", + ) + self.vf = vf_config.build(framework=self.framework) + + @override(RLModule) + def get_train_action_dist_cls(self) -> Type[Distribution]: + return self.action_dist_cls + + @override(RLModule) + def get_exploration_action_dist_cls(self) -> Type[Distribution]: + return self.action_dist_cls + + @override(RLModule) + def get_inference_action_dist_cls(self) -> Type[Distribution]: + return self.action_dist_cls + + @override(RLModule) + def output_specs_inference(self) -> SpecDict: + return [Columns.ACTIONS] + + @override(RLModule) + def output_specs_exploration(self) -> SpecDict: + return [Columns.ACTION_DIST_INPUTS, Columns.ACTIONS, Columns.ACTION_LOGP] + + @override(RLModule) + def output_specs_train(self) -> SpecDict: + return [ + Columns.ACTION_DIST_INPUTS, + Columns.ACTIONS, + Columns.ACTION_LOGP, + Columns.VF_PREDS, + ] + + @abstractmethod + def pi(self, batch: Dict[str, TensorType]) -> Dict[str, TensorType]: + """Computes the policy outputs given a batch of data. + + Args: + batch: The input batch to pass through the policy head. + + Returns: + A dict mapping Column names to batches of policy outputs. + """ + + @abstractmethod + def _compute_values(self, batch) -> Any: + """Computes values using the vf-specific network(s) and given a batch of data. + + Args: + batch: The input batch to pass through this RLModule (value function + encoder and vf-head). + + Returns: + A dict mapping ModuleIDs to batches of value function outputs (already + squeezed on the last dimension (which should have shape (1,) b/c of the + single value output node). However, for complex multi-agent settings with + shareed value networks, the output might look differently (e.g. a single + return batch without the ModuleID-based mapping). + """ + + +class AutoregressiveActionTorchRLM(TorchRLModule, AutoregressiveActionRLM): + @override(AutoregressiveActionRLM) + def pi( + self, batch: Dict[str, TensorType], inference: bool = False + ) -> Dict[str, TensorType]: + pi_outs = {} + + # Prior forward pass. + prior_out = self.prior(batch) + prior_logits = torch.cat( + [ + prior_out, + # We add zeros for the posterior logits, which we do not have at + # this point of time. + torch.zeros(size=(prior_out.shape[0], self.required_output_dims[1])), + ], + dim=-1, + ) + # Get the prior action distribution to sample the prior action. + if inference: + # If in inference mode, we need to set the distribution to be deterministic. + prior_action_dist = self.action_dist_cls.from_logits( + prior_logits + ).to_deterministic() + # If in inference mode, we can sample in a simple way. + prior_action = prior_action_dist._flat_child_distributions[0].sample() + else: + prior_action_dist = self.action_dist_cls.from_logits(prior_logits) + # Note, `TorchMultiDistribution.from_logits` does set the `logits`, but not + # the `probs` attribute. We need to set the `probs` attribute to be able to + # sample from the distribution in a differentiable way. + prior_action_dist._flat_child_distributions[0].probs = torch.softmax( + prior_out, dim=-1 + ) + prior_action_dist._flat_child_distributions[0].logits = None + # Otherwise, we need to be able to backpropagate through the prior action + # that's why we sample from the distribution using the `rsample` method. + # TODO (simon, sven): Check, if we need to return the one-hot sampled action + # instead of the real-valued one. + prior_action = torch.argmax( + prior_action_dist._flat_child_distributions[0].rsample(), + dim=-1, + ) + + # Posterior forward pass. + posterior_batch = torch.cat([batch, prior_action.view(-1, 1)], dim=-1) + posterior_out = self.posterior(posterior_batch) + # Concatenate the prior and posterior logits to get the final logits. + posterior_logits = torch.cat([prior_out, posterior_out], dim=-1) + if inference: + posterior_action_dist = self.action_dist_cls.from_logits( + posterior_logits + ).to_deterministic() + # Sample the posterior action. + posterior_action = posterior_action_dist._flat_child_distributions[ + 1 + ].sample() + + else: + # Get the posterior action distribution to sample the posterior action. + posterior_action_dist = self.action_dist_cls.from_logits(posterior_logits) + # Sample the posterior action. + posterior_action = posterior_action_dist._flat_child_distributions[ + 1 + ].sample() + + # We need the log probabilities of the sampled actions for the loss + # calculation. + prior_action_logp = prior_action_dist._flat_child_distributions[0].logp( + prior_action + ) + posterior_action_logp = posterior_action_dist._flat_child_distributions[ + 1 + ].logp(posterior_action) + pi_outs[Columns.ACTION_LOGP] = prior_action_logp + posterior_action_logp + # We also need the input to the action distribution to calculate the + # KL-divergence. + pi_outs[Columns.ACTION_DIST_INPUTS] = posterior_logits + + # Concatenate the prior and posterior actions and log probabilities. + pi_outs[Columns.ACTIONS] = (prior_action, posterior_action) + + return pi_outs + + @override(TorchRLModule) + def _forward_inference(self, batch: Dict[str, TensorType]) -> Dict[str, TensorType]: + + # Encoder forward pass. + encoder_out = self.encoder(batch) + + # Policy head forward pass. + return self.pi(encoder_out[ENCODER_OUT], inference=True) + + @override(TorchRLModule) + def _forward_exploration( + self, batch: Dict[str, TensorType], **kwargs + ) -> Dict[str, TensorType]: + # Encoder forward pass. + encoder_out = self.encoder(batch) + + # Policy head forward pass. + return self.pi(encoder_out[ENCODER_OUT], inference=False) + + @override(TorchRLModule) + def _forward_train(self, batch: Dict[str, TensorType]) -> Dict[str, TensorType]: + + outs = {} + + # Encoder forward pass. + encoder_out = self.encoder(batch) + + # Policy head forward pass. + outs.update(self.pi(encoder_out[ENCODER_OUT])) + + # Value function head forward pass. + vf_out = self.vf(encoder_out[ENCODER_OUT]) + outs[Columns.VF_PREDS] = vf_out.squeeze(-1) + + return outs + + @override(AutoregressiveActionRLM) + def _compute_values(self, batch, device=None): + infos = batch.pop(Columns.INFOS, None) + batch = convert_to_torch_tensor(batch, device=device) + if infos is not None: + batch[Columns.INFOS] = infos + + # Encoder forward pass. + encoder_outs = self.encoder(batch)[ENCODER_OUT] + + # Value head forward pass. + vf_out = self.vf(encoder_outs) + + # Squeeze out last dimension (single node value head). + return vf_out.squeeze(-1) diff --git a/rllib/models/torch/torch_distributions.py b/rllib/models/torch/torch_distributions.py index d9f42c0ec473..d5f3bccc220c 100644 --- a/rllib/models/torch/torch_distributions.py +++ b/rllib/models/torch/torch_distributions.py @@ -600,8 +600,13 @@ def sample(self): @staticmethod @override(Distribution) - def required_input_dim(space: gym.Space, input_lens: List[int], **kwargs) -> int: - return sum(input_lens) + def required_input_dim( + space: gym.Space, input_lens: List[int], as_list: bool = False, **kwargs + ) -> int: + if as_list: + return input_lens + else: + return sum(input_lens) @classmethod @override(Distribution) From 7709dec241e2ad8eed487bc6afc29e5be87ae387 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Fri, 24 May 2024 13:17:44 -0700 Subject: [PATCH 12/65] [ci][microcheck/11] move get_changed_tests to Test class (#45499) This is part of the series to move some of the function to compute microcheck test coverage to the Test class. This will allow me to reuse the logic more easier in other places. Should be a pure refactoring. Test: - CI Signed-off-by: can --- ci/ray_ci/test_tester.py | 30 +----------- ci/ray_ci/tester.py | 61 +---------------------- release/ray_release/test.py | 67 +++++++++++++++++++++++++- release/ray_release/tests/test_test.py | 24 +++++++++ 4 files changed, 93 insertions(+), 89 deletions(-) diff --git a/ci/ray_ci/test_tester.py b/ci/ray_ci/test_tester.py index 46cdf0e178df..0b8819d8b6de 100644 --- a/ci/ray_ci/test_tester.py +++ b/ci/ray_ci/test_tester.py @@ -17,8 +17,6 @@ _get_flaky_test_targets, _get_tag_matcher, _get_new_tests, - _get_changed_files, - _get_changed_tests, _get_human_specified_tests, ) from ray_release.test import Test, TestState @@ -128,7 +126,7 @@ def test_get_test_targets() -> None: "ray_release.test.Test.gen_high_impact_tests", return_value={"step": test_objects}, ), mock.patch( - "ci.ray_ci.tester._get_changed_tests", + "ray_release.test.Test.get_changed_tests", return_value=set(), ), mock.patch( "ci.ray_ci.tester._get_new_tests", @@ -256,7 +254,7 @@ def test_get_high_impact_test_targets() -> None: "ci.ray_ci.tester._get_new_tests", return_value=test["new_tests"], ), mock.patch( - "ci.ray_ci.tester._get_changed_tests", + "ray_release.test.Test.get_changed_tests", return_value=test["changed_tests"], ), mock.patch( "ci.ray_ci.tester._get_human_specified_tests", @@ -285,30 +283,6 @@ def test_get_new_tests(mock_gen_from_s3, mock_run_script_with_output) -> None: ) == {"//new_test"} -@mock.patch.dict( - os.environ, - {"BUILDKITE_PULL_REQUEST_BASE_BRANCH": "base", "BUILDKITE_COMMIT": "commit"}, -) -@mock.patch("subprocess.check_call") -@mock.patch("subprocess.check_output") -def test_get_changed_files(mock_check_output, mock_check_call) -> None: - mock_check_output.return_value = b"file1\nfile2\n" - assert _get_changed_files() == {"file1", "file2"} - - -@mock.patch("ci.ray_ci.tester._get_test_targets_per_file") -@mock.patch("ci.ray_ci.tester._get_changed_files") -def test_get_changed_tests( - mock_get_changed_files, mock_get_test_targets_per_file -) -> None: - mock_get_changed_files.return_value = {"test_src", "build_src"} - mock_get_test_targets_per_file.side_effect = ( - lambda x: {"//t1", "//t2"} if x == "test_src" else {} - ) - - assert _get_changed_tests() == {"//t1", "//t2"} - - @mock.patch.dict( os.environ, {"BUILDKITE_PULL_REQUEST_BASE_BRANCH": "base", "BUILDKITE_COMMIT": "commit"}, diff --git a/ci/ray_ci/tester.py b/ci/ray_ci/tester.py index e94117c3f8f8..14f92a4f0918 100644 --- a/ci/ray_ci/tester.py +++ b/ci/ray_ci/tester.py @@ -417,7 +417,7 @@ def _get_high_impact_test_targets( if test.get_oncall() == team } new_tests = _get_new_tests(os_prefix, container) - changed_tests = _get_changed_tests() + changed_tests = Test.get_changed_tests(bazel_workspace_dir) human_specified_tests = _get_human_specified_tests() return ( @@ -464,65 +464,6 @@ def _get_new_tests(prefix: str, container: TesterContainer) -> Set[str]: return local_test_targets.difference(db_test_targets) -def _get_changed_tests() -> Set[str]: - """ - Get all changed tests in the current PR - """ - changed_files = _get_changed_files() - logger.info(f"Changed files: {changed_files}") - return set( - itertools.chain.from_iterable( - [_get_test_targets_per_file(file) for file in _get_changed_files()] - ) - ) - - -def _get_test_targets_per_file(file: str) -> Set[str]: - """ - Get the test target from a file path - """ - try: - package = ( - subprocess.check_output(["bazel", "query", file], cwd=bazel_workspace_dir) - .decode() - .strip() - ) - if not package: - return set() - targets = subprocess.check_output( - ["bazel", "query", f"tests(attr('srcs', {package}, //...))"], - cwd=bazel_workspace_dir, - ) - targets = { - target.strip() - for target in targets.decode().splitlines() - if target is not None - } - logger.info(f"Found test targets for file {file}: {targets}") - - return targets - except subprocess.CalledProcessError: - logger.info(f"File {file} is not a test target") - return set() - - -def _get_changed_files() -> Set[str]: - """ - Get all changed files in the current PR - """ - base = os.environ.get("BUILDKITE_PULL_REQUEST_BASE_BRANCH") - head = os.environ.get("BUILDKITE_COMMIT") - if not base or not head: - # if not in a PR, return an empty set - return set() - - changes = subprocess.check_output( - ["git", "diff", "--name-only", f"origin/{base}...{head}"], - cwd=bazel_workspace_dir, - ) - return {file.strip() for file in changes.decode().splitlines() if file is not None} - - def _get_flaky_test_targets( team: str, operating_system: str, yaml_dir: Optional[str] = None ) -> List[str]: diff --git a/release/ray_release/test.py b/release/ray_release/test.py index 868adfac0cb5..b5d5bfef3b9d 100644 --- a/release/ray_release/test.py +++ b/release/ray_release/test.py @@ -3,10 +3,11 @@ import enum import os import platform +import subprocess import json import time from itertools import chain -from typing import Awaitable, Optional, List, Dict +from typing import Awaitable, Optional, List, Dict, Set from dataclasses import dataclass import aioboto3 @@ -211,6 +212,70 @@ def gen_high_impact_tests(cls, prefix: str) -> Dict[str, List]: return step_id_to_tests + @classmethod + def get_changed_tests(cls, bazel_workspace_dir: str) -> Set[str]: + """ + Get all changed tests in the current PR + """ + return set( + chain.from_iterable( + [ + cls._get_test_targets_per_file(file, bazel_workspace_dir) + for file in cls._get_changed_files(bazel_workspace_dir) + ] + ) + ) + + @classmethod + def _get_changed_files(cls, bazel_workspace_dir: str) -> Set[str]: + """ + Get all changed files in the current PR + """ + base = os.environ.get("BUILDKITE_PULL_REQUEST_BASE_BRANCH") + head = os.environ.get("BUILDKITE_COMMIT") + if not base or not head: + # if not in a PR, return an empty set + return set() + + changes = subprocess.check_output( + ["git", "diff", "--name-only", f"origin/{base}...{head}"], + cwd=bazel_workspace_dir, + ) + return { + file.strip() for file in changes.decode().splitlines() if file is not None + } + + @classmethod + def _get_test_targets_per_file( + cls, file: str, bazel_workspace_dir: str + ) -> Set[str]: + """ + Get the test target from a file path + """ + try: + package = ( + subprocess.check_output( + ["bazel", "query", file], cwd=bazel_workspace_dir + ) + .decode() + .strip() + ) + if not package: + return set() + targets = subprocess.check_output( + ["bazel", "query", f"tests(attr('srcs', {package}, //...))"], + cwd=bazel_workspace_dir, + ) + targets = { + target.strip() + for target in targets.decode().splitlines() + if target is not None + } + + return targets + except subprocess.CalledProcessError: + return set() + def is_jailed_with_open_issue(self, ray_github: Repository) -> bool: """ Returns whether this test is jailed with open issue. diff --git a/release/ray_release/tests/test_test.py b/release/ray_release/tests/test_test.py index b9a756b5b3ed..cb9265aeabcd 100644 --- a/release/ray_release/tests/test_test.py +++ b/release/ray_release/tests/test_test.py @@ -381,5 +381,29 @@ def test_get_test_target(): assert Test({"name": input}).get_target() == output +@mock.patch.dict( + os.environ, + {"BUILDKITE_PULL_REQUEST_BASE_BRANCH": "base", "BUILDKITE_COMMIT": "commit"}, +) +@mock.patch("subprocess.check_call") +@mock.patch("subprocess.check_output") +def test_get_changed_files(mock_check_output, mock_check_call) -> None: + mock_check_output.return_value = b"file1\nfile2\n" + assert Test._get_changed_files("") == {"file1", "file2"} + + +@mock.patch("ray_release.test.Test._get_test_targets_per_file") +@mock.patch("ray_release.test.Test._get_changed_files") +def test_get_changed_tests( + mock_get_changed_files, mock_get_test_targets_per_file +) -> None: + mock_get_changed_files.return_value = {"test_src", "build_src"} + mock_get_test_targets_per_file.side_effect = ( + lambda x, _: {"//t1", "//t2"} if x == "test_src" else {} + ) + + assert Test.get_changed_tests("") == {"//t1", "//t2"} + + if __name__ == "__main__": sys.exit(pytest.main(["-v", __file__])) From 10b255e5fb607d34761448099ebd03af24badf4e Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Fri, 24 May 2024 13:49:20 -0700 Subject: [PATCH 13/65] Revert "Unpin setproctitle, fixes #37727" (#45539) Reverts ray-project/ray#40289. Breaking https://github.com/orgs/anyscale/projects/76/views/1?pane=issue&itemId=64249695. Close #45502 Test: - https://buildkite.com/ray-project/postmerge-macos/builds/942 --- ci/env/install-dependencies.sh | 2 +- python/setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/env/install-dependencies.sh b/ci/env/install-dependencies.sh index 784800bcc2c7..67e7d05bb826 100755 --- a/ci/env/install-dependencies.sh +++ b/ci/env/install-dependencies.sh @@ -514,7 +514,7 @@ install_thirdparty_packages() { fi mkdir -p "${WORKSPACE_DIR}/python/ray/thirdparty_files" RAY_THIRDPARTY_FILES="$(realpath "${WORKSPACE_DIR}/python/ray/thirdparty_files")" - CC=gcc python -m pip install psutil==5.9.6 "setproctitle>=1.2.2,<1.4" colorama==0.4.6 --target="${RAY_THIRDPARTY_FILES}" + CC=gcc python -m pip install psutil==5.9.6 setproctitle==1.2.2 colorama==0.4.6 --target="${RAY_THIRDPARTY_FILES}" } install_dependencies() { diff --git a/python/setup.py b/python/setup.py index c402875e0f44..8c1d573f5109 100644 --- a/python/setup.py +++ b/python/setup.py @@ -516,7 +516,7 @@ def build(build_python, build_java, build_cpp): # that certain flags will not be passed along such as --user or sudo. # TODO(rkn): Fix this. if not os.getenv("SKIP_THIRDPARTY_INSTALL"): - pip_packages = ["psutil", "setproctitle>=1.2.2,<1.4", "colorama"] + pip_packages = ["psutil", "setproctitle==1.2.2", "colorama"] subprocess.check_call( [ sys.executable, From 21534bbb63ca8d58052337797d8b2f4320807374 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Fri, 24 May 2024 14:57:17 -0700 Subject: [PATCH 14/65] [doc] add a doc page about ci testing workflow on a PR (#45446) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit As title, add a doc page about the new ci testing workflow (microcheck, go, premerge, etc.). I'll follow up with another PR to add more images to the doc. Test: - https://anyscale-ray--45446.com.readthedocs.build/en/45446/ray-contribute/ci.html Screenshot 2024-05-20 at 11 02 29 AM --------- Signed-off-by: can Signed-off-by: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Co-authored-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/ray-contribute/ci.rst | 52 +++++++++++++++++++ .../ray-contribute/getting-involved.rst | 1 + 2 files changed, 53 insertions(+) create mode 100644 doc/source/ray-contribute/ci.rst diff --git a/doc/source/ray-contribute/ci.rst b/doc/source/ray-contribute/ci.rst new file mode 100644 index 000000000000..ba6981f934e9 --- /dev/null +++ b/doc/source/ray-contribute/ci.rst @@ -0,0 +1,52 @@ +CI Testing Workflow on PRs +========================== + +This guide helps contributors to understand the Continuous Integration (CI) +workflow on a PR. Here CI stands for the automated testing of the codebase +on the PR. + +`microcheck`: default tests on your PR +-------------------------------------- +With every commit on your PR, by default, we'll run a set of tests +called `microcheck`. + +These tests are designed to be 90% accurate at catching bugs on your +PR while running only 10% of the full test suite. As a result, +microcheck typically finishes twice as fast and twice cheaper than +the full test suite. Some of the notable features of microcheck are: + +* If a new test is added or an existing test is modified in a pull + request, microcheck will ensure these tests are included. +* You can manually add more tests to microcheck by including the following line + in the body of your git commit message: + `@microcheck TEST_TARGET01 TEST_TARGET02 ....`. This line must be in the + body of your message, starting from the second line or + below (the first line is the commit message title). For example, here + is how I manually add tests in my pull request:: + + // git command to add commit message + git commit -a -s + + // content of the commit message + run other serve doc tests + + @microcheck //doc:source/serve/doc_code/distilbert //doc:source/serve/doc_code/object_detection //doc:source/serve/doc_code/stable_diffusion + + Signed-off-by: can + +If microcheck passes, you'll see a green checkmark on your PR. If it +fails, you'll see a red cross. In either case, you'll see a summary of +the test run statuses in the github UI. + + +Additional tests at merge time +------------------------------ +In this workflow, to merge your PR, simply click on the Enable auto-merge +button (or ask a committer to do so). This will trigger additional test +cases, and the PR will merge automatically once they finish and pass. + +Alternatively, you can also add a `go` label to manually trigger the full +test suite on your PR (be mindful that this is less recommended but we +understand you know best about the need of your PR). While we anticipate +this will be rarely needed, if you do require it constantly, please let +us know. We are continuously improve the effectiveness of microcheck. diff --git a/doc/source/ray-contribute/getting-involved.rst b/doc/source/ray-contribute/getting-involved.rst index c936b2dc7819..2d4b76b00a29 100644 --- a/doc/source/ray-contribute/getting-involved.rst +++ b/doc/source/ray-contribute/getting-involved.rst @@ -10,6 +10,7 @@ Getting Involved / Contributing :hidden: development + ci docs writing-code-snippets fake-autoscaler From f9ac0505b41fcaaa38f134129d5bc1e7eee0a4e0 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Fri, 24 May 2024 15:53:55 -0700 Subject: [PATCH 15/65] [ci][microcheck/12] move get_new_tests and get_human_specified_tests to Test class (#45500) This is part of the series to move some of the function to compute microcheck test coverage to the Test class. This will allow me to reuse the logic more easier in other places. Should be a pure refactoring. Test: - CI --------- Signed-off-by: can --- ci/ray_ci/test_tester.py | 14 +------------ ci/ray_ci/tester.py | 28 ++------------------------ release/ray_release/test.py | 25 +++++++++++++++++++++++ release/ray_release/tests/test_test.py | 11 ++++++++++ 4 files changed, 39 insertions(+), 39 deletions(-) diff --git a/ci/ray_ci/test_tester.py b/ci/ray_ci/test_tester.py index 0b8819d8b6de..86bbee6b859f 100644 --- a/ci/ray_ci/test_tester.py +++ b/ci/ray_ci/test_tester.py @@ -17,7 +17,6 @@ _get_flaky_test_targets, _get_tag_matcher, _get_new_tests, - _get_human_specified_tests, ) from ray_release.test import Test, TestState @@ -257,7 +256,7 @@ def test_get_high_impact_test_targets() -> None: "ray_release.test.Test.get_changed_tests", return_value=test["changed_tests"], ), mock.patch( - "ci.ray_ci.tester._get_human_specified_tests", + "ray_release.test.Test.get_human_specified_tests", return_value=test["human_tests"], ): assert ( @@ -283,17 +282,6 @@ def test_get_new_tests(mock_gen_from_s3, mock_run_script_with_output) -> None: ) == {"//new_test"} -@mock.patch.dict( - os.environ, - {"BUILDKITE_PULL_REQUEST_BASE_BRANCH": "base", "BUILDKITE_COMMIT": "commit"}, -) -@mock.patch("subprocess.check_call") -@mock.patch("subprocess.check_output") -def test_get_human_specified_tests(mock_check_output, mock_check_call) -> None: - mock_check_output.return_value = b"hi\n@microcheck //test01 //test02\nthere" - assert _get_human_specified_tests() == {"//test01", "//test02"} - - def test_get_flaky_test_targets() -> None: test_harness = [ { diff --git a/ci/ray_ci/tester.py b/ci/ray_ci/tester.py index 14f92a4f0918..ec2967689398 100644 --- a/ci/ray_ci/tester.py +++ b/ci/ray_ci/tester.py @@ -1,6 +1,5 @@ import itertools import os -import subprocess import sys from typing import List, Set, Tuple, Optional @@ -18,7 +17,7 @@ from ci.ray_ci.linux_tester_container import LinuxTesterContainer from ci.ray_ci.windows_tester_container import WindowsTesterContainer from ci.ray_ci.tester_container import TesterContainer -from ci.ray_ci.utils import docker_login, ci_init, logger +from ci.ray_ci.utils import docker_login, ci_init from ray_release.test import Test, TestState CUDA_COPYRIGHT = """ @@ -418,7 +417,7 @@ def _get_high_impact_test_targets( } new_tests = _get_new_tests(os_prefix, container) changed_tests = Test.get_changed_tests(bazel_workspace_dir) - human_specified_tests = _get_human_specified_tests() + human_specified_tests = Test.get_human_specified_tests(bazel_workspace_dir) return ( high_impact_tests.union(new_tests) @@ -427,29 +426,6 @@ def _get_high_impact_test_targets( ) -def _get_human_specified_tests() -> Set[str]: - """ - Get all test targets that are specified by humans - """ - base = os.environ.get("BUILDKITE_PULL_REQUEST_BASE_BRANCH") - head = os.environ.get("BUILDKITE_COMMIT") - if not base or not head: - # if not in a PR, return an empty set - return set() - - tests = set() - messages = subprocess.check_output( - ["git", "rev-list", "--format=%b", f"origin/{base}...{head}"], - cwd=bazel_workspace_dir, - ) - for message in messages.decode().splitlines(): - if message.startswith(MICROCHECK_COMMAND): - tests = tests.union(message[len(MICROCHECK_COMMAND) :].strip().split(" ")) - logger.info(f"Human specified tests: {tests}") - - return tests - - def _get_new_tests(prefix: str, container: TesterContainer) -> Set[str]: """ Get all local test targets that are not in database diff --git a/release/ray_release/test.py b/release/ray_release/test.py index b5d5bfef3b9d..fe1df459fd90 100644 --- a/release/ray_release/test.py +++ b/release/ray_release/test.py @@ -28,6 +28,8 @@ get_write_state_machine_aws_bucket, ) +MICROCHECK_COMMAND = "@microcheck" + AWS_TEST_KEY = "ray_tests" AWS_TEST_RESULT_KEY = "ray_test_results" DEFAULT_PYTHON_VERSION = tuple( @@ -212,6 +214,29 @@ def gen_high_impact_tests(cls, prefix: str) -> Dict[str, List]: return step_id_to_tests + @classmethod + def get_human_specified_tests(cls, bazel_workspace_dir: str) -> Set[str]: + """ + Get all test targets that are specified by humans + """ + base = os.environ.get("BUILDKITE_PULL_REQUEST_BASE_BRANCH") + head = os.environ.get("BUILDKITE_COMMIT") + if not base or not head: + # if not in a PR, return an empty set + return set() + + tests = set() + messages = subprocess.check_output( + ["git", "rev-list", "--format=%b", f"origin/{base}...{head}"], + cwd=bazel_workspace_dir, + ) + for message in messages.decode().splitlines(): + if not message.startswith(MICROCHECK_COMMAND): + continue + tests = tests.union(message[len(MICROCHECK_COMMAND) :].strip().split(" ")) + + return tests + @classmethod def get_changed_tests(cls, bazel_workspace_dir: str) -> Set[str]: """ diff --git a/release/ray_release/tests/test_test.py b/release/ray_release/tests/test_test.py index cb9265aeabcd..2513eb1ea401 100644 --- a/release/ray_release/tests/test_test.py +++ b/release/ray_release/tests/test_test.py @@ -405,5 +405,16 @@ def test_get_changed_tests( assert Test.get_changed_tests("") == {"//t1", "//t2"} +@mock.patch.dict( + os.environ, + {"BUILDKITE_PULL_REQUEST_BASE_BRANCH": "base", "BUILDKITE_COMMIT": "commit"}, +) +@mock.patch("subprocess.check_call") +@mock.patch("subprocess.check_output") +def test_get_human_specified_tests(mock_check_output, mock_check_call) -> None: + mock_check_output.return_value = b"hi\n@microcheck //test01 //test02\nthere" + assert Test.get_human_specified_tests("") == {"//test01", "//test02"} + + if __name__ == "__main__": sys.exit(pytest.main(["-v", __file__])) From 91a6ed2f11f412eac65de78eef61c229f7aa09ec Mon Sep 17 00:00:00 2001 From: Andrew Sy Kim Date: Sat, 25 May 2024 01:04:26 -0400 Subject: [PATCH 16/65] [Docs][KubeRay] add example for distributed checkpointing with kuberay and gcsfuse (#45043) Signed-off-by: Andrew Sy Kim --- doc/source/cluster/kubernetes/examples.md | 2 + .../distributed-checkpointing-with-gcsfuse.md | 296 ++++++++++++++++++ 2 files changed, 298 insertions(+) create mode 100644 doc/source/cluster/kubernetes/examples/distributed-checkpointing-with-gcsfuse.md diff --git a/doc/source/cluster/kubernetes/examples.md b/doc/source/cluster/kubernetes/examples.md index 7a867f4866a3..10fbe2ed8b71 100644 --- a/doc/source/cluster/kubernetes/examples.md +++ b/doc/source/cluster/kubernetes/examples.md @@ -13,6 +13,7 @@ examples/text-summarizer-rayservice examples/rayjob-batch-inference-example examples/rayjob-kueue-priority-scheduling examples/rayjob-kueue-gang-scheduling +examples/distributed-checkpointing-with-gcsfuse ``` @@ -26,3 +27,4 @@ This section presents example Ray workloads to try out on your Kubernetes cluste - {ref}`kuberay-batch-inference-example` - {ref}`kuberay-kueue-priority-scheduling-example` - {ref}`kuberay-kueue-gang-scheduling-example` +- {ref}`kuberay-distributed-checkpointing-gcsefuse` diff --git a/doc/source/cluster/kubernetes/examples/distributed-checkpointing-with-gcsfuse.md b/doc/source/cluster/kubernetes/examples/distributed-checkpointing-with-gcsfuse.md new file mode 100644 index 000000000000..c8c512095fee --- /dev/null +++ b/doc/source/cluster/kubernetes/examples/distributed-checkpointing-with-gcsfuse.md @@ -0,0 +1,296 @@ +(kuberay-distributed-checkpointing-gcsefuse)= + +# Distributed checkpointing with KubeRay and GCSFuse + +This example orchestrates distributed checkpointing with KubeRay, using the GCSFuse CSI driver and Google Cloud Storage as the remote storage system. +To illustrate the concepts, this guide uses the [Finetuning a Pytorch Image Classifier with Ray Train](https://docs.ray.io/en/latest/train/examples/pytorch/pytorch_resnet_finetune.html) example. + +## Why distributed checkpointing with GCSFuse? + +In large-scale, high-performance machine learning, distributed checkpointing is crucial for fault tolerance, ensuring that if a node fails during training, Ray can resume the process from the latest saved checkpoint instead of starting from scratch. +While it's possible to directly reference remote storage paths (e.g., `gs://my-checkpoint-bucket`), using Google Cloud Storage FUSE (GCSFuse) has distinct advantages for distributed applications. GCSFuse allows you to mount Cloud Storage buckets like local file systems, making checkpoint management more intuitive for distributed applications that rely on these semantics. Furthermore, GCSFuse is designed for high-performance workloads, delivering the performance and scalability you need for distributed checkpointing of large models. + +[Distributed checkpointing](https://docs.ray.io/en/latest/train/user-guides/checkpoints.html), in combination with [GCSFuse](https://cloud.google.com/storage/docs/gcs-fuse), allows for larger-scale model training with increased availability and efficiency. + +## Create a Kubernetes cluster on GKE + +Create a GKE cluster with the [GCSFuse CSI driver](https://cloud.google.com/kubernetes-engine/docs/how-to/persistent-volumes/cloud-storage-fuse-csi-driver) and [Workload Identity](https://cloud.google.com/kubernetes-engine/docs/how-to/workload-identity) enabled, as well as a GPU node pool with 4 L4 GPUs: +``` +export PROJECT_ID= +gcloud container clusters create kuberay-with-gcsfuse \ + --addons GcsFuseCsiDriver \ + --cluster-version=1.29.4 \ + --location=us-east4-c \ + --machine-type=g2-standard-8 \ + --release-channel=rapid \ + --num-nodes=4 \ + --accelerator type=nvidia-l4,count=1,gpu-driver-version=latest \ + --workload-pool=${PROJECT_ID}.svc.id.goog +``` + +Verify the successful creation of your cluster with 4 GPUs: +``` +$ kubectl get nodes "-o=custom-columns=NAME:.metadata.name,GPU:.status.allocatable.nvidia\.com/gpu" +NAME GPU +gke-kuberay-with-gcsfuse-default-pool-xxxx-0000 1 +gke-kuberay-with-gcsfuse-default-pool-xxxx-1111 1 +gke-kuberay-with-gcsfuse-default-pool-xxxx-2222 1 +gke-kuberay-with-gcsfuse-default-pool-xxxx-3333 1 +``` + +## Install the KubeRay operator + +Follow [Deploy a KubeRay operator](kuberay-operator-deploy) to install the latest stable KubeRay operator from the Helm repository. +The KubeRay operator Pod must be on the CPU node if you set up the taint for the GPU node pool correctly. + +## Configuring the GCS Bucket + +Create a GCS bucket that Ray uses as the remote filesystem. +``` +BUCKET= +gcloud storage buckets create gs://$BUCKET --uniform-bucket-level-access +``` + +Create a Kubernetes ServiceAccount that grants the RayCluster access to mount the GCS bucket: +``` +kubectl create serviceaccount pytorch-distributed-training +``` + +Bind the `roles/storage.objectUser` role to the Kubernetes service account and bucket IAM policy. +See [Identifying projects](https://cloud.google.com/resource-manager/docs/creating-managing-projects#identifying_projects) to find your project ID and project number: +``` +PROJECT_ID= +PROJECT_NUMBER= +gcloud storage buckets add-iam-policy-binding gs://${BUCKET} --member "principal://iam.googleapis.com/projects/${PROJECT_NUMBER}/locations/global/workloadIdentityPools/${PROJECT_ID}.svc.id.goog/subject/ns/default/sa/pytorch-distributed-training" --role "roles/storage.objectUser" +``` + +See [Access Cloud Storage buckets with the Cloud Storage FUSE CSI driver](https://cloud.google.com/kubernetes-engine/docs/how-to/persistent-volumes/cloud-storage-fuse-csi-driver) for more details. + +## Deploy the RayJob + +Download the RayJob that executes all the steps documented in [Finetuning a Pytorch Image Classifier with Ray Train](https://docs.ray.io/en/latest/train/examples/pytorch/pytorch_resnet_finetune.html). The [source code](https://github.com/ray-project/kuberay/tree/master/ray-operator/config/samples/pytorch-resnet-image-classifier) is also in the KubeRay repository. + +``` +curl -LO https://raw.githubusercontent.com/ray-project/kuberay/master/ray-operator/config/samples/pytorch-resnet-image-classifier/ray-job.pytorch-image-classifier.yaml +``` + +Modify the RayJob by replacing all instances of the `GCS_BUCKET` placeholder with the Google Cloud Storage bucket you created earlier. Alternatively you can use `sed`: +``` +sed -i "s/GCS_BUCKET/$BUCKET/g" ray-job.pytorch-image-classifier.yaml +``` + +Deploy the RayJob: +``` +kubectl create -f ray-job.pytorch-image-classifier.yaml +``` + +The deployed RayJob includes the following configuration to enable distributed checkpointing to a shared filesystem: +* 4 Ray workers, each with a single GPU. +* All Ray nodes use the `pytorch-distributed-training` ServiceAccount, which we created earlier. +* Includes volumes that are managed by the `gcsfuse.csi.storage.gke.io` CSI driver. +* Mounts a shared storage path `/mnt/cluster_storage`, backed by the GCS bucket you created earlier. + +You can configure the Pod with annotations, which allows for finer grain control of the GCSFuse sidecar container. See [Specify Pod annotations](https://cloud.google.com/kubernetes-engine/docs/how-to/persistent-volumes/cloud-storage-fuse-csi-driver#pod-annotations) for more details. +``` +annotations: + gke-gcsfuse/volumes: "true" + gke-gcsfuse/cpu-limit: "0" + gke-gcsfuse/memory-limit: 5Gi + gke-gcsfuse/ephemeral-storage-limit: 10Gi +``` + +You can also specify mount options when defining the GCSFuse container volume: +``` +csi: + driver: gcsfuse.csi.storage.gke.io + volumeAttributes: + bucketName: GCS_BUCKET + mountOptions: "implicit-dirs,uid=1000,gid=100" +``` + +See [Mount options](https://cloud.google.com/kubernetes-engine/docs/how-to/persistent-volumes/cloud-storage-fuse-csi-driver#mount-options) to learn more about mount options. + +Logs from the Ray job should indicate the use of the shared remote filesystem in `/mnt/cluster_storage` and the checkpointing directory. For example: +``` +Training finished iteration 10 at 2024-04-29 10:22:08. Total running time: 1min 30s +╭─────────────────────────────────────────╮ +│ Training result │ +├─────────────────────────────────────────┤ +│ checkpoint_dir_name checkpoint_000009 │ +│ time_this_iter_s 6.47154 │ +│ time_total_s 74.5547 │ +│ training_iteration 10 │ +│ acc 0.24183 │ +│ loss 0.06882 │ +╰─────────────────────────────────────────╯ +Training saved a checkpoint for iteration 10 at: (local)/mnt/cluster_storage/finetune-resnet/TorchTrainer_cbb82_00000_0_2024-04-29_10-20-37/checkpoint_000009 +``` + +## Inspect checkpointing data + +Once the RayJob completes, you can inspect the contents of your bucket using a tool like [gsutil](https://cloud.google.com/storage/docs/gsutil). +``` +gsutil ls gs://my-ray-bucket/** +gs://my-ray-bucket/finetune-resnet/ +gs://my-ray-bucket/finetune-resnet/.validate_storage_marker +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/ +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000007/ +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000007/checkpoint.pt +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000008/ +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000008/checkpoint.pt +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000009/ +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000009/checkpoint.pt +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/error.pkl +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/error.txt +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/events.out.tfevents.1714436502.orch-image-classifier-nc2sq-raycluster-tdrfx-head-xzcl8 +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/events.out.tfevents.1714436809.orch-image-classifier-zz4sj-raycluster-vn7kz-head-lwx8k +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/params.json +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/params.pkl +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/progress.csv +gs://my-ray-bucket/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/result.json +gs://my-ray-bucket/finetune-resnet/basic-variant-state-2024-04-29_17-21-29.json +gs://my-ray-bucket/finetune-resnet/basic-variant-state-2024-04-29_17-26-35.json +gs://my-ray-bucket/finetune-resnet/experiment_state-2024-04-29_17-21-29.json +gs://my-ray-bucket/finetune-resnet/experiment_state-2024-04-29_17-26-35.json +gs://my-ray-bucket/finetune-resnet/trainer.pkl +gs://my-ray-bucket/finetune-resnet/tuner.pkl + +``` + +## Resuming from checkpoint + +In the event of a failed job, you can use the latest checkpoint to resume training of the model. This example configures `TorchTrainer` to automatically resume +from the latest checkpoint: +```python +experiment_path = os.path.expanduser("/mnt/cluster_storage/finetune-resnet") +if TorchTrainer.can_restore(experiment_path): + trainer = TorchTrainer.restore(experiment_path, + train_loop_per_worker=train_loop_per_worker, + train_loop_config=train_loop_config, + scaling_config=scaling_config, + run_config=run_config, + ) +else: + trainer = TorchTrainer( + train_loop_per_worker=train_loop_per_worker, + train_loop_config=train_loop_config, + scaling_config=scaling_config, + run_config=run_config, + ) +``` + +You can verify automatic checkpoint recovery by redeploying the same RayJob: +``` +kubectl create -f ray-job.pytorch-image-classifier.yaml +``` + +If the previous job succeeded, the training job should restore the checkpoint state from the `checkpoint_000009` directory +and then immediately complete training with 0 iterations: +``` +2024-04-29 15:51:32,528 INFO experiment_state.py:366 -- Trying to find and download experiment checkpoint at /mnt/cluster_storage/finetune-resnet +2024-04-29 15:51:32,651 INFO experiment_state.py:396 -- A remote experiment checkpoint was found and will be used to restore the previous experiment state. +2024-04-29 15:51:32,652 INFO tune_controller.py:404 -- Using the newest experiment state file found within the experiment directory: experiment_state-2024-04-29_15-43-40.json + +View detailed results here: /mnt/cluster_storage/finetune-resnet +To visualize your results with TensorBoard, run: `tensorboard --logdir /home/ray/ray_results/finetune-resnet` + +Result( + metrics={'loss': 0.070047477101968, 'acc': 0.23529411764705882}, + path='/mnt/cluster_storage/finetune-resnet/TorchTrainer_ecc04_00000_0_2024-04-29_15-43-40', + filesystem='local', + checkpoint=Checkpoint(filesystem=local, path=/mnt/cluster_storage/finetune-resnet/TorchTrainer_ecc04_00000_0_2024-04-29_15-43-40/checkpoint_000009) +) +``` + +If the previous job failed at an earlier checkpoint, the job should resume from the last saved checkpoint and run until `max_epochs=10`. For example, if the last run +failed at epoch 7, the training automatically resumes using `checkpoint_000006` and run 3 more iterations until epoch 10: +``` +(TorchTrainer pid=611, ip=10.108.2.65) Restored on 10.108.2.65 from checkpoint: Checkpoint(filesystem=local, path=/mnt/cluster_storage/finetune-resnet/TorchTrainer_96923_00000_0_2024-04-29_17-21-29/checkpoint_000006) +(RayTrainWorker pid=671, ip=10.108.2.65) Setting up process group for: env:// [rank=0, world_size=4] +(TorchTrainer pid=611, ip=10.108.2.65) Started distributed worker processes: +(TorchTrainer pid=611, ip=10.108.2.65) - (ip=10.108.2.65, pid=671) world_rank=0, local_rank=0, node_rank=0 +(TorchTrainer pid=611, ip=10.108.2.65) - (ip=10.108.1.83, pid=589) world_rank=1, local_rank=0, node_rank=1 +(TorchTrainer pid=611, ip=10.108.2.65) - (ip=10.108.0.72, pid=590) world_rank=2, local_rank=0, node_rank=2 +(TorchTrainer pid=611, ip=10.108.2.65) - (ip=10.108.3.76, pid=590) world_rank=3, local_rank=0, node_rank=3 +(RayTrainWorker pid=589, ip=10.108.1.83) Downloading: "https://download.pytorch.org/models/resnet50-0676ba61.pth" to /home/ray/.cache/torch/hub/checkpoints/resnet50-0676ba61.pth +(RayTrainWorker pid=671, ip=10.108.2.65) + 0%| | 0.00/97.8M [00:00 Date: Sat, 25 May 2024 00:42:33 -0700 Subject: [PATCH 17/65] [Data] Add `override_num_blocks` to `from_pandas` and perform auto-partition (#44937) A common pattern is to load a DataFrame containing file URIs with from_pandas and then loading those URIs with map_batches. If you have a single large DataFrame, the subsequent operator (e.g., for reading) won't be parallelized because from_pandas produces one input block. To fix this issue, this PR automatically splits DataFrames into a good number of blocks, and allows the user to override the number of blocks. Signed-off-by: Balaji Veeramani --- python/ray/data/__init__.py | 1 + .../logical/operators/from_operators.py | 6 ++ python/ray/data/_internal/pandas_block.py | 34 ++++++++++- python/ray/data/_internal/util.py | 4 +- python/ray/data/read_api.py | 59 ++++++++++++++++++- python/ray/data/tests/test_consumption.py | 25 +++++++- python/ray/data/tests/test_csv.py | 4 +- python/ray/data/tests/test_ecosystem.py | 6 +- python/ray/data/tests/test_json.py | 4 +- python/ray/data/tests/test_map.py | 4 +- python/ray/data/tests/test_pandas.py | 27 ++++++++- python/ray/data/tests/test_parquet.py | 4 +- python/ray/data/tests/test_sort.py | 4 +- 13 files changed, 159 insertions(+), 23 deletions(-) diff --git a/python/ray/data/__init__.py b/python/ray/data/__init__.py index 20d2a9d2c263..901a282775c5 100644 --- a/python/ray/data/__init__.py +++ b/python/ray/data/__init__.py @@ -26,6 +26,7 @@ from ray.data.read_api import ( # noqa: F401 from_arrow, from_arrow_refs, + from_blocks, from_dask, from_huggingface, from_items, diff --git a/python/ray/data/_internal/logical/operators/from_operators.py b/python/ray/data/_internal/logical/operators/from_operators.py index 057c07f9f7f6..146a8b7eac15 100644 --- a/python/ray/data/_internal/logical/operators/from_operators.py +++ b/python/ray/data/_internal/logical/operators/from_operators.py @@ -42,6 +42,12 @@ class FromItems(AbstractFrom): pass +class FromBlocks(AbstractFrom): + """Logical operator for `from_blocks`.""" + + pass + + class FromNumpy(AbstractFrom): """Logical operator for `from_numpy`.""" diff --git a/python/ray/data/_internal/pandas_block.py b/python/ray/data/_internal/pandas_block.py index 974c9d3733eb..599ebff222c9 100644 --- a/python/ray/data/_internal/pandas_block.py +++ b/python/ray/data/_internal/pandas_block.py @@ -271,7 +271,9 @@ def to_numpy( def to_arrow(self) -> "pyarrow.Table": import pyarrow - return pyarrow.table(self._table) + # Set `preserve_index=False` so that Arrow doesn't add a '__index_level_0__' + # column to the resulting table. + return pyarrow.Table.from_pandas(self._table, preserve_index=False) @staticmethod def numpy_to_block( @@ -632,3 +634,33 @@ def gen(): def block_type(self) -> BlockType: return BlockType.PANDAS + + +def _estimate_dataframe_size(df: "pandas.DataFrame") -> int: + """Estimate the size of a pandas DataFrame. + + This function is necessary because `DataFrame.memory_usage` doesn't count values in + columns with `dtype=object`. + + The runtime complexity is linear in the number of values, so don't use this in + performance-critical code. + + Args: + df: The DataFrame to estimate the size of. + + Returns: + The estimated size of the DataFrame in bytes. + """ + size = 0 + for column in df.columns: + if df[column].dtype == object: + for item in df[column]: + if isinstance(item, str): + size += len(item) + elif isinstance(item, np.ndarray): + size += item.nbytes + else: + size += 8 # pandas assumes object values are 8 bytes. + else: + size += df[column].nbytes + return size diff --git a/python/ray/data/_internal/util.py b/python/ray/data/_internal/util.py index 954be73b508e..c91f34f1214b 100644 --- a/python/ray/data/_internal/util.py +++ b/python/ray/data/_internal/util.py @@ -649,10 +649,8 @@ def capitalize(s: str): def pandas_df_to_arrow_block(df: "pandas.DataFrame") -> "Block": from ray.data.block import BlockAccessor, BlockExecStats + block = BlockAccessor.for_block(df).to_arrow() stats = BlockExecStats.builder() - import pyarrow as pa - - block = pa.table(df) return ( block, BlockAccessor.for_block(block).get_metadata( diff --git a/python/ray/data/read_api.py b/python/ray/data/read_api.py index aeb6240015b9..d30045cb3085 100644 --- a/python/ray/data/read_api.py +++ b/python/ray/data/read_api.py @@ -1,5 +1,6 @@ import collections import logging +import math import os import warnings from typing import ( @@ -25,12 +26,14 @@ from ray.data._internal.lazy_block_list import LazyBlockList from ray.data._internal.logical.operators.from_operators import ( FromArrow, + FromBlocks, FromItems, FromNumpy, FromPandas, ) from ray.data._internal.logical.operators.read_operator import Read from ray.data._internal.logical.optimizers import LogicalPlan +from ray.data._internal.pandas_block import _estimate_dataframe_size from ray.data._internal.plan import ExecutionPlan from ray.data._internal.remote_fn import cached_remote_fn from ray.data._internal.stats import DatasetStats @@ -104,6 +107,37 @@ logger = logging.getLogger(__name__) +@DeveloperAPI +def from_blocks(blocks: List[Block]): + """Create a :class:`~ray.data.Dataset` from a list of blocks. + + This method is primarily used for testing. Unlike other methods like + :func:`~ray.data.from_pandas` and :func:`~ray.data.from_arrow`, this method + gaurentees that it won't modify the number of blocks. + + Args: + blocks: List of blocks to create the dataset from. + + Returns: + A :class:`~ray.data.Dataset` holding the blocks. + """ + block_refs = [ray.put(block) for block in blocks] + metadata = [ + BlockAccessor.for_block(block).get_metadata(input_files=None, exec_stats=None) + for block in blocks + ] + from_blocks_op = FromBlocks(block_refs, metadata) + logical_plan = LogicalPlan(from_blocks_op) + return MaterializedDataset( + ExecutionPlan( + BlockList(block_refs, metadata, owned_by_consumer=False), + DatasetStats(metadata={"FromBlocks": metadata}, parent=None), + run_by_consumer=False, + ), + logical_plan, + ) + + @PublicAPI def from_items( items: List[Any], @@ -2359,7 +2393,8 @@ def from_modin(df: "modin.pandas.dataframe.DataFrame") -> MaterializedDataset: @PublicAPI def from_pandas( - dfs: Union["pandas.DataFrame", List["pandas.DataFrame"]] + dfs: Union["pandas.DataFrame", List["pandas.DataFrame"]], + override_num_blocks: Optional[int] = None, ) -> MaterializedDataset: """Create a :class:`~ray.data.Dataset` from a list of pandas dataframes. @@ -2373,10 +2408,14 @@ def from_pandas( Create a Ray Dataset from a list of Pandas DataFrames. >>> ray.data.from_pandas([df, df]) - MaterializedDataset(num_blocks=2, num_rows=6, schema={a: int64, b: int64}) + MaterializedDataset(num_blocks=1, num_rows=6, schema={a: int64, b: int64}) Args: dfs: A pandas dataframe or a list of pandas dataframes. + override_num_blocks: Override the number of output blocks from all read tasks. + By default, the number of output blocks is dynamically decided based on + input data size and available resources. You shouldn't manually set this + value in most cases. Returns: :class:`~ray.data.Dataset` holding data read from the dataframes. @@ -2386,13 +2425,27 @@ def from_pandas( if isinstance(dfs, pd.DataFrame): dfs = [dfs] + context = DataContext.get_current() + num_blocks = override_num_blocks + if num_blocks is None: + total_size = sum(_estimate_dataframe_size(df) for df in dfs) + num_blocks = max(math.ceil(total_size / context.target_max_block_size), 1) + + if len(dfs) > 1: + # I assume most users pass a single DataFrame as input. For simplicity, I'm + # concatenating DataFrames, even though it's not efficient. + ary = pd.concat(dfs, axis=0) + else: + ary = dfs[0] + dfs = np.array_split(ary, num_blocks) + from ray.air.util.data_batch_conversion import ( _cast_ndarray_columns_to_tensor_extension, ) - context = DataContext.get_current() if context.enable_tensor_extension_casting: dfs = [_cast_ndarray_columns_to_tensor_extension(df.copy()) for df in dfs] + return from_pandas_refs([ray.put(df) for df in dfs]) diff --git a/python/ray/data/tests/test_consumption.py b/python/ray/data/tests/test_consumption.py index 5581e0265017..5151d9b65e4b 100644 --- a/python/ray/data/tests/test_consumption.py +++ b/python/ray/data/tests/test_consumption.py @@ -679,6 +679,27 @@ def test_convert_types(ray_start_regular_shared): assert arrow_ds.map(lambda x: {"a": (x["id"],)}).take() == [{"a": [0]}] +@pytest.mark.parametrize( + "input_blocks", + [ + [pd.DataFrame({"column": ["spam"]}), pd.DataFrame({"column": ["ham", "eggs"]})], + [ + pa.Table.from_pydict({"column": ["spam"]}), + pa.Table.from_pydict({"column": ["ham", "eggs"]}), + ], + ], +) +def test_from_blocks(input_blocks, ray_start_regular_shared): + ds = ray.data.from_blocks(input_blocks) + + output_blocks = [ray.get(block_ref) for block_ref in ds.get_internal_block_refs()] + assert len(input_blocks) == len(output_blocks) + assert all( + input_block.equals(output_block) + for input_block, output_block in zip(input_blocks, output_blocks) + ) + + def test_from_items(ray_start_regular_shared): ds = ray.data.from_items(["hello", "world"]) assert extract_values("item", ds.take()) == ["hello", "world"] @@ -781,7 +802,7 @@ def test_iter_batches_basic(ray_start_regular_shared): df3 = pd.DataFrame({"one": [7, 8, 9], "two": [8, 9, 10]}) df4 = pd.DataFrame({"one": [10, 11, 12], "two": [11, 12, 13]}) dfs = [df1, df2, df3, df4] - ds = ray.data.from_pandas(dfs) + ds = ray.data.from_blocks(dfs) # Default. for batch, df in zip(ds.iter_batches(batch_size=None, batch_format="pandas"), dfs): @@ -1179,7 +1200,7 @@ def test_iter_batches_grid(ray_start_regular_shared): ) running_size += block_size num_rows = running_size - ds = ray.data.from_pandas(dfs) + ds = ray.data.from_blocks(dfs) for batch_size in np.random.randint( 1, num_rows + 1, size=batch_size_samples ): diff --git a/python/ray/data/tests/test_csv.py b/python/ray/data/tests/test_csv.py index 5d207f8d4b18..be8bfac48312 100644 --- a/python/ray/data/tests/test_csv.py +++ b/python/ray/data/tests/test_csv.py @@ -689,7 +689,7 @@ def test_csv_write(ray_start_regular_shared, fs, data_path, endpoint_url): storage_options = dict(client_kwargs=dict(endpoint_url=endpoint_url)) # Single block. df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) - ds = ray.data.from_pandas([df1]) + ds = ray.data.from_blocks([df1]) ds._set_uuid("data") ds.write_csv(data_path, filesystem=fs) file_path = os.path.join(data_path, "data_000000_000000.csv") @@ -697,7 +697,7 @@ def test_csv_write(ray_start_regular_shared, fs, data_path, endpoint_url): # Two blocks. df2 = pd.DataFrame({"one": [4, 5, 6], "two": ["e", "f", "g"]}) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) ds._set_uuid("data") ds.write_csv(data_path, filesystem=fs) file_path2 = os.path.join(data_path, "data_000001_000000.csv") diff --git a/python/ray/data/tests/test_ecosystem.py b/python/ray/data/tests/test_ecosystem.py index 17df126ccee6..4d124907ff44 100644 --- a/python/ray/data/tests/test_ecosystem.py +++ b/python/ray/data/tests/test_ecosystem.py @@ -33,7 +33,7 @@ def test_to_dask(ray_start_regular_shared, ds_format): df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) df2 = pd.DataFrame({"one": [4, 5, 6], "two": ["e", "f", "g"]}) df = pd.concat([df1, df2]) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) if ds_format == "arrow": ds = ds.map_batches(lambda df: df, batch_format="pyarrow", batch_size=None) ddf = ds.to_dask() @@ -52,7 +52,7 @@ def test_to_dask(ray_start_regular_shared, ds_format): df1["two"] = df1["two"].astype(pd.StringDtype()) df2["two"] = df2["two"].astype(pd.StringDtype()) df = pd.concat([df1, df2]) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) if ds_format == "arrow": ds = ds.map_batches(lambda df: df, batch_format="pyarrow", batch_size=None) ddf = ds.to_dask( @@ -76,7 +76,7 @@ def test_to_dask(ray_start_regular_shared, ds_format): df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) df2 = pd.DataFrame({"three": [4, 5, 6], "four": ["e", "f", "g"]}) df = pd.concat([df1, df2]) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) if ds_format == "arrow": ds = ds.map_batches(lambda df: df, batch_format="pyarrow", batch_size=None) ddf = ds.to_dask(verify_meta=False) diff --git a/python/ray/data/tests/test_json.py b/python/ray/data/tests/test_json.py index 1a90069fc7bc..789fb9e3b89c 100644 --- a/python/ray/data/tests/test_json.py +++ b/python/ray/data/tests/test_json.py @@ -488,7 +488,7 @@ def test_json_write(ray_start_regular_shared, fs, data_path, endpoint_url): storage_options = dict(client_kwargs=dict(endpoint_url=endpoint_url)) # Single block. df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) - ds = ray.data.from_pandas([df1]) + ds = ray.data.from_blocks([df1]) ds._set_uuid("data") ds.write_json(data_path, filesystem=fs) file_path = os.path.join(data_path, "data_000000_000000.json") @@ -500,7 +500,7 @@ def test_json_write(ray_start_regular_shared, fs, data_path, endpoint_url): # Two blocks. df2 = pd.DataFrame({"one": [4, 5, 6], "two": ["e", "f", "g"]}) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) ds._set_uuid("data") ds.write_json(data_path, filesystem=fs) file_path2 = os.path.join(data_path, "data_000001_000000.json") diff --git a/python/ray/data/tests/test_map.py b/python/ray/data/tests/test_map.py index 38a6aacdd8a5..a511eb7255c3 100644 --- a/python/ray/data/tests/test_map.py +++ b/python/ray/data/tests/test_map.py @@ -830,7 +830,7 @@ def test_map_batches_block_bundling_skewed_manual( ray_start_regular_shared, block_sizes, batch_size, expected_num_blocks ): num_blocks = len(block_sizes) - ds = ray.data.from_pandas( + ds = ray.data.from_blocks( [pd.DataFrame({"a": [1] * block_size}) for block_size in block_sizes] ) # Confirm that we have the expected number of initial blocks. @@ -856,7 +856,7 @@ def test_map_batches_block_bundling_skewed_auto( ray_start_regular_shared, block_sizes, batch_size ): num_blocks = len(block_sizes) - ds = ray.data.from_pandas( + ds = ray.data.from_blocks( [pd.DataFrame({"a": [1] * block_size}) for block_size in block_sizes] ) # Confirm that we have the expected number of initial blocks. diff --git a/python/ray/data/tests/test_pandas.py b/python/ray/data/tests/test_pandas.py index ec27c296a296..9bb157789ef4 100644 --- a/python/ray/data/tests/test_pandas.py +++ b/python/ray/data/tests/test_pandas.py @@ -48,6 +48,31 @@ def test_from_pandas(ray_start_regular_shared, enable_pandas_block): ctx.enable_pandas_block = old_enable_pandas_block +def test_from_pandas_default_num_blocks(ray_start_regular_shared, restore_data_context): + ray.data.DataContext.get_current().target_max_block_size = 8 * 1024 * 1024 # 8 MiB + + record = {"number": 0, "string": "\0"} + record_size_bytes = 8 + 1 # 8 bytes for int64 and 1 byte for char + dataframe_size_bytes = 64 * 1024 * 1024 # 64 MiB + num_records = int(dataframe_size_bytes / record_size_bytes) + df = pd.DataFrame.from_records([record] * num_records) + + ds = ray.data.from_pandas(df) + + # If the target block size is 8 MiB, the DataFrame should be split into + # 64 MiB / (8 MiB / block) = 8 blocks. + assert ds.materialize().num_blocks() == 8 + + +@pytest.mark.parametrize("num_inputs", [1, 2]) +def test_from_pandas_override_num_blocks(num_inputs, ray_start_regular_shared): + df = pd.DataFrame({"number": [0]}) + + ds = ray.data.from_pandas([df] * num_inputs, override_num_blocks=2) + + assert ds.materialize().num_blocks() == 2 + + @pytest.mark.parametrize("enable_pandas_block", [False, True]) def test_from_pandas_refs(ray_start_regular_shared, enable_pandas_block): ctx = ray.data.context.DataContext.get_current() @@ -113,7 +138,7 @@ def test_to_pandas_refs(ray_start_regular_shared): def test_pandas_roundtrip(ray_start_regular_shared, tmp_path): df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) df2 = pd.DataFrame({"one": [4, 5, 6], "two": ["e", "f", "g"]}) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_pandas([df1, df2], override_num_blocks=2) dfds = ds.to_pandas() assert pd.concat([df1, df2], ignore_index=True).equals(dfds) diff --git a/python/ray/data/tests/test_parquet.py b/python/ray/data/tests/test_parquet.py index 1e8c5c931a19..d9ecfdb0e09f 100644 --- a/python/ray/data/tests/test_parquet.py +++ b/python/ray/data/tests/test_parquet.py @@ -875,7 +875,7 @@ def test_parquet_write(ray_start_regular_shared, fs, data_path, endpoint_url): df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) df2 = pd.DataFrame({"one": [4, 5, 6], "two": ["e", "f", "g"]}) df = pd.concat([df1, df2]) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) path = os.path.join(data_path, "test_parquet_dir") if fs is None: os.mkdir(path) @@ -928,7 +928,7 @@ def test_parquet_write_create_dir( df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) df2 = pd.DataFrame({"one": [4, 5, 6], "two": ["e", "f", "g"]}) df = pd.concat([df1, df2]) - ds = ray.data.from_pandas([df1, df2]) + ds = ray.data.from_blocks([df1, df2]) path = os.path.join(data_path, "test_parquet_dir") # Set the uuid to a known value so that we can easily get the parquet file names. data_key = "data" diff --git a/python/ray/data/tests/test_sort.py b/python/ray/data/tests/test_sort.py index a0d0eb1f5c67..d66bf8f0d7a0 100644 --- a/python/ray/data/tests/test_sort.py +++ b/python/ray/data/tests/test_sort.py @@ -119,7 +119,7 @@ def test_sort_arrow( offset += shard if offset < num_items: dfs.append(pd.DataFrame({"a": a[offset:], "b": b[offset:]})) - ds = ray.data.from_pandas(dfs).map_batches( + ds = ray.data.from_blocks(dfs).map_batches( lambda t: t, batch_format="pyarrow", batch_size=None ) @@ -235,7 +235,7 @@ def test_sort_pandas(ray_start_regular, num_items, parallelism, use_push_based_s offset += shard if offset < num_items: dfs.append(pd.DataFrame({"a": a[offset:], "b": b[offset:]})) - ds = ray.data.from_pandas(dfs) + ds = ray.data.from_blocks(dfs) def assert_sorted(sorted_ds, expected_rows): assert [tuple(row.values()) for row in sorted_ds.iter_rows()] == list( From 93dce13cbcd6871afd0ed61bd46cd989945667f3 Mon Sep 17 00:00:00 2001 From: Stephanie Wang Date: Sat, 25 May 2024 20:56:50 -0700 Subject: [PATCH 18/65] [core][experimental] Support NCCL-based torch.Tensors nested inside Python objects (#45473) Allows torch.Tensors nested inside Python objects to be transferred via NCCL using the following syntax: ```python with InputNode() as inp: dag = sender.send.bind(inp) dag = dag.with_type_hint(TorchTensorType(transport="nccl")) dag = receiver.recv.bind(dag) ``` We implement this by using an additional shared memory channel to pass CPU data, with a "nested" NCCL channel to pass the GPU data. Here is the send procedure for the above code: 1. Serialize the data. Extract out all tensors that are on the GPU and replace them with some kind of placeholder. 2. Send a list of metadata through the meta_channel. 3. Send the GPU tensors through the NCCL channel. 4. Send the rest of the CPU data through a cpu_data_channel, with the placeholders for the GPU tensors. Note that if the TorchTensorType doesn't have a shape and dtype specified, we currently use the separate meta_channel to pass metadata for the serialized tensors, as introduced in #45332. To elide the cpu_data_channel, the user should now use `TorchTensorType(direct_return=True)`, to indicate that no CPU data is sent along with the GPU data. To elide the meta_channel, the user should declare the shape and dtype, e.g., `TorchTensorType(shape=(10, ), dtype=torch.float16)`. ## Related issue number Closes #45306. --------- Signed-off-by: Stephanie Wang Co-authored-by: SangBin Cho --- python/ray/dag/compiled_dag_node.py | 15 +- python/ray/dag/dag_node.py | 11 +- .../experimental/test_torch_tensor_dag.py | 189 +++++++++++++++- python/ray/experimental/channel/common.py | 44 +++- .../channel/serialization_context.py | 37 ++- .../channel/shared_memory_channel.py | 56 ++++- .../channel/torch_tensor_nccl_channel.py | 213 ++++++++++++++++-- .../experimental/channel/torch_tensor_type.py | 36 ++- 8 files changed, 549 insertions(+), 52 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index c40e1bf8471b..e0cb598d9444 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -115,8 +115,8 @@ def _prep_task(self, task: "ExecutableTask") -> None: else: task.resolved_inputs.append(inp) - for type_hint in task.input_type_hints: - type_hint.register_custom_serializer() + for typ_hint in task.input_type_hints: + typ_hint.register_custom_serializer() task.output_type_hint.register_custom_serializer() input_reader: ReaderInterface = SynchronousReader(task.input_channels) @@ -766,13 +766,16 @@ def teardown(self, wait: bool): for actor in outer.actor_refs: logger.info(f"Cancelling compiled worker on actor: {actor}") - for actor, tasks in outer.actor_to_executable_tasks.items(): + # Cancel all actor loops in parallel. + cancel_refs = [ + actor.__ray_call__.remote(do_cancel_executable_tasks, tasks) + for actor, tasks in outer.actor_to_executable_tasks.items() + ] + for cancel_ref in cancel_refs: try: # TODO(swang): Suppress exceptions from actors trying to # read closed channels when DAG is being torn down. - ray.get( - actor.__ray_call__.remote(do_cancel_executable_tasks, tasks) - ) + ray.get(cancel_ref, timeout=30) except Exception: logger.exception("Error cancelling worker task") pass diff --git a/python/ray/dag/dag_node.py b/python/ray/dag/dag_node.py index 3c66ddd20869..ce48bcc51333 100644 --- a/python/ray/dag/dag_node.py +++ b/python/ray/dag/dag_node.py @@ -66,7 +66,16 @@ def __init__( self._type_hint: Optional[ChannelOutputType] = ChannelOutputType() def with_type_hint(self, typ: ChannelOutputType): - self._type_hint = copy.deepcopy(typ) + if typ.is_direct_return: + old_contains_typ = self._type_hint.contains_type + self._type_hint = copy.deepcopy(typ) + if old_contains_typ is not None and typ.contains_type is None: + # The contained type was set before the return + # type, and the new return type doesn't have a + # contained type set. + self._type_hint.set_contains_type(old_contains_typ) + else: + self._type_hint.set_contains_type(typ) return self @property diff --git a/python/ray/dag/tests/experimental/test_torch_tensor_dag.py b/python/ray/dag/tests/experimental/test_torch_tensor_dag.py index 16a97f1a7a12..b0c9efe02615 100644 --- a/python/ray/dag/tests/experimental/test_torch_tensor_dag.py +++ b/python/ray/dag/tests/experimental/test_torch_tensor_dag.py @@ -3,6 +3,7 @@ import os import sys import torch +import time import pytest @@ -78,7 +79,9 @@ def test_torch_tensor_p2p(ray_start_regular): # Test torch.Tensor sent between actors. with InputNode() as inp: dag = sender.send.bind(shape, dtype, inp) - dag = dag.with_type_hint(TorchTensorType(shape, dtype)) + # TODO(swang): Test that we are using the minimum number of + # channels/messages when direct_return=True. + dag = dag.with_type_hint(TorchTensorType(shape, dtype, direct_return=True)) dag = receiver.recv.bind(dag) compiled_dag = dag.experimental_compile() @@ -94,7 +97,9 @@ def test_torch_tensor_p2p(ray_start_regular): # Passing tensors of a similar or smaller shape is okay. with InputNode() as inp: dag = sender.send.bind(shape, dtype, inp) - dag = dag.with_type_hint(TorchTensorType((20,), dtype)) + # TODO(swang): Test that we are using the minimum number of + # channels/messages when direct_return=True. + dag = dag.with_type_hint(TorchTensorType((20,), dtype, direct_return=True)) dag = receiver.recv.bind(dag) compiled_dag = dag.experimental_compile() for i in range(3): @@ -108,7 +113,9 @@ def test_torch_tensor_p2p(ray_start_regular): # Passing a much larger tensor will error. with InputNode() as inp: dag = sender.send.bind(1_000_000, dtype, inp) - dag = dag.with_type_hint(TorchTensorType(shape, dtype)) + # TODO(swang): Test that we are using the minimum number of + # channels/messages when direct_return=True. + dag = dag.with_type_hint(TorchTensorType(shape, dtype, direct_return=True)) dag = receiver.recv.bind(dag) compiled_dag = dag.experimental_compile() output_channel = compiled_dag.execute(1) @@ -116,6 +123,26 @@ def test_torch_tensor_p2p(ray_start_regular): result = output_channel.begin_read() compiled_dag.teardown() + # Passing a torch.tensor inside of other data is okay even if + # direct_return=True, if `transport` is not set. + with InputNode() as inp: + dag = sender.send_dict_with_tuple_args.bind(inp) + dag = dag.with_type_hint( + TorchTensorType( + shape=shape, + dtype=dtype, + direct_return=True, + ) + ) + dag = receiver.recv_dict.bind(dag) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute((shape, dtype, 1)) + output_channel.begin_read() + output_channel.end_read() + compiled_dag.teardown() + @pytest.mark.parametrize("ray_start_regular", [{"num_cpus": 4}], indirect=True) def test_torch_tensor_as_dag_input(ray_start_regular): @@ -133,7 +160,11 @@ def test_torch_tensor_as_dag_input(ray_start_regular): # Test torch.Tensor as input. with InputNode() as inp: - torch_inp = inp.with_type_hint(TorchTensorType(shape, dtype)) + # TODO(swang): Test that we are using the minimum number of + # channels/messages when direct_return=True. + torch_inp = inp.with_type_hint( + TorchTensorType(shape, dtype, direct_return=True) + ) dag = receiver.recv.bind(torch_inp) compiled_dag = dag.experimental_compile() @@ -178,7 +209,11 @@ def test_torch_tensor_nccl(ray_start_regular): with InputNode() as inp: dag = sender.send.bind(shape, dtype, inp) - dag = dag.with_type_hint(TorchTensorType(shape, dtype, transport="nccl")) + # TODO(swang): Test that we are using the minimum number of + # channels/messages when direct_return=True. + dag = dag.with_type_hint( + TorchTensorType(shape, dtype, transport="nccl", direct_return=True) + ) dag = receiver.recv.bind(dag) # Test normal execution. @@ -243,7 +278,9 @@ def test_torch_tensor_nccl_dynamic(ray_start_regular): with InputNode() as inp: dag = sender.send_with_tuple_args.bind(inp) - dag = dag.with_type_hint(TorchTensorType(transport="nccl")) + # TODO(swang): Test that we are using the minimum number of + # channels/messages when direct_return=True. + dag = dag.with_type_hint(TorchTensorType(transport="nccl", direct_return=True)) dag = receiver.recv.bind(dag) compiled_dag = dag.experimental_compile() @@ -281,7 +318,13 @@ def test_torch_tensor_nccl_wrong_shape(ray_start_regular): # Passing tensors of the wrong shape will error. with InputNode() as inp: dag = sender.send.bind(shape, dtype, inp) - dag = dag.with_type_hint(TorchTensorType((20,), dtype, transport="nccl")) + dag = dag.with_type_hint( + TorchTensorType( + (20,), + dtype, + transport="nccl", + ) + ) dag = receiver.recv.bind(dag) compiled_dag = dag.experimental_compile() @@ -292,8 +335,136 @@ def test_torch_tensor_nccl_wrong_shape(ray_start_regular): compiled_dag.teardown() - ray.kill(sender) - ray.kill(receiver) + # TODO(swang): This currently requires time.sleep to avoid some issue with + # following tests. + time.sleep(3) + + +@pytest.mark.parametrize("ray_start_regular", [{"num_cpus": 4}], indirect=True) +def test_torch_tensor_nccl_nested(ray_start_regular): + """ + Test nested torch.Tensor passed via NCCL. Its shape and dtype is statically + declared. + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + assert ( + sum(node["Resources"].get("GPU", 0) for node in ray.nodes()) > 1 + ), "This test requires at least 2 GPUs" + + actor_cls = TorchTensorWorker.options(num_gpus=1) + + sender = actor_cls.remote() + receiver = actor_cls.remote() + + shape = (10,) + dtype = torch.float16 + + with InputNode() as inp: + dag = sender.send_dict_with_tuple_args.bind(inp) + dag = dag.with_type_hint( + TorchTensorType(shape=shape, dtype=dtype, transport="nccl") + ) + dag = receiver.recv_dict.bind(dag) + + compiled_dag = dag.experimental_compile() + + for i in range(3): + args = (shape, dtype, 1) + + output_channel = compiled_dag.execute(args) + # TODO(swang): Replace with fake ObjectRef. + result = output_channel.begin_read() + expected_result = {0: (0, shape, dtype)} + assert result == expected_result + output_channel.end_read() + + compiled_dag.teardown() + + +@pytest.mark.parametrize("ray_start_regular", [{"num_cpus": 4}], indirect=True) +def test_torch_tensor_nccl_nested_dynamic(ray_start_regular): + """ + Test nested torch.Tensor passed via NCCL. Its shape and dtype is + dynamically declared, and there may be multiple tensors. + """ + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + assert ( + sum(node["Resources"].get("GPU", 0) for node in ray.nodes()) > 1 + ), "This test requires at least 2 GPUs" + + actor_cls = TorchTensorWorker.options(num_gpus=1) + + sender = actor_cls.remote() + receiver = actor_cls.remote() + + with InputNode() as inp: + dag = sender.send_dict_with_tuple_args.bind(inp) + dag = dag.with_type_hint(TorchTensorType(transport="nccl")) + dag = receiver.recv_dict.bind(dag) + + compiled_dag = dag.experimental_compile() + + for i in range(3): + i += 1 + + shape = (10 * i,) + dtype = torch.float16 + args = (shape, dtype, i) + + output_channel = compiled_dag.execute(args) + # TODO(swang): Replace with fake ObjectRef. + result = output_channel.begin_read() + expected_result = {j: (j, shape, dtype) for j in range(i)} + assert result == expected_result + output_channel.end_read() + + compiled_dag.teardown() + + +@pytest.mark.parametrize("ray_start_regular", [{"num_cpus": 4}], indirect=True) +def test_torch_tensor_nccl_direct_return_error(ray_start_regular): + if not USE_GPU: + pytest.skip("NCCL tests require GPUs") + + assert ( + sum(node["Resources"].get("GPU", 0) for node in ray.nodes()) > 1 + ), "This test requires at least 2 GPUs" + + actor_cls = TorchTensorWorker.options(num_cpus=0, num_gpus=1) + + sender = actor_cls.remote() + receiver = actor_cls.remote() + + shape = (10,) + dtype = torch.float16 + + # Passing a non-tensor value when direct_return=True and tranport="nccl" + # fails. + with InputNode() as inp: + dag = sender.send_dict_with_tuple_args.bind(inp) + dag = dag.with_type_hint( + TorchTensorType( + transport=TorchTensorType.NCCL, + direct_return=True, + ) + ) + dag = receiver.recv_dict.bind(dag) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute((shape, dtype, 1)) + with pytest.raises(OSError): + output_channel.begin_read() + + compiled_dag.teardown() + + # TODO(swang): This currently requires time.sleep to avoid some issue with + # following tests. + time.sleep(3) if __name__ == "__main__": diff --git a/python/ray/experimental/channel/common.py b/python/ray/experimental/channel/common.py index b0f173ca7369..4ec632db4132 100644 --- a/python/ray/experimental/channel/common.py +++ b/python/ray/experimental/channel/common.py @@ -1,5 +1,6 @@ import asyncio import concurrent +import copy import threading from dataclasses import dataclass from typing import Any, Dict, List, Optional @@ -16,6 +17,9 @@ @PublicAPI(stability="alpha") class ChannelOutputType: + def __init__(self): + self._contains_type: Optional["ChannelOutputType"] = None + def register_custom_serializer(self) -> None: """ Register any custom serializers needed to pass data of this type. This @@ -29,7 +33,41 @@ def register_custom_serializer(self) -> None: default device. Instead, these should be extracted from the worker-local _SerializationContext. """ - pass + if self._contains_type is not None: + self._contains_type.register_custom_serializer() + + @property + def is_direct_return(self) -> bool: + """ + Some channels may contain other values that should be sent via a + different channel. This returns whether the value is a direct return or + if it is "nested" inside a different channel. + """ + return True + + @property + def contains_type(self) -> "ChannelOutputType": + """ + Some channel values may contain an object that should be sent through a + different channel. For example, a Python object containing a GPU tensor + may be sent over two channels, one to serialize the Python data on CPU + memory and another to transfer the GPU data over NCCL. This function + returns the type of this nested value, if any. + """ + return self._contains_type + + def set_contains_type(self, typ: "ChannelOutputType") -> None: + """ + Mark that values sent on this channel may contain objects that should + be sent through a different channel. + """ + from ray.experimental.channel.torch_tensor_type import TorchTensorType + + if typ is not None: + assert isinstance( + typ, TorchTensorType + ), "Contained type must be of type TorchTensorType" + self._contains_type = copy.deepcopy(typ) def create_channel( self, @@ -51,6 +89,10 @@ def create_channel( raise NotImplementedError def requires_nccl(self) -> bool: + if self._contains_type is not None: + if self._contains_type.requires_nccl(): + return True + # By default, channels do not require NCCL. return False diff --git a/python/ray/experimental/channel/serialization_context.py b/python/ray/experimental/channel/serialization_context.py index 2a70e85b328b..4ad13db74722 100644 --- a/python/ray/experimental/channel/serialization_context.py +++ b/python/ray/experimental/channel/serialization_context.py @@ -1,4 +1,4 @@ -from typing import TYPE_CHECKING, Optional +from typing import TYPE_CHECKING, List, Optional, Union if TYPE_CHECKING: import numpy as np @@ -9,11 +9,32 @@ class _SerializationContext: def __init__(self): self.torch_device: Optional["torch.device"] = None self.use_external_transport: bool = False + self.tensors: List["torch.Tensor"] = [] + + def set_use_external_transport(self, use_external_transport: bool) -> None: + self.use_external_transport = use_external_transport def set_torch_device(self, torch_device: "torch.device") -> None: self.torch_device = torch_device - def serialize_tensor(self, tensor: "torch.Tensor") -> "np.ndarray": + def reset_tensors(self, tensors: List["torch.Tensor"]) -> List["torch.Tensor"]: + prev_tensors = self.tensors + self.tensors = tensors + return prev_tensors + + def serialize_tensor(self, tensor: "torch.Tensor") -> Union[int, "np.ndarray"]: + if self.use_external_transport and tensor.device == self.torch_device: + # External transport is enabled and we found a tensor that matches + # our device. Add the actual tensor to a buffer. The buffer of + # tensors should later be popped by the caller and sent via + # external transport. + self.tensors.append(tensor) + # Return a placeholder. + return len(self.tensors) - 1 + + return self.serialize_to_numpy(tensor) + + def serialize_to_numpy(self, tensor: "torch.Tensor") -> "np.ndarray": # Transfer through Ray's shared memory store for now. # TODO(swang): This requires two copies, one to transfer from GPU to # CPU and another from CPU to shared memory. Ideally we should elide @@ -24,12 +45,20 @@ def serialize_tensor(self, tensor: "torch.Tensor") -> "np.ndarray": return tensor.numpy() - def deserialize_tensor(self, np_array: "np.ndarray"): + def deserialize_tensor(self, val: Union["np.ndarray", int]): + # Found a placeholder for a tensor that was serialized via NCCL. + # Replace it with the corresponding deserialized tensor. + if isinstance(val, int): + return self.tensors[val] + + return self.deserialize_from_numpy(val) + + def deserialize_from_numpy(self, np_array: "np.ndarray"): import torch # TODO(swang): Support local P2P transfers if available. # If there is a GPU assigned to this worker, move it there. - if self.torch_device.type == "cuda": + if self.torch_device is not None and self.torch_device.type == "cuda": # Use zero-copy from_numpy() because we are going to copy to GPU # anyway. # TODO: Pin the np_array memory to reduce data movement time. diff --git a/python/ray/experimental/channel/shared_memory_channel.py b/python/ray/experimental/channel/shared_memory_channel.py index 94c93c35bd56..b99c7bd5fd4c 100644 --- a/python/ray/experimental/channel/shared_memory_channel.py +++ b/python/ray/experimental/channel/shared_memory_channel.py @@ -3,7 +3,9 @@ from typing import Any, List, Optional, Union import ray +from ray._raylet import SerializedObject from ray.experimental.channel.common import ChannelInterface, ChannelOutputType +from ray.experimental.channel.torch_tensor_type import TorchTensorType from ray.util.annotations import PublicAPI # Logger for this module. It should be configured at the entry point @@ -106,8 +108,35 @@ def create_channel( A ChannelInterface that can be used to pass data of this type. """ + if self._contains_type is not None: + assert isinstance( + self._contains_type, TorchTensorType + ), "_contains_type must be of type TorchTensorType" + + from ray.experimental.channel.torch_tensor_nccl_channel import ( + NestedTorchTensorNcclChannel, + ) + + if self._contains_type.requires_nccl(): + cpu_data_typ = SharedMemoryType( + buffer_size_bytes=self.buffer_size_bytes + ) + return NestedTorchTensorNcclChannel( + writer, + readers, + gpu_data_typ=self._contains_type, + cpu_data_typ=cpu_data_typ, + ) + return Channel(writer, readers) + def set_nccl_group_id(self, group_id: str) -> None: + assert self.requires_nccl() + + # Shared memory channels don't need NCCL but they can + # contain objects that use NCCL. + self._contains_type.set_nccl_group_id(group_id) + @PublicAPI(stability="alpha") class Channel(ChannelInterface): @@ -311,17 +340,22 @@ def __reduce__(self): def write(self, value: Any): self.ensure_registered_as_writer() - try: - serialized_value = self._worker.get_serialization_context().serialize(value) - except TypeError as e: - sio = io.StringIO() - ray.util.inspect_serializability(value, print_file=sio) - msg = ( - "Could not serialize the put value " - f"{repr(value)}:\n" - f"{sio.getvalue()}" - ) - raise TypeError(msg) from e + if not isinstance(value, SerializedObject): + try: + serialized_value = self._worker.get_serialization_context().serialize( + value + ) + except TypeError as e: + sio = io.StringIO() + ray.util.inspect_serializability(value, print_file=sio) + msg = ( + "Could not serialize the put value " + f"{repr(value)}:\n" + f"{sio.getvalue()}" + ) + raise TypeError(msg) from e + else: + serialized_value = value self._worker.core_worker.experimental_channel_put_serialized( serialized_value, diff --git a/python/ray/experimental/channel/torch_tensor_nccl_channel.py b/python/ray/experimental/channel/torch_tensor_nccl_channel.py index b4764cc6514b..19020cad0612 100644 --- a/python/ray/experimental/channel/torch_tensor_nccl_channel.py +++ b/python/ray/experimental/channel/torch_tensor_nccl_channel.py @@ -1,7 +1,8 @@ +import io import logging import uuid from types import ModuleType -from typing import TYPE_CHECKING, List, Optional +from typing import TYPE_CHECKING, Any, List, Optional, Union import ray import ray.util.serialization @@ -25,6 +26,151 @@ logger = logging.getLogger(__name__) +class NestedTorchTensorNcclChannel(ChannelInterface): + def __init__( + self, + writer: ray.actor.ActorHandle, + readers: List[ray.actor.ActorHandle], + gpu_data_typ: "TorchTensorType", + cpu_data_typ: Optional["SharedMemoryType"] = None, + _gpu_data_channel: Optional["TorchTensorNcclChannel"] = None, + _cpu_data_channel: Optional["Channel"] = None, + ): + """ + Can be used to send GPU tensors nested inside other data. The data is + sent via shared memory while the GPU tensors are sent through a P2P + transport (NCCL). + + NOTE: This class is currently not thread-safe because it reads and + writes the worker-local + ray.experimental.channel.serialization_context._SerializationContext + when serializing data. + """ + self._writer = writer + self._readers = readers + + if _gpu_data_channel is not None or _cpu_data_channel is not None: + # This path is used when the NestedTorchTensorNcclChannel is being + # deserialized. + assert ( + writer is None + and readers is None + and gpu_data_typ is None + and cpu_data_typ is None + ) + assert _gpu_data_channel is not None and _cpu_data_channel is not None + self._gpu_data_channel = _gpu_data_channel + self._cpu_data_channel = _cpu_data_channel + else: + # This path is used when the NestedTorchTensorNcclChannel is first + # being created, by the writer of the channel. + self._gpu_data_channel: TorchTensorNcclChannel = ( + gpu_data_typ.create_channel(writer, readers) + ) + self._cpu_data_channel: Optional["Channel"] = None + if cpu_data_typ is not None: + self._cpu_data_channel = cpu_data_typ.create_channel(writer, readers) + + # Used for serialization. + self._worker = ray._private.worker.global_worker + self._worker.check_connected() + + ctx = ChannelContext.get_current() + self.serialization_ctx = ctx.serialization_context + assert self.serialization_ctx is not None + + @classmethod + def from_channels( + cls, + gpu_data_channel: "TorchTensorNcclChannel", + cpu_data_channel: Optional["Channel"], + ): + return cls( + writer=None, + readers=None, + gpu_data_typ=None, + cpu_data_typ=None, + _gpu_data_channel=gpu_data_channel, + _cpu_data_channel=cpu_data_channel, + ) + + def __reduce__(self): + return ( + NestedTorchTensorNcclChannel.from_channels, + (self._gpu_data_channel, self._cpu_data_channel), + ) + + def ensure_registered_as_writer(self): + self._gpu_data_channel.ensure_registered_as_writer() + if self._cpu_data_channel is not None: + self._cpu_data_channel.ensure_registered_as_writer() + + def ensure_registered_as_reader(self): + self._gpu_data_channel.ensure_registered_as_reader() + if self._cpu_data_channel is not None: + self._cpu_data_channel.ensure_registered_as_reader() + + def write(self, value: Any): + self.serialization_ctx.reset_tensors([]) + # All tensors found in `value` will be transferred via NCCL. + self.serialization_ctx.set_use_external_transport(True) + + try: + # Serialize the data. All tensors that match our current device + # will be extracted into the serialization context and replaced + # with a placeholder. + serialized_cpu_data = self._worker.get_serialization_context().serialize( + value + ) + except TypeError as e: + sio = io.StringIO() + ray.util.inspect_serializability(value, print_file=sio) + msg = ( + "Could not serialize the put value " + f"{repr(value)}:\n" + f"{sio.getvalue()}" + ) + raise TypeError(msg) from e + finally: + # Pop the tensors that were found during serialization of `value`. + tensors_to_send = self.serialization_ctx.reset_tensors([]) + # Reset the serialization method to now serialize torch.Tensors + # normally. + self.serialization_ctx.set_use_external_transport(False) + + # Send the extracted tensors through a GPU-specific channel. + self._gpu_data_channel.write(tensors_to_send) + # Send the rest of the data, with placeholders for the extracted + # tensors, through a CPU-specific channel. + self._cpu_data_channel.write(serialized_cpu_data) + + def begin_read(self) -> Any: + tensors = self._gpu_data_channel.begin_read() + + if self._gpu_data_channel.has_static_type(): + # If the channel was declared with a static TorchTensorType, then + # the task is allowed to return at most one tensor, and its shape + # and dtype must match the declared type. Wrap the tensor in a + # list since the following calls expect a list. + tensors = [tensors] + + self.serialization_ctx.reset_tensors(tensors) + data = self._cpu_data_channel.begin_read() + self.serialization_ctx.reset_tensors([]) + + return data + + def end_read(self) -> None: + self._gpu_data_channel.end_read() + if self._cpu_data_channel: + self._cpu_data_channel.end_read() + + def close(self) -> None: + self._gpu_data_channel.close() + if self._cpu_data_channel is not None: + self._cpu_data_channel.close() + + @DeveloperAPI class TorchTensorNcclChannel(ChannelInterface): def __init__( @@ -150,16 +296,35 @@ def _get_tensor_meta(self, tensor: "torch.Tensor") -> Optional["TorchTensorType" def write( self, - tensor: "torch.Tensor", + tensors: Union["torch.Tensor", List["torch.Tensor"], Exception], ): - if isinstance(tensor, ray.exceptions.RayTaskError): + if isinstance(tensors, ray.exceptions.RayTaskError): # TODO(swang): Write exceptions to the meta channel if it is # available. - raise tensor - - meta = self._get_tensor_meta(tensor) - if meta is not None: - self._meta_channel.write(meta) + raise tensors + + if isinstance(tensors, list): + meta_list = [] + for tensor in tensors: + meta_list.append(self._get_tensor_meta(tensor)) + if self.has_static_type(): + # Make sure that there is exactly one tensor to send, and its + # metadata should have matched the static type. + if meta_list != [None]: + raise ValueError( + "DAGNode annotated with " + "TorchTensorType(shape=shape, dtype=dtype))` can return at " + "most one tensor with the declared `shape` and `dtype`. " + "Use TorchTensorType() if value contains more than one " + "tensor or tensor of dynamic size." + ) + else: + self._meta_channel.write(meta_list) + else: + meta = self._get_tensor_meta(tensors) + if meta is not None: + self._meta_channel.write(meta) + tensors = [tensors] # NOTE(swang): We must send the metadata *before* launching the NCCL # send. We are using blocking NCCL ops, so the following calls will @@ -167,23 +332,35 @@ def write( # kernel together before either can proceed. Therefore, we send the # metadata first so that the receiver can read the metadata and then # launch the same NCCL op. - # TODO: If there are multiple readers, can replace with a - # broadcast. - for rank in self._reader_ranks: - self._nccl_group.send(tensor, rank) + for tensor in tensors: + # TODO: If there are multiple readers, can replace with a + # broadcast. + for rank in self._reader_ranks: + self._nccl_group.send(tensor, rank) + + def _begin_read_single_tensor(self, typ: "TorchTensorType") -> "torch.Tensor": + buf = self.torch.zeros(typ.shape, dtype=typ.dtype, device=self._device) + self._nccl_group.recv(buf, self._writer_rank) + return buf - def begin_read(self) -> "torch.Tensor": + def begin_read(self) -> Union["torch.Tensor", List["torch.Tensor"]]: if self._meta_channel is not None: - typ = self._meta_channel.begin_read() + meta = self._meta_channel.begin_read() # It's safe to release the channel because shape and dtype should get # copied during deserialization. self._meta_channel.end_read() else: - typ = self._typ + meta = self._typ - buf = self.torch.zeros(typ.shape, dtype=typ.dtype, device=self._device) - self._nccl_group.recv(buf, self._writer_rank) - return buf + if not isinstance(meta, list): + return self._begin_read_single_tensor(meta) + + bufs: List["torch.Tensor"] = [] + for typ in meta: + bufs.append(self._begin_read_single_tensor(typ)) + # TODO: Sync CUDA stream after receiving all tensors, instead of after + # each tensor. + return bufs def end_read(self) -> None: return diff --git a/python/ray/experimental/channel/torch_tensor_type.py b/python/ray/experimental/channel/torch_tensor_type.py index 17ead075c85f..7c987f108311 100644 --- a/python/ray/experimental/channel/torch_tensor_type.py +++ b/python/ray/experimental/channel/torch_tensor_type.py @@ -1,3 +1,4 @@ +import logging from typing import TYPE_CHECKING, List, Optional, Tuple, Union import ray @@ -7,6 +8,8 @@ if TYPE_CHECKING: import torch +logger = logging.getLogger(__name__) + # 100KB to store metadata and/or exceptions. # NOTE(swang): This will consume memory but it should not affect performance # because we only copy the actual data stored, not the maximum size of the @@ -36,7 +39,8 @@ def __init__( self, shape: Union[int, Tuple[int], str] = AUTO, dtype: "torch.dtype" = AUTO, - transport: Optional[str] = None, + transport: Optional[str] = AUTO, + direct_return: Optional[bool] = False, ): """ A type hint that can be used to annotate DAG nodes that return a @@ -55,13 +59,19 @@ def __init__( maximum size of the tensor. If a DAG node's returned serialized tensor exceeds this size, the task will error. For tensors passed via NCCL, the returned tensor must *match* the given - shape; if it does not match, the task will error. + shape; if it does not match, the task will error. Specifying + the shape and dtype ahead of time will eliminate the + performance overhead from an additional metadata transfer. dtype: The expected dtype of the torch.Tensor. Similar to the shape, this may be statically or dynamically declared. transport: "auto" (default) means that tensors will be passed via host memory, using numpy as the serialization format. Pass TorchTensorType.NCCL or "nccl" to use NCCL instead, avoiding the host memory copy. + direct_return: Whether the tensor is sent directly or inside of + other data. If a non-default `transport` is used, this allows + the sender and receiver to eliminate performance overhead from + an additional data transfer. """ super().__init__() @@ -72,10 +82,29 @@ def __init__( self.shape = shape self.dtype = dtype + self.direct_return = direct_return + + if transport not in [self.AUTO, self.NCCL]: + raise ValueError( + "`transport` must be TorchTensorType.AUTO or TorchTensorType.NCCL" + ) self.transport = transport + self._nccl_group_id: Optional[str] = None + if self.direct_return and self.transport == self.AUTO: + logger.info( + "TorchTensorType(direct_return=True) has no effect when " + "`transport` is TorchTensorType.AUTO (default)." + ) + + @property + def is_direct_return(self) -> bool: + return self.direct_return + def register_custom_serializer(self) -> None: + super().register_custom_serializer() + import torch default_device = _get_default_torch_device() @@ -96,6 +125,9 @@ def deserialize(b): deserializer=deserialize, ) + def set_contains_type(self, typ: "ChannelOutputType") -> None: + raise ValueError("TorchTensorType cannot contain other types") + def create_channel( self, writer: Optional["ray.actor.ActorHandle"], From a77726912b8094e65d1dfe48b9da1c7ce1a92ca2 Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Sat, 25 May 2024 23:10:26 -0700 Subject: [PATCH 19/65] [Data] Upgrade Arrow version to 16 in CI (#45565) The most recent stable version of Arrow is 16. So, this PR bumps the version in CI from 15 to 16. Signed-off-by: Balaji Veeramani --- .buildkite/core.rayci.yml | 4 ++-- .buildkite/data.rayci.yml | 24 +++++++++---------- ...ild.wanda.yaml => data16.build.wanda.yaml} | 6 ++--- 3 files changed, 17 insertions(+), 17 deletions(-) rename ci/docker/{data15.build.wanda.yaml => data16.build.wanda.yaml} (83%) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 25a8ba72a287..0c0bb5921e75 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -142,10 +142,10 @@ steps: commands: - bazel run //ci/ray_ci:test_in_docker -- python/ray/util/dask/... python/ray/tests/modin/... core - --build-name data15build + --build-name data16build --parallelism-per-worker 2 depends_on: - - data15build + - data16build - forge - label: ":ray: core: dashboard tests" diff --git a/.buildkite/data.rayci.yml b/.buildkite/data.rayci.yml index 7c5b528dc9f6..0d1b2ac5e28c 100644 --- a/.buildkite/data.rayci.yml +++ b/.buildkite/data.rayci.yml @@ -7,8 +7,8 @@ steps: - name: data6build wanda: ci/docker/data6.build.wanda.yaml - - name: data15build - wanda: ci/docker/data15.build.wanda.yaml + - name: data16build + wanda: ci/docker/data16.build.wanda.yaml - name: databuild-multipy label: "wanda: databuild-py{{matrix}}" @@ -42,7 +42,7 @@ steps: --except-tags data_integration,doctest depends_on: data6build - - label: ":database: data: arrow 15 tests" + - label: ":database: data: arrow 16 tests" tags: - python - data @@ -52,9 +52,9 @@ steps: - bazel run //ci/ray_ci:test_in_docker -- //python/ray/data/... //python/ray/air/... data --workers "$${BUILDKITE_PARALLEL_JOB_COUNT}" --worker-id "$${BUILDKITE_PARALLEL_JOB}" --parallelism-per-worker 3 - --build-name data15build + --build-name data16build --except-tags data_integration,doctest - depends_on: data15build + depends_on: data16build - label: ":database: data: arrow nightly tests" tags: @@ -91,17 +91,17 @@ steps: commands: # doc tests - bazel run //ci/ray_ci:test_in_docker -- python/ray/... //doc/... data - --build-name data15build + --build-name data16build --except-tags gpu --only-tags doctest --parallelism-per-worker 2 # doc examples - bazel run //ci/ray_ci:test_in_docker -- //doc/... data - --build-name data15build + --build-name data16build --except-tags gpu,post_wheel_build,doctest --parallelism-per-worker 2 --skip-ray-installation - depends_on: data15build + depends_on: data16build - label: ":database: data: doc gpu tests" tags: @@ -144,9 +144,9 @@ steps: instance_type: small commands: - bazel run //ci/ray_ci:test_in_docker -- python/ray/dashboard/... data - --build-name data15build + --build-name data16build --parallelism-per-worker 3 - depends_on: data15build + depends_on: data16build - label: ":database: data: flaky tests" tags: @@ -158,9 +158,9 @@ steps: commands: - bazel run //ci/ray_ci:test_in_docker -- //... data --run-flaky-tests --parallelism-per-worker 3 - --build-name data15build + --build-name data16build --except-tags gpu_only,gpu - depends_on: data15build + depends_on: data16build - label: ":database: data: flaky gpu tests" tags: diff --git a/ci/docker/data15.build.wanda.yaml b/ci/docker/data16.build.wanda.yaml similarity index 83% rename from ci/docker/data15.build.wanda.yaml rename to ci/docker/data16.build.wanda.yaml index 20a9b459d0ee..6a9af8b99d99 100644 --- a/ci/docker/data15.build.wanda.yaml +++ b/ci/docker/data16.build.wanda.yaml @@ -1,4 +1,4 @@ -name: "data15build" +name: "data16build" froms: ["cr.ray.io/rayproject/oss-ci-base_ml"] dockerfile: ci/docker/data.build.Dockerfile srcs: @@ -10,6 +10,6 @@ srcs: - python/requirements/ml/data-requirements.txt - python/requirements/ml/data-test-requirements.txt build_args: - - ARROW_VERSION=15.* + - ARROW_VERSION=16.* tags: - - cr.ray.io/rayproject/data15build + - cr.ray.io/rayproject/data16build From 15c294ed6ca3bfd8dcda8f958c354abc9c28295f Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Sun, 26 May 2024 07:39:59 -0700 Subject: [PATCH 20/65] [ci][microcheck/13] reuse the logic to determine microcheck tests (#45504) Currently the logic to determine the list of microcheck tests and their rayci step ids are diverging. This PR move more function into Test class, so we converge more these two logic. This make sure that we will trigger the right buildkite steps to cover most if not all microcheck tests. Test: - CI --------- Signed-off-by: can --- .buildkite/data.rayci.yml | 2 +- .../determine_microcheck_step_ids.py | 9 +- ci/ray_ci/test_tester.py | 70 +------------- ci/ray_ci/tester.py | 38 ++------ release/ray_release/test.py | 61 +++++++++--- release/ray_release/tests/test_test.py | 93 +++++++++++++++---- 6 files changed, 146 insertions(+), 127 deletions(-) diff --git a/.buildkite/data.rayci.yml b/.buildkite/data.rayci.yml index 0d1b2ac5e28c..4b1a0defe788 100644 --- a/.buildkite/data.rayci.yml +++ b/.buildkite/data.rayci.yml @@ -57,10 +57,10 @@ steps: depends_on: data16build - label: ":database: data: arrow nightly tests" + if: pipeline.id == "0189e759-8c96-4302-b6b5-b4274406bf89" || pipeline.id == "018f4f1e-1b73-4906-9802-92422e3badaa" tags: - python - data - - skip-on-premerge instance_type: medium parallelism: 2 commands: diff --git a/ci/ray_ci/automation/determine_microcheck_step_ids.py b/ci/ray_ci/automation/determine_microcheck_step_ids.py index d4ca520e5161..4d15da5f8e6e 100644 --- a/ci/ray_ci/automation/determine_microcheck_step_ids.py +++ b/ci/ray_ci/automation/determine_microcheck_step_ids.py @@ -1,4 +1,5 @@ import click +import os from ci.ray_ci.utils import ci_init from ray_release.test import ( @@ -8,6 +9,8 @@ MACOS_TEST_PREFIX, ) +BAZEL_WORKSPACE_DIR = os.environ.get("BUILD_WORKSPACE_DIRECTORY", "") + @click.command() def main() -> None: @@ -16,9 +19,9 @@ def main() -> None: """ ci_init() steps = ( - list(Test.gen_high_impact_tests(LINUX_TEST_PREFIX).keys()) - + list(Test.gen_high_impact_tests(WINDOWS_TEST_PREFIX).keys()) - + list(Test.gen_high_impact_tests(MACOS_TEST_PREFIX).keys()) + Test.gen_microcheck_step_ids(LINUX_TEST_PREFIX, BAZEL_WORKSPACE_DIR) + .union(Test.gen_microcheck_step_ids(WINDOWS_TEST_PREFIX, BAZEL_WORKSPACE_DIR)) + .union(Test.gen_microcheck_step_ids(MACOS_TEST_PREFIX, BAZEL_WORKSPACE_DIR)) ) print(",".join(steps)) diff --git a/ci/ray_ci/test_tester.py b/ci/ray_ci/test_tester.py index 86bbee6b859f..1a2157dee889 100644 --- a/ci/ray_ci/test_tester.py +++ b/ci/ray_ci/test_tester.py @@ -13,10 +13,9 @@ _get_container, _get_all_test_query, _get_test_targets, - _get_high_impact_test_targets, + _get_new_tests, _get_flaky_test_targets, _get_tag_matcher, - _get_new_tests, ) from ray_release.test import Test, TestState @@ -121,15 +120,12 @@ def test_get_test_targets() -> None: ), mock.patch( "ray_release.test.Test.gen_from_s3", return_value=test_objects, - ), mock.patch( - "ray_release.test.Test.gen_high_impact_tests", - return_value={"step": test_objects}, - ), mock.patch( - "ray_release.test.Test.get_changed_tests", - return_value=set(), ), mock.patch( "ci.ray_ci.tester._get_new_tests", return_value=set(), + ), mock.patch( + "ray_release.test.Test.gen_microcheck_tests", + return_value={test.get_target() for test in test_objects}, ): assert set( _get_test_targets( @@ -211,64 +207,6 @@ def test_get_all_test_query() -> None: ) -def test_get_high_impact_test_targets() -> None: - test_harness = [ - { - "input": [], - "new_tests": set(), - "changed_tests": set(), - "human_tests": set(), - "output": set(), - }, - { - "input": [ - _stub_test( - { - "name": "linux://core_good", - "team": "core", - } - ), - _stub_test( - { - "name": "linux://serve_good", - "team": "serve", - } - ), - ], - "new_tests": {"//core_new"}, - "changed_tests": {"//core_new"}, - "human_tests": {"//human_test"}, - "output": { - "//core_good", - "//core_new", - "//human_test", - }, - }, - ] - for test in test_harness: - with mock.patch( - "ray_release.test.Test.gen_high_impact_tests", - return_value={"step": test["input"]}, - ), mock.patch( - "ci.ray_ci.tester._get_new_tests", - return_value=test["new_tests"], - ), mock.patch( - "ray_release.test.Test.get_changed_tests", - return_value=test["changed_tests"], - ), mock.patch( - "ray_release.test.Test.get_human_specified_tests", - return_value=test["human_tests"], - ): - assert ( - _get_high_impact_test_targets( - "core", - "linux", - LinuxTesterContainer("test", skip_ray_installation=True), - ) - == test["output"] - ) - - @mock.patch("ci.ray_ci.tester_container.TesterContainer.run_script_with_output") @mock.patch("ray_release.test.Test.gen_from_s3") def test_get_new_tests(mock_gen_from_s3, mock_run_script_with_output) -> None: diff --git a/ci/ray_ci/tester.py b/ci/ray_ci/tester.py index ec2967689398..06a1f312808b 100644 --- a/ci/ray_ci/tester.py +++ b/ci/ray_ci/tester.py @@ -1,4 +1,3 @@ -import itertools import os import sys from typing import List, Set, Tuple, Optional @@ -37,7 +36,6 @@ """ # noqa: E501 DEFAULT_EXCEPT_TAGS = {"manual"} -MICROCHECK_COMMAND = "@microcheck" # Gets the path of product/tools/docker (i.e. the parent of 'common') bazel_workspace_dir = os.environ.get("BUILD_WORKSPACE_DIRECTORY", "") @@ -394,38 +392,20 @@ def _get_test_targets( if get_high_impact_tests: # run high impact test cases, so we include only high impact tests in the list # of targets provided by users - high_impact_tests = _get_high_impact_test_targets( - team, operating_system, container - ) + prefix = f"{operating_system}:" + # TODO(can): we should also move the logic of _get_new_tests into the + # gen_microcheck_tests function; this is currently blocked by the fact that + # we need a container to run _get_new_tests + high_impact_tests = Test.gen_microcheck_tests( + prefix=prefix, + bazel_workspace_dir=bazel_workspace_dir, + team=team, + ).union(_get_new_tests(prefix, container)) final_targets = high_impact_tests.intersection(final_targets) return list(final_targets) -def _get_high_impact_test_targets( - team: str, operating_system: str, container: TesterContainer -) -> Set[str]: - """ - Get all test targets that are high impact - """ - os_prefix = f"{operating_system}:" - step_id_to_tests = Test.gen_high_impact_tests(prefix=os_prefix) - high_impact_tests = { - test.get_name().lstrip(os_prefix) - for test in itertools.chain.from_iterable(step_id_to_tests.values()) - if test.get_oncall() == team - } - new_tests = _get_new_tests(os_prefix, container) - changed_tests = Test.get_changed_tests(bazel_workspace_dir) - human_specified_tests = Test.get_human_specified_tests(bazel_workspace_dir) - - return ( - high_impact_tests.union(new_tests) - .union(changed_tests) - .union(human_specified_tests) - ) - - def _get_new_tests(prefix: str, container: TesterContainer) -> Set[str]: """ Get all local test targets that are not in database diff --git a/release/ray_release/test.py b/release/ray_release/test.py index fe1df459fd90..6341053930a7 100644 --- a/release/ray_release/test.py +++ b/release/ray_release/test.py @@ -198,24 +198,63 @@ def gen_from_s3(cls, prefix: str): ] @classmethod - def gen_high_impact_tests(cls, prefix: str) -> Dict[str, List]: + def gen_microcheck_step_ids(cls, prefix: str, bazel_workspace_dir: str) -> Set[str]: """ - Obtain the mapping from rayci step id to high impact tests with the given prefix + This function is used to get the buildkite step ids of the microcheck tests + with the given test prefix. This is used to determine the buildkite steps in + the microcheck pipeline. + """ + step_ids = set() + test_targets = cls.gen_microcheck_tests(prefix, bazel_workspace_dir) + for test_target in test_targets: + test = cls.gen_from_name(f"{prefix}{test_target}") + if not test: + continue + recent_results = test.get_test_results() + if not recent_results: + continue + test_step_ids = { + result.rayci_step_id + for result in recent_results + if result.commit == recent_results[0].commit and result.rayci_step_id + } + if test_step_ids and not step_ids.intersection(test_step_ids): + step_ids.add(sorted(test_step_ids)[0]) + + return step_ids + + @classmethod + def gen_microcheck_tests( + cls, prefix: str, bazel_workspace_dir: str, team: Optional[str] = None + ) -> Set[str]: + """ + Obtain all microcheck tests with the given prefix + """ + high_impact_tests = Test._gen_high_impact_tests(prefix, team) + changed_tests = Test._get_changed_tests(bazel_workspace_dir) + human_specified_tests = Test._get_human_specified_tests(bazel_workspace_dir) + + return high_impact_tests.union(changed_tests, human_specified_tests) + + @classmethod + def _gen_high_impact_tests( + cls, prefix: str, team: Optional[str] = None + ) -> Set[str]: + """ + Obtain all high impact tests with the given prefix """ high_impact_tests = [ test for test in cls.gen_from_s3(prefix) if test.is_high_impact() ] - step_id_to_tests = {} - for test in high_impact_tests: - step_id = test.get_test_results(limit=1)[0].rayci_step_id - if not step_id: - continue - step_id_to_tests[step_id] = step_id_to_tests.get(step_id, []) + [test] + if team: + high_impact_tests = [ + test for test in high_impact_tests if test.get_oncall() == team + ] - return step_id_to_tests + return {test.get_target() for test in high_impact_tests} @classmethod - def get_human_specified_tests(cls, bazel_workspace_dir: str) -> Set[str]: + def _get_human_specified_tests(cls, bazel_workspace_dir: str) -> Set[str]: """ Get all test targets that are specified by humans """ @@ -238,7 +277,7 @@ def get_human_specified_tests(cls, bazel_workspace_dir: str) -> Set[str]: return tests @classmethod - def get_changed_tests(cls, bazel_workspace_dir: str) -> Set[str]: + def _get_changed_tests(cls, bazel_workspace_dir: str) -> Set[str]: """ Get all changed tests in the current PR """ diff --git a/release/ray_release/tests/test_test.py b/release/ray_release/tests/test_test.py index 2513eb1ea401..1a662dc68b71 100644 --- a/release/ray_release/tests/test_test.py +++ b/release/ray_release/tests/test_test.py @@ -61,11 +61,11 @@ def _stub_test(val: dict) -> Test: def _stub_test_result( - status: ResultStatus = ResultStatus.SUCCESS, rayci_step_id="123" + status: ResultStatus = ResultStatus.SUCCESS, rayci_step_id="123", commit="456" ) -> TestResult: return TestResult( status=status.value, - commit="1234567890", + commit=commit, branch="master", url="url", timestamp=0, @@ -333,41 +333,45 @@ def _mock_gen_test_result( ] -@patch("ray_release.test.Test.gen_from_s3") -def gen_high_impact_tests(mock_gen_from_s3) -> None: +@patch("ray_release.test.Test.gen_microcheck_test") +@patch("ray_release.test.Test.gen_from_name") +def gen_microcheck_step_ids(mock_gen_from_name, mock_gen_microcheck_test) -> None: core_test = MockTest( { - "name": "core_test", + "name": "linux://core_test", Test.KEY_IS_HIGH_IMPACT: "false", "test_results": [ - _stub_test_result(rayci_step_id="corebuild"), + _stub_test_result(rayci_step_id="corebuild", commit="123"), ], } ) data_test_01 = MockTest( { - "name": "data_test_01", + "name": "linux://data_test_01", Test.KEY_IS_HIGH_IMPACT: "true", "test_results": [ - _stub_test_result(rayci_step_id="databuild"), + _stub_test_result(rayci_step_id="databuild", commit="123"), ], } ) data_test_02 = MockTest( { - "name": "data_test_02", + "name": "linux://data_test_02", Test.KEY_IS_HIGH_IMPACT: "true", "test_results": [ - _stub_test_result(rayci_step_id="databuild"), + _stub_test_result(rayci_step_id="data15build", commit="123"), + _stub_test_result(rayci_step_id="databuild", commit="123"), + _stub_test_result(rayci_step_id="databuild", commit="456"), ], } ) + all_tests = [core_test, data_test_01, data_test_02] + mock_gen_microcheck_test.return_value = [test.get_target() for test in all_tests] + mock_gen_from_name.side_effect = lambda x: [ + test for test in all_tests if test.get_name() == x + ][0] - mock_gen_from_s3.return_value = [core_test, data_test_01, data_test_02] - - assert Test.gen_high_impact_tests("linux") == { - "databuild": [data_test_01, data_test_02] - } + assert Test.gen_microcheck_step_ids("linux", "") == {"databuild"} def test_get_test_target(): @@ -402,7 +406,7 @@ def test_get_changed_tests( lambda x, _: {"//t1", "//t2"} if x == "test_src" else {} ) - assert Test.get_changed_tests("") == {"//t1", "//t2"} + assert Test._get_changed_tests("") == {"//t1", "//t2"} @mock.patch.dict( @@ -413,7 +417,62 @@ def test_get_changed_tests( @mock.patch("subprocess.check_output") def test_get_human_specified_tests(mock_check_output, mock_check_call) -> None: mock_check_output.return_value = b"hi\n@microcheck //test01 //test02\nthere" - assert Test.get_human_specified_tests("") == {"//test01", "//test02"} + assert Test._get_human_specified_tests("") == {"//test01", "//test02"} + + +def test_gen_microcheck_tests() -> None: + test_harness = [ + { + "input": [], + "changed_tests": set(), + "human_tests": set(), + "output": set(), + }, + { + "input": [ + _stub_test( + { + "name": "linux://core_good", + "team": "core", + Test.KEY_IS_HIGH_IMPACT: "true", + } + ), + _stub_test( + { + "name": "linux://serve_good", + "team": "serve", + Test.KEY_IS_HIGH_IMPACT: "true", + } + ), + ], + "changed_tests": {"//core_new"}, + "human_tests": {"//human_test"}, + "output": { + "//core_good", + "//core_new", + "//human_test", + }, + }, + ] + for test in test_harness: + with mock.patch( + "ray_release.test.Test.gen_from_s3", + return_value=test["input"], + ), mock.patch( + "ray_release.test.Test._get_changed_tests", + return_value=test["changed_tests"], + ), mock.patch( + "ray_release.test.Test._get_human_specified_tests", + return_value=test["human_tests"], + ): + assert ( + Test.gen_microcheck_tests( + prefix="linux", + bazel_workspace_dir="", + team="core", + ) + == test["output"] + ) if __name__ == "__main__": From bbbf51e6a0f258ed33a8114a0a7dfb285c0686dd Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Mon, 27 May 2024 10:12:35 +0200 Subject: [PATCH 21/65] [RLlib] Fix metrics bug (wrt individual agent returns) in `MultiAgentEnvRunner`. (#45543) --- rllib/env/multi_agent_env_runner.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index 043083aaa11a..ddad8b123815 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -610,8 +610,9 @@ def get_metrics(self) -> ResultDict: episode_return += return_eps2 episode_duration_s += eps2.get_duration_s() for sa_eps in eps2.agent_episodes.values(): - agent_episode_returns[str(sa_eps.agent_id)] += return_eps2 - module_episode_returns[sa_eps.module_id] += return_eps2 + return_sa = sa_eps.get_return() + agent_episode_returns[str(sa_eps.agent_id)] += return_sa + module_episode_returns[sa_eps.module_id] += return_sa del self._ongoing_episodes_for_metrics[eps.id_] self._log_episode_metrics( From baffe070eb64f9dc36797e9cab59214da9a3b811 Mon Sep 17 00:00:00 2001 From: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> Date: Mon, 27 May 2024 21:18:13 -0700 Subject: [PATCH 22/65] [Core] Adjust NodeDeathInfo in raylet (#45533) In #45357, in the case of preemption, and after the draining deadline, we expect autoscaler to directly SIGKILL the node without sending SIGTERM first. However, autoscaler would send SIGTERM first then SIGKILL. Therefore, this PR does the following: - At Raylet, distinguish if a SIGTERM is initiated from preemption draining or normal termination based on existing draining request - Construct node death info accordingly and send UnregisterNode() RPC to GCS Signed-off-by: Rui Qiao --- python/ray/tests/test_draining.py | 21 ++++++++---- src/ray/raylet/main.cc | 20 +++++++++-- src/ray/raylet/node_manager.cc | 11 ++---- src/ray/raylet/node_manager.h | 5 +++ .../placement_group_resource_manager_test.cc | 5 +-- .../scheduling/cluster_resource_scheduler.h | 1 + .../cluster_resource_scheduler_test.cc | 6 ++-- .../scheduling/cluster_task_manager_test.cc | 24 ++++++------- .../scheduling/local_resource_manager.cc | 34 +++++++++++++------ .../scheduling/local_resource_manager.h | 31 ++++++++++------- .../scheduling/local_resource_manager_test.cc | 10 +++--- 11 files changed, 105 insertions(+), 63 deletions(-) diff --git a/python/ray/tests/test_draining.py b/python/ray/tests/test_draining.py index 864b4322da65..360e5d5d34ad 100644 --- a/python/ray/tests/test_draining.py +++ b/python/ray/tests/test_draining.py @@ -143,7 +143,11 @@ def ping(self): assert worker_node["DeathReasonMessage"] == "preemption" -def test_preemption_after_draining_deadline(monkeypatch, ray_start_cluster): +@pytest.mark.parametrize( + "graceful", + [True, False], +) +def test_preemption_after_draining_deadline(monkeypatch, ray_start_cluster, graceful): monkeypatch.setenv("RAY_health_check_failure_threshold", "3") monkeypatch.setenv("RAY_health_check_timeout_ms", "100") monkeypatch.setenv("RAY_health_check_period_ms", "1000") @@ -182,9 +186,8 @@ def ping(self): ) assert is_accepted - # Simulate node provider forcefully terminates the worker node - # after the draining deadline. - cluster.remove_node(worker_node, False) + # Simulate autoscaler terminates the worker node after the draining deadline. + cluster.remove_node(worker_node, graceful) wait_for_condition( lambda: {node["NodeID"] for node in ray.nodes() if (node["Alive"])} @@ -384,7 +387,11 @@ def get_node_id(): ray.get(obj, timeout=2) == head_node_id -def test_draining_reason(ray_start_cluster): +@pytest.mark.parametrize( + "graceful", + [False, True], +) +def test_draining_reason(ray_start_cluster, graceful): cluster = ray_start_cluster cluster.add_node(num_cpus=1, resources={"node1": 1}) ray.init( @@ -414,8 +421,8 @@ def ping(self): ) assert is_accepted - # Simulate node provider forcefully terminates the worker node - cluster.remove_node(node2, False) + # Simulate autoscaler terminates the worker node after the draining deadline. + cluster.remove_node(node2, graceful) try: ray.get(actor.ping.remote()) raise diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index caaab409f28a..2942bd50ea4a 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -439,11 +439,25 @@ int main(int argc, char *argv[]) { raylet->Start(); })); - auto signal_handler = [shutdown_raylet_gracefully_internal]( + auto signal_handler = [&raylet, shutdown_raylet_gracefully_internal]( const boost::system::error_code &error, int signal_number) { ray::rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); - node_death_info.set_reason_message("received SIGTERM"); + optional drain_request = + raylet->node_manager().GetLocalDrainRequest(); + RAY_LOG(INFO) << "received SIGTERM. Existing local drain request = " + << (drain_request.has_value() ? drain_request->DebugString() : "None"); + if (drain_request.has_value() && + drain_request->reason() == + ray::rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_PREEMPTION && + drain_request->deadline_timestamp_ms() != 0 && + drain_request->deadline_timestamp_ms() < current_time_ms()) { + node_death_info.set_reason(ray::rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); + node_death_info.set_reason_message(drain_request->reason_message()); + } else { + node_death_info.set_reason(ray::rpc::NodeDeathInfo::EXPECTED_TERMINATION); + node_death_info.set_reason_message("received SIGTERM"); + } + shutdown_raylet_gracefully_internal(node_death_info); }; boost::asio::signal_set signals(main_service); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 0f01440b9d11..a4be13e6b533 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1974,11 +1974,8 @@ void NodeManager::HandleDrainRaylet(rpc::DrainRayletRequest request, const bool is_idle = cluster_resource_scheduler_->GetLocalResourceManager().IsLocalNodeIdle(); if (is_idle) { - rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_IDLE); - node_death_info.set_reason_message(request.reason_message()); cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - request.deadline_timestamp_ms(), node_death_info); + request); reply->set_is_accepted(true); } else { reply->set_is_accepted(false); @@ -1989,11 +1986,7 @@ void NodeManager::HandleDrainRaylet(rpc::DrainRayletRequest request, // Non-rejectable draining request. RAY_CHECK_EQ(request.reason(), rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_PREEMPTION); - rpc::NodeDeathInfo node_death_info; - node_death_info.set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); - node_death_info.set_reason_message(request.reason_message()); - cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - request.deadline_timestamp_ms(), node_death_info); + cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining(request); reply->set_is_accepted(true); } diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 4b5ff79793cf..e39bde0505c4 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -223,6 +223,11 @@ class NodeManager : public rpc::NodeManagerServiceHandler, return mutable_object_provider_; } + /// Get the local drain request. + optional GetLocalDrainRequest() const { + return cluster_resource_scheduler_->GetLocalResourceManager().GetLocalDrainRequest(); + } + private: void ReleaseWorker(const WorkerID &worker_id) { leased_workers_.erase(worker_id); diff --git a/src/ray/raylet/placement_group_resource_manager_test.cc b/src/ray/raylet/placement_group_resource_manager_test.cc index 607d82d9e25a..e650898a6e92 100644 --- a/src/ray/raylet/placement_group_resource_manager_test.cc +++ b/src/ray/raylet/placement_group_resource_manager_test.cc @@ -184,9 +184,10 @@ TEST_F(NewPlacementGroupResourceManagerTest, TestNewPrepareBundleDuringDraining) ASSERT_TRUE(new_placement_group_resource_manager_->PrepareBundles(bundle1_specs)); // Drain the node, new bundle prepare will fail. - rpc::NodeDeathInfo node_death_info; + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); cluster_resource_scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max(), node_death_info); + drain_request); ASSERT_FALSE(new_placement_group_resource_manager_->PrepareBundles(bundle2_specs)); // Prepared bundles can still be committed. new_placement_group_resource_manager_->CommitBundles(bundle1_specs); diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.h b/src/ray/raylet/scheduling/cluster_resource_scheduler.h index fbefe86a6b3b..55b40477a64d 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.h +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.h @@ -125,6 +125,7 @@ class ClusterResourceScheduler { bool requires_object_store_memory); LocalResourceManager &GetLocalResourceManager() { return *local_resource_manager_; } + ClusterResourceManager &GetClusterResourceManager() { return *cluster_resource_manager_; } diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc b/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc index d9c3b67b4c6f..e7c3e3df5444 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler_test.cc @@ -362,9 +362,9 @@ TEST_F(ClusterResourceSchedulerTest, NodeAffinitySchedulingStrategyTest) { ASSERT_TRUE(resource_scheduler.GetLocalResourceManager().AllocateLocalTaskResources( resource_request, task_allocation)); // Drain the local node so that it's not schedulable for new tasks. - rpc::NodeDeathInfo node_death_info; - resource_scheduler.GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + resource_scheduler.GetLocalResourceManager().SetLocalNodeDraining(drain_request); scheduling_strategy.mutable_node_affinity_scheduling_strategy()->set_node_id( local_node_id.Binary()); diff --git a/src/ray/raylet/scheduling/cluster_task_manager_test.cc b/src/ray/raylet/scheduling/cluster_task_manager_test.cc index 3beed07fe7b3..2fe7eec7452a 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager_test.cc +++ b/src/ray/raylet/scheduling/cluster_task_manager_test.cc @@ -743,9 +743,9 @@ TEST_F(ClusterTaskManagerTest, DrainingWhileResolving) { ASSERT_EQ(pool_.workers.size(), 1); // Drain the local node. - rpc::NodeDeathInfo node_death_info; - scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + scheduler_->GetLocalResourceManager().SetLocalNodeDraining(drain_request); // Arg is resolved. missing_objects_.erase(missing_arg); @@ -1078,9 +1078,9 @@ TEST_F(ClusterTaskManagerTest, NotOKPopWorkerAfterDrainingTest) { AddNode(remote_node_id, 5); // Drain the local node. - rpc::NodeDeathInfo node_death_info; - scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + scheduler_->GetLocalResourceManager().SetLocalNodeDraining(drain_request); pool_.callbacks[task1.GetTaskSpecification().GetRuntimeEnvHash()].front()( nullptr, PopWorkerStatus::WorkerPendingRegistration, ""); @@ -2637,9 +2637,9 @@ TEST_F(ClusterTaskManagerTest, PopWorkerBeforeDraining) { task_manager_.QueueAndScheduleTask(task, false, false, &reply, callback); // Drain the local node. - rpc::NodeDeathInfo node_death_info; - scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + scheduler_->GetLocalResourceManager().SetLocalNodeDraining(drain_request); std::shared_ptr worker = std::make_shared(WorkerID::FromRandom(), 1234); @@ -2677,9 +2677,9 @@ TEST_F(ClusterTaskManagerTest, UnscheduleableWhileDraining) { AddNode(remote_node_id, 5); // Drain the local node. - rpc::NodeDeathInfo node_death_info; - scheduler_->GetLocalResourceManager().SetLocalNodeDraining( - std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + scheduler_->GetLocalResourceManager().SetLocalNodeDraining(drain_request); RayTask spillback_task = CreateTask({{ray::kCPU_ResourceLabel, 1}}); rpc::RequestWorkerLeaseReply spillback_reply; diff --git a/src/ray/raylet/scheduling/local_resource_manager.cc b/src/ray/raylet/scheduling/local_resource_manager.cc index e056f8c42d17..06a1b413175a 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.cc +++ b/src/ray/raylet/scheduling/local_resource_manager.cc @@ -254,9 +254,8 @@ NodeResources LocalResourceManager::ToNodeResources() const { node_resources.available = local_resources_.available.ToNodeResourceSet(); node_resources.total = local_resources_.total.ToNodeResourceSet(); node_resources.labels = local_resources_.labels; - node_resources.is_draining = is_local_node_draining_; - node_resources.draining_deadline_timestamp_ms = - local_node_draining_deadline_timestamp_ms_; + node_resources.is_draining = IsLocalNodeDraining(); + node_resources.draining_deadline_timestamp_ms = GetDrainingDeadline(); return node_resources; } @@ -330,8 +329,7 @@ void LocalResourceManager::PopulateResourceViewSyncMessage( } resource_view_sync_message.set_is_draining(IsLocalNodeDraining()); - resource_view_sync_message.set_draining_deadline_timestamp_ms( - local_node_draining_deadline_timestamp_ms_); + resource_view_sync_message.set_draining_deadline_timestamp_ms(GetDrainingDeadline()); for (const auto &iter : last_idle_times_) { if (iter.second == absl::nullopt) { @@ -383,7 +381,8 @@ std::optional LocalResourceManager::CreateSyncMessage( void LocalResourceManager::OnResourceOrStateChanged() { if (IsLocalNodeDraining() && IsLocalNodeIdle()) { RAY_LOG(INFO) << "The node is drained, continue to shut down raylet..."; - shutdown_raylet_gracefully_(node_death_info_); + rpc::NodeDeathInfo node_death_info = DeathInfoFromDrainRequest(); + shutdown_raylet_gracefully_(std::move(node_death_info)); } ++version_; @@ -393,6 +392,22 @@ void LocalResourceManager::OnResourceOrStateChanged() { resource_change_subscriber_(ToNodeResources()); } +rpc::NodeDeathInfo LocalResourceManager::DeathInfoFromDrainRequest() { + rpc::NodeDeathInfo death_info; + RAY_CHECK(drain_request_.has_value()); + if (drain_request_->reason() == + rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_IDLE_TERMINATION) { + death_info.set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_IDLE); + death_info.set_reason_message(drain_request_->reason_message()); + } else { + RAY_CHECK_EQ(drain_request_->reason(), + rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_PREEMPTION); + death_info.set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); + death_info.set_reason_message(drain_request_->reason_message()); + } + return death_info; +} + bool LocalResourceManager::ResourcesExist(scheduling::ResourceID resource_id) const { return local_resources_.total.Has(resource_id); } @@ -445,11 +460,8 @@ void LocalResourceManager::RecordMetrics() const { } void LocalResourceManager::SetLocalNodeDraining( - int64_t draining_deadline_timestamp_ms, const rpc::NodeDeathInfo &node_death_info) { - RAY_CHECK_GE(draining_deadline_timestamp_ms, 0); - is_local_node_draining_ = true; - local_node_draining_deadline_timestamp_ms_ = draining_deadline_timestamp_ms; - node_death_info_ = node_death_info; + const rpc::DrainRayletRequest &drain_request) { + drain_request_ = std::make_optional(drain_request); OnResourceOrStateChanged(); } diff --git a/src/ray/raylet/scheduling/local_resource_manager.h b/src/ray/raylet/scheduling/local_resource_manager.h index a238fffd8ec4..e206b7b9ec9f 100644 --- a/src/ray/raylet/scheduling/local_resource_manager.h +++ b/src/ray/raylet/scheduling/local_resource_manager.h @@ -31,6 +31,7 @@ #include "ray/gcs/gcs_client/gcs_client.h" #include "ray/util/logging.h" #include "src/ray/protobuf/gcs.pb.h" +#include "src/ray/protobuf/node_manager.pb.h" namespace ray { @@ -154,10 +155,17 @@ class LocalResourceManager : public syncer::ReporterInterface { /// Change the local node to the draining state. /// After that, no new tasks can be scheduled onto the local node. - void SetLocalNodeDraining(int64_t draining_deadline_timestamp_ms, - const rpc::NodeDeathInfo &node_death_info); + void SetLocalNodeDraining(const rpc::DrainRayletRequest &drain_request); - bool IsLocalNodeDraining() const { return is_local_node_draining_; } + bool IsLocalNodeDraining() const { return drain_request_.has_value(); } + + /// Get the local drain request. + std::optional GetLocalDrainRequest() const { + return drain_request_; + } + + /// Generate node death info from existing drain request. + rpc::NodeDeathInfo DeathInfoFromDrainRequest(); private: struct ResourceUsage { @@ -206,6 +214,12 @@ class LocalResourceManager : public syncer::ReporterInterface { absl::optional GetResourceIdleTime() const; + /// Get the draining deadline if node is in draining state. + /// + /// \return The draining deadline if node is in draining state, otherwise -1. + int64_t GetDrainingDeadline() const { + return drain_request_.has_value() ? drain_request_->deadline_timestamp_ms() : -1; + } /// Identifier of local node. scheduling::NodeID local_node_id_; /// Resources of local node. @@ -226,15 +240,8 @@ class LocalResourceManager : public syncer::ReporterInterface { // Version of this resource. It will incr by one whenever the state changed. int64_t version_ = 0; - // Whether the local node is being drained or not. - bool is_local_node_draining_ = false; - // The value is the timestamp when - // the node will be force killed. - // 0 if there is no deadline. - int64_t local_node_draining_deadline_timestamp_ms_ = -1; - - /// This is set when the node is being drained and indicates the reason for draining. - rpc::NodeDeathInfo node_death_info_; + /// The draining request this node received. + std::optional drain_request_; FRIEND_TEST(ClusterResourceSchedulerTest, SchedulingUpdateTotalResourcesTest); FRIEND_TEST(ClusterResourceSchedulerTest, AvailableResourceInstancesOpsTest); diff --git a/src/ray/raylet/scheduling/local_resource_manager_test.cc b/src/ray/raylet/scheduling/local_resource_manager_test.cc index 07d64a5331a6..c4ef3e243f7a 100644 --- a/src/ray/raylet/scheduling/local_resource_manager_test.cc +++ b/src/ray/raylet/scheduling/local_resource_manager_test.cc @@ -155,8 +155,9 @@ TEST_F(LocalResourceManagerTest, NodeDrainingTest) { manager->AllocateLocalTaskResources(resource_request, task_allocation); } - rpc::NodeDeathInfo node_death_info; - manager->SetLocalNodeDraining(std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + manager->SetLocalNodeDraining(drain_request); ASSERT_TRUE(manager->IsLocalNodeDraining()); // Make the node idle so that the node is drained and terminated. @@ -182,8 +183,9 @@ TEST_F(LocalResourceManagerTest, ObjectStoreMemoryDrainingTest) { *used_object_store = 1; manager->UpdateAvailableObjectStoreMemResource(); - rpc::NodeDeathInfo node_death_info; - manager->SetLocalNodeDraining(std::numeric_limits::max(), node_death_info); + rpc::DrainRayletRequest drain_request; + drain_request.set_deadline_timestamp_ms(std::numeric_limits::max()); + manager->SetLocalNodeDraining(drain_request); ASSERT_TRUE(manager->IsLocalNodeDraining()); // Free object store memory so that the node is drained and terminated. From d9761d7d49df53383877ff9d8aeb2977270af53c Mon Sep 17 00:00:00 2001 From: Lonnie Liu <95255098+aslonnie@users.noreply.github.com> Date: Tue, 28 May 2024 07:20:56 -0700 Subject: [PATCH 23/65] [dashboard] fix test that relies on tight timing (#45561) metrics might not show up immediately Signed-off-by: Lonnie Liu --- dashboard/tests/test_dashboard.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/dashboard/tests/test_dashboard.py b/dashboard/tests/test_dashboard.py index 4170d594aeed..a5254179d948 100644 --- a/dashboard/tests/test_dashboard.py +++ b/dashboard/tests/test_dashboard.py @@ -1318,12 +1318,16 @@ async def make_blocking_call(): addr = ray_context["raylet_ip_address"] prom_addresses = [f"{addr}:{dashboard_consts.DASHBOARD_METRIC_PORT}"] - metrics_samples: Dict[str, List[Sample]] = fetch_prometheus_metrics(prom_addresses) - print(metrics_samples) + def check_lag_metrics(): + metrics_samples: Dict[str, List[Sample]] = fetch_prometheus_metrics( + prom_addresses + ) + lag_metric_samples = metrics_samples["ray_dashboard_event_loop_lag_seconds"] + assert len(lag_metric_samples) > 0 + assert any(sample.value > 1 for sample in lag_metric_samples) + return True - lag_metric_samples = metrics_samples["ray_dashboard_event_loop_lag_seconds"] - assert len(lag_metric_samples) > 0 - assert any(sample.value > 1 for sample in lag_metric_samples) + wait_for_condition(check_lag_metrics) if __name__ == "__main__": From 5f01083b543272259067b2d2636cff2c2a18aa15 Mon Sep 17 00:00:00 2001 From: simonsays1980 Date: Tue, 28 May 2024 17:14:26 +0200 Subject: [PATCH 24/65] [RLlib] EnvRunners fix min/max-metrics window sizes (from `inf` to `config.metrics_num_episodes_for_smoothing`). (#45575) --- rllib/env/multi_agent_env_runner.py | 2 ++ rllib/env/single_agent_env_runner.py | 8 ++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index ddad8b123815..857b56081fd1 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -901,6 +901,7 @@ def _log_episode_metrics(self, length, ret, sec, agents=None, modules=None): EPISODE_RETURN_MIN: ret, }, reduce="min", + window=self.config.metrics_num_episodes_for_smoothing, ) self.metrics.log_dict( { @@ -908,4 +909,5 @@ def _log_episode_metrics(self, length, ret, sec, agents=None, modules=None): EPISODE_RETURN_MAX: ret, }, reduce="max", + window=self.config.metrics_num_episodes_for_smoothing, ) diff --git a/rllib/env/single_agent_env_runner.py b/rllib/env/single_agent_env_runner.py index 065e70781ae5..e18e32d7010a 100644 --- a/rllib/env/single_agent_env_runner.py +++ b/rllib/env/single_agent_env_runner.py @@ -818,7 +818,7 @@ def _log_episode_metrics(self, length, ret, sec): ) # For some metrics, log min/max as well. - self.metrics.log_value(EPISODE_LEN_MIN, length, reduce="min") - self.metrics.log_value(EPISODE_RETURN_MIN, ret, reduce="min") - self.metrics.log_value(EPISODE_LEN_MAX, length, reduce="max") - self.metrics.log_value(EPISODE_RETURN_MAX, ret, reduce="max") + self.metrics.log_value(EPISODE_LEN_MIN, length, reduce="min", window=win) + self.metrics.log_value(EPISODE_RETURN_MIN, ret, reduce="min", window=win) + self.metrics.log_value(EPISODE_LEN_MAX, length, reduce="max", window=win) + self.metrics.log_value(EPISODE_RETURN_MAX, ret, reduce="max", window=win) From c4d6bcfbb1473fd2f81f603e283bb8e9314a63b9 Mon Sep 17 00:00:00 2001 From: Peyton Murray Date: Tue, 28 May 2024 09:19:59 -0700 Subject: [PATCH 25/65] [Core] Add placeholder callback methods as members of RayDaskCallback (#45160) ## Why are these changes needed? This PR adds the overridable methods for `RayDaskCallback` to the base class. Previously, the user had a few choices for instantiating a `RayDaskCallback`: ```python RayDaskCallback(ray_presubmit=my_custom_presubmit_func) ``` or ```python with RayDaskCallback(ray_presubmit=my_custom_presubmit_func) as cb: ... ``` or by subclassing: ```python class MyCallback(RayDaskCallback): def _ray_presubmit(self, task, key, deps): .... ``` Currently the constructor for `RayDaskCallback` dynamically generates the callback methods and attaches them to the class instance upon instantiation using `setattr`. This pattern doesn't work with developer tooling including language servers, static code analysis tools, or documentation generators because these functions are generated dynamically. Additionally the documentation for dask-on-ray instructs users to use the third approach (using subclassing) rather than passing methods to the constructor. Finally, the docstring for `RayDaskCallback.__init__` currently has function signatures and pseudo-docstrings for the callback methods the user should be implementing. This docstring is itself malformed, so Sphinx mistakenly tries to interpret parts of this as references which do not exist. At the request of @c21 I've actually implemented these functions as they appear in the docstring. Type hints are incomplete here, but match what was already in the previous docstring. I also added a summary in the dask-on-ray documentation. Unblocks #39658. --- doc/source/ray-more-libs/dask-on-ray.rst | 12 ++ python/ray/util/dask/callbacks.py | 174 +++++++++++++---------- 2 files changed, 107 insertions(+), 79 deletions(-) diff --git a/doc/source/ray-more-libs/dask-on-ray.rst b/doc/source/ray-more-libs/dask-on-ray.rst index 5ff6578471eb..918628b8baa9 100644 --- a/doc/source/ray-more-libs/dask-on-ray.rst +++ b/doc/source/ray-more-libs/dask-on-ray.rst @@ -271,3 +271,15 @@ execution time exceeds some user-defined threshold: *submitted*, not executed. This callback API is currently unstable and subject to change. + +.. autosummary:: + :nosignatures: + :toctree: doc/ + + ray.util.dask.callbacks.RayDaskCallback + ray.util.dask.callbacks.RayDaskCallback._ray_presubmit + ray.util.dask.callbacks.RayDaskCallback._ray_postsubmit + ray.util.dask.callbacks.RayDaskCallback._ray_pretask + ray.util.dask.callbacks.RayDaskCallback._ray_posttask + ray.util.dask.callbacks.RayDaskCallback._ray_postsubmit_all + ray.util.dask.callbacks.RayDaskCallback._ray_finish diff --git a/python/ray/util/dask/callbacks.py b/python/ray/util/dask/callbacks.py index 8cbc5c311f9f..458402b059ac 100644 --- a/python/ray/util/dask/callbacks.py +++ b/python/ray/util/dask/callbacks.py @@ -1,7 +1,9 @@ import contextlib +from ray import ObjectRef from collections import namedtuple, defaultdict from datetime import datetime +from typing import Any, List, Optional from dask.callbacks import Callback @@ -48,85 +50,6 @@ class RayDaskCallback(Callback): ray_active = set() def __init__(self, **kwargs): - """ - Ray-specific callbacks: - - def _ray_presubmit(task, key, deps): - Run before submitting a Ray task. If this callback returns a - non-`None` value, a Ray task will _not_ be created and this - value will be used as the would-be task's result value. - - Args: - task: A Dask task, where the first tuple item is - the task function, and the remaining tuple items are - the task arguments (either the actual argument values, - or Dask keys into the deps dictionary whose - corresponding values are the argument values). - key: The Dask graph key for the given task. - deps: The dependencies of this task. - - Returns: - Either None, in which case a Ray task will be submitted, or - a non-None value, in which case a Ray task will not be - submitted and this return value will be used as the - would-be task result value. - - - def _ray_postsubmit(task, key, deps, object_ref): - Run after submitting a Ray task. - - Args: - task: A Dask task, where the first tuple item is - the task function, and the remaining tuple items are - the task arguments (either the actual argument values, - or Dask keys into the deps dictionary whose - corresponding values are the argument values). - key: The Dask graph key for the given task. - deps: The dependencies of this task. - object_ref (ray.ObjectRef): The object reference for the - return value of the Ray task. - - - def _ray_pretask(key, object_refs): - Run before executing a Dask task within a Ray task. This - executes after the task has been submitted, within a Ray - worker. The return value of this task will be passed to the - _ray_posttask callback, if provided. - - Args: - key: The Dask graph key for the Dask task. - object_refs (List[ray.ObjectRef]): The object references - for the arguments of the Ray task. - - Returns: - A value that will be passed to the corresponding - _ray_posttask callback, if said callback is defined. - - - def _ray_posttask(key, result, pre_state): - Run after executing a Dask task within a Ray task. This - executes within a Ray worker. This callback receives the return - value of the _ray_pretask callback, if provided. - - Args: - key: The Dask graph key for the Dask task. - result: The task result value. - pre_state: The return value of the corresponding - _ray_pretask callback, if said callback is defined. - - - def _ray_postsubmit_all(object_refs, dsk): - Run after all Ray tasks have been submitted. - - Args: - object_refs (List[ray.ObjectRef]): The object references - for the output (leaf) Ray tasks of the task graph. - dsk: The Dask graph. - - - def _ray_finish(result): - Run after all Ray tasks have finished executing and the final - result has been returned. - - Args: - result: The final result (output) of the Dask - computation, before any repackaging is done by - Dask collection-specific post-compute callbacks. - """ for cb in CBS: cb_func = kwargs.pop(cb, None) if cb_func is not None: @@ -156,6 +79,99 @@ def unregister(self): type(self).ray_active.remove(self._ray_callback) super().unregister() + def _ray_presubmit(self, task, key, deps) -> Optional[Any]: + """Run before submitting a Ray task. + + If this callback returns a non-`None` value, Ray does _not_ create + a task and uses this value as the would-be task's result value. + + Args: + task: A Dask task, where the first tuple item is + the task function, and the remaining tuple items are + the task arguments, which are either the actual argument values, + or Dask keys into the deps dictionary whose + corresponding values are the argument values. + key: The Dask graph key for the given task. + deps: The dependencies of this task. + + Returns: + Either None, in which case Ray submits a task, or + a non-None value, in which case Ray task doesn't submit + a task and uses this return value as the + would-be task result value. + """ + pass + + def _ray_postsubmit(self, task, key, deps, object_ref: ObjectRef): + """Run after submitting a Ray task. + + Args: + task: A Dask task, where the first tuple item is + the task function, and the remaining tuple items are + the task arguments, which are either the actual argument values, + or Dask keys into the deps dictionary whose + corresponding values are the argument values. + key: The Dask graph key for the given task. + deps: The dependencies of this task. + object_ref: The object reference for the + return value of the Ray task. + + """ + pass + + def _ray_pretask(self, key, object_refs: List[ObjectRef]): + """Run before executing a Dask task within a Ray task. + + This method executes after Ray submits the task within a Ray + worker. Ray passes the return value of this task to the + _ray_posttask callback, if provided. + + Args: + key: The Dask graph key for the Dask task. + object_refs: The object references + for the arguments of the Ray task. + + Returns: + A value that Ray passes to the corresponding + _ray_posttask callback, if the callback is defined. + """ + pass + + def _ray_posttask(self, key, result, pre_state): + """Run after executing a Dask task within a Ray task. + + This method executes within a Ray worker. This callback receives the + return value of the _ray_pretask callback, if provided. + + Args: + key: The Dask graph key for the Dask task. + result: The task result value. + pre_state: The return value of the corresponding + _ray_pretask callback, if said callback is defined. + """ + pass + + def _ray_postsubmit_all(self, object_refs: List[ObjectRef], dsk): + """Run after Ray submits all tasks. + + Args: + object_refs: The object references + for the output (leaf) Ray tasks of the task graph. + dsk: The Dask graph. + """ + pass + + def _ray_finish(self, result): + """Run after Ray finishes executing all Ray tasks and returns the final + result. + + Args: + result: The final result (output) of the Dask + computation, before any repackaging is done by + Dask collection-specific post-compute callbacks. + """ + pass + class add_ray_callbacks: def __init__(self, *callbacks): From ce0932baa4b3dc535ceb968cea1aca354a5b983e Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Tue, 28 May 2024 10:26:37 -0700 Subject: [PATCH 26/65] [core] add method to kill aws instance to simulate chaos (#45546) Signed-off-by: hongchaodeng --- python/ray/_private/test_utils.py | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/python/ray/_private/test_utils.py b/python/ray/_private/test_utils.py index 16fc00dbb939..62e51016af2e 100644 --- a/python/ray/_private/test_utils.py +++ b/python/ray/_private/test_utils.py @@ -1533,6 +1533,25 @@ def _kill_resource(self, node_id, node_to_kill_ip, node_to_kill_port): ) self.killed.add(node_id) + def _terminate_ec2_instance(self, ip): + # This command uses IMDSv2 to get the host instance id and region. + # After that it terminates itself using aws cli. + multi_line_command = ( + 'TOKEN=$(curl -X PUT "http://169.254.169.254/latest/api/token" -H "X-aws-ec2-metadata-token-ttl-seconds: 21600");' # noqa: E501 + 'instanceId=$(curl -H "X-aws-ec2-metadata-token: $TOKEN" http://169.254.169.254/latest/meta-data/instance-id/);' # noqa: E501 + 'region=$(curl -H "X-aws-ec2-metadata-token: $TOKEN" http://169.254.169.254/latest/meta-data/placement/region);' # noqa: E501 + "aws ec2 terminate-instances --region $region --instance-ids $instanceId" # noqa: E501 + ) + # This is a feature on Anyscale platform that enables + # easy ssh access to worker nodes. + ssh_command = f"ssh -o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null -p 2222 ray@{ip} '{multi_line_command}'" # noqa: E501 + + result = subprocess.run( + ssh_command, shell=True, capture_output=True, text=True, check=True + ) + print(f"STDOUT:\n{result.stdout}\n") + print(f"STDERR:\n{result.stderr}\n") + def _kill_raylet(self, ip, port, graceful=False): import grpc from grpc._channel import _InactiveRpcError From fa0e6d7dbab6d9e36c7b39889065d012b8ffbe1a Mon Sep 17 00:00:00 2001 From: Peyton Murray Date: Tue, 28 May 2024 10:44:16 -0700 Subject: [PATCH 27/65] [Doc] Rework Serve example page (#45231) ## Why are these changes needed? This PR changes the grouping of the Serve examples page, and allows the grouping of any example page to be configurable by changing the value of the `groupby` key in `examples.yml`. ## Related issue number Closes #44457. ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [x] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [x] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [x] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: pdmurray Signed-off-by: Peyton Murray Co-authored-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/custom_directives.py | 77 +++++++++++++++++++++++++++++---- doc/source/data/examples.yml | 1 + doc/source/serve/examples.yml | 14 +++++- doc/source/train/examples.yml | 3 +- 4 files changed, 84 insertions(+), 11 deletions(-) diff --git a/doc/source/custom_directives.py b/doc/source/custom_directives.py index ed6e1b165975..3e45de0cb91a 100644 --- a/doc/source/custom_directives.py +++ b/doc/source/custom_directives.py @@ -1,6 +1,7 @@ from collections.abc import Iterable from enum import Enum import re +from collections import defaultdict import sphinx from typing import List, Dict, Union, Callable, Any, Optional, Tuple import copy @@ -403,6 +404,10 @@ def formatted_name(cls: type) -> str: """Return the formatted name for the class.""" raise NotImplementedError + @classmethod + def key(cls: type) -> str: + raise NotImplementedError + class Contributor(ExampleEnum): RAY_TEAM = "Maintained by the Ray Team" @@ -418,6 +423,10 @@ def tag(self): def formatted_name(cls): return "All Examples" + @classmethod + def key(cls: type) -> str: + return "contributor" + class UseCase(ExampleEnum): """Use case type for example metadata.""" @@ -431,6 +440,10 @@ class UseCase(ExampleEnum): def formatted_name(cls): return "Use Case" + @classmethod + def key(cls: type) -> str: + return "use_case" + class SkillLevel(ExampleEnum): """Skill level type for example metadata.""" @@ -443,6 +456,10 @@ class SkillLevel(ExampleEnum): def formatted_name(cls): return "Skill Level" + @classmethod + def key(cls: type) -> str: + return "skill_level" + class Framework(ExampleEnum): """Framework type for example metadata.""" @@ -462,6 +479,24 @@ class Framework(ExampleEnum): def formatted_name(cls): return "Framework" + @classmethod + def key(cls: type) -> str: + return "framework" + + +class RelatedTechnology(ExampleEnum): + ML_APPLICATIONS = "ML Applications" + INTEGRATIONS = "Integrations" + AI_ACCELERATORS = "AI Accelerators" + + @classmethod + def formatted_name(cls): + return "Related Technology" + + @classmethod + def key(cls: type) -> str: + return "related_technology" + class Library(ExampleEnum): """Library type for example metadata.""" @@ -474,6 +509,10 @@ class Library(ExampleEnum): def formatted_name(cls): return "Library" + @classmethod + def key(cls: type) -> str: + return "library" + @classmethod def from_path(cls, path: Union[pathlib.Path, str]) -> "Library": """Instantiate a Library instance from a path. @@ -534,6 +573,9 @@ def __init__( else: self.contributor = Contributor.RAY_TEAM + related_technology = config.get("related_technology", "").strip() + if related_technology: + self.related_technology = RelatedTechnology(related_technology) self.skill_level = SkillLevel(config.get("skill_level")) self.title = config.get("title") @@ -590,6 +632,17 @@ def __init__( self.columns_to_show = self.parse_columns_to_show( config.get("columns_to_show", []) ) + groupby = config.get("groupby", "skill_level") + for cls in ExampleEnum.__subclasses__(): + if cls.key() == groupby: + self.groupby = cls + break + else: + valid_classes = [cls.key() for cls in ExampleEnum.__subclasses__()] + raise ValueError( + f"Unable to find class to group example entries by {groupby}. " + f"Valid choices are {valid_classes}.", + ) def parse_columns_to_show(self, columns: str) -> Dict[str, type]: """Parse the columns to show in the library example page for the config. @@ -675,17 +728,23 @@ def render_library_examples(config: pathlib.Path = None) -> bs4.BeautifulSoup: if config is None: config = (pathlib.Path(app.confdir) / pagename).parent / "examples.yml" - # Separate the examples into different skill levels - examples = { - SkillLevel.BEGINNER: [], - SkillLevel.INTERMEDIATE: [], - SkillLevel.ADVANCED: [], - } # Keep track of whether any of the examples have frameworks metadata; the # column will not be shown if no frameworks metadata exists on any example. + + # Group the examples by the ExampleConfig.groupby value: + examples = defaultdict(list) example_config = ExampleConfig(config, app.srcdir) for example in example_config: - examples[example.skill_level].append(example) + try: + group = getattr(example, example_config.groupby.key()) + except AttributeError as e: + raise AttributeError( + f"Example {example.link} has no {example_config.groupby.key()} " + "key, but needs one because the examples for library " + f"{example_config.library.value} are configured to be grouped " + f"by {example_config.groupby.key()}." + ) from e + examples[group].append(example) # Construct a table of examples soup = bs4.BeautifulSoup() @@ -700,12 +759,12 @@ def render_library_examples(config: pathlib.Path = None) -> bs4.BeautifulSoup: soup.append(page_text) container = soup.new_tag("div", attrs={"class": "example-index"}) - for level, examples in examples.items(): + for group, examples in examples.items(): if not examples: continue header = soup.new_tag("h2", attrs={"class": "example-header"}) - header.append(level.value) + header.append(group.value) container.append(header) table = soup.new_tag("table", attrs={"class": ["table", "example-table"]}) diff --git a/doc/source/data/examples.yml b/doc/source/data/examples.yml index 8f25b358b969..291893550f0c 100644 --- a/doc/source/data/examples.yml +++ b/doc/source/data/examples.yml @@ -1,6 +1,7 @@ text: Below are examples for using Ray Data for batch inference workloads with a variety of frameworks and use cases. columns_to_show: - frameworks +groupby: skill_level examples: - title: Image Classification Batch Inference with PyTorch ResNet152 skill_level: beginner diff --git a/doc/source/serve/examples.yml b/doc/source/serve/examples.yml index 9093a19c53f6..93551cecd99b 100644 --- a/doc/source/serve/examples.yml +++ b/doc/source/serve/examples.yml @@ -1,4 +1,5 @@ text: Below are tutorials for exploring Ray Serve capabilities and learning how to integrate different modeling frameworks. +groupby: related_technology examples: - title: Serve ML Models skill_level: beginner @@ -7,12 +8,14 @@ examples: use_cases: - computer vision link: tutorials/serve-ml-models + related_technology: ml applications - title: Serve a Stable Diffusion Model skill_level: beginner use_cases: - computer vision - generative ai link: tutorials/stable-diffusion + related_technology: ml applications - title: Serve a Large Language Model skill_level: beginner use_cases: @@ -26,22 +29,26 @@ examples: use_cases: - natural language processing link: tutorials/text-classification + related_technology: ml applications - title: Serve an Object Detection Model skill_level: beginner use_cases: - computer vision link: tutorials/object-detection + related_technology: ml applications - title: Serve an Inference Model on AWS NeuronCores Using FastAPI skill_level: intermediate use_cases: - natural language processing link: tutorials/aws-neuron-core-inference + related_technology: ai accelerators - title: Serve an Inference with Stable Diffusion Model on AWS NeuronCores Using FastAPI skill_level: intermediate use_cases: - computer vision - generative ai link: tutorials/aws-neuron-core-inference-stable-diffusion + related_technology: ai accelerators - title: Serve a model on Intel Gaudi Accelerator skill_level: intermediate frameworks: @@ -50,6 +57,7 @@ examples: - generative ai - large language models link: tutorials/intel-gaudi-inference + related_technology: ai accelerators - title: Scale a Gradio App with Ray Serve skill_level: intermediate use_cases: @@ -57,6 +65,7 @@ examples: - large language models - natural language processing link: tutorials/gradio-integration + related_technology: integrations - title: Serve a Text Generator with Request Batching skill_level: intermediate use_cases: @@ -64,6 +73,7 @@ examples: - large language models - natural language processing link: tutorials/batch + related_technology: integrations - title: Serve a Chatbot with Request and Response Streaming skill_level: intermediate use_cases: @@ -71,13 +81,15 @@ examples: - large language models - natural language processing link: tutorials/streaming + related_technology: ml applications - title: Serving models with Triton Server in Ray Serve skill_level: intermediate use_cases: - computer vision - generative ai link: tutorials/triton-server-integration - + related_technology: integrations - title: Serve a Java App skill_level: advanced link: tutorials/java + related_technology: integrations diff --git a/doc/source/train/examples.yml b/doc/source/train/examples.yml index d46b53964b41..3130c590c911 100644 --- a/doc/source/train/examples.yml +++ b/doc/source/train/examples.yml @@ -1,6 +1,7 @@ -text: Below are examples for using Ray Train with a variety of frameworks and use cases. Ray Train makes it easy to scale out each of these examples to a large cluster of GPUs! +text: Below are examples for using Ray Train with a variety of frameworks and use cases. Ray Train makes it easy to scale out each of these examples to a large cluster of GPUs. columns_to_show: - frameworks +groupby: skill_level examples: - title: Train an image classifier with PyTorch skill_level: beginner From 678823dd0788abe6f776c22311a1ecf62f387869 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Tue, 28 May 2024 20:44:41 +0200 Subject: [PATCH 28/65] [RLlib] Change/fix MetricsLogger `log_n_dicts()` logic. (#45585) --- rllib/BUILD | 2 +- rllib/algorithms/algorithm.py | 8 +- rllib/algorithms/dqn/dqn.py | 12 +- rllib/algorithms/ppo/ppo.py | 8 +- rllib/core/learner/learner_group.py | 4 +- .../examples/evaluation/custom_evaluation.py | 2 +- rllib/tuned_examples/dqn/cartpole_dqn.py | 3 +- rllib/tuned_examples/ppo/cartpole_ppo.py | 6 +- rllib/utils/metrics/metrics_logger.py | 131 ++++++++++- rllib/utils/metrics/stats.py | 222 ++++++++++++------ 10 files changed, 300 insertions(+), 98 deletions(-) diff --git a/rllib/BUILD b/rllib/BUILD index 097ad26ca80c..e6966fa341b6 100644 --- a/rllib/BUILD +++ b/rllib/BUILD @@ -376,7 +376,7 @@ py_test( py_test( name = "learning_tests_cartpole_ppo", main = "tuned_examples/ppo/cartpole_ppo.py", - tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "no_tf_static_graph"], + tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "torch_only"], size = "large", srcs = ["tuned_examples/ppo/cartpole_ppo.py"], args = ["--as-test", "--enable-new-api-stack"] diff --git a/rllib/algorithms/algorithm.py b/rllib/algorithms/algorithm.py index 5646fcd2eee1..6c4e771f0ac4 100644 --- a/rllib/algorithms/algorithm.py +++ b/rllib/algorithms/algorithm.py @@ -1304,7 +1304,7 @@ def _env_runner_remote(worker, num, round, iter): ) num_episodes = env_runner_results[NUM_EPISODES] else: - self.metrics.log_n_dicts( + self.metrics.merge_and_log_n_dicts( all_metrics, key=(EVALUATION_RESULTS, ENV_RUNNER_RESULTS), ) @@ -1492,7 +1492,7 @@ def _env_runner_remote(worker, num, round, iter): ) num_episodes = env_runner_results[NUM_EPISODES] else: - self.metrics.log_n_dicts( + self.metrics.merge_and_log_n_dicts( all_metrics, key=(EVALUATION_RESULTS, ENV_RUNNER_RESULTS), ) @@ -1633,7 +1633,7 @@ def training_step(self) -> ResultDict: train_batch = train_batch.as_multi_agent() # Reduce EnvRunner metrics over the n EnvRunners. - self.metrics.log_n_dicts(env_runner_results, key=ENV_RUNNER_RESULTS) + self.metrics.merge_and_log_n_dicts(env_runner_results, key=ENV_RUNNER_RESULTS) # Only train if train_batch is not empty. # In an extreme situation, all rollout workers die during the @@ -3327,7 +3327,7 @@ def _run_one_training_iteration_and_evaluation_in_parallel_wo_thread( NUM_ENV_STEPS_SAMPLED_THIS_ITER, 0 ) else: - self.metrics.log_n_dicts( + self.metrics.merge_and_log_n_dicts( all_metrics, key=(EVALUATION_RESULTS, ENV_RUNNER_RESULTS), ) diff --git a/rllib/algorithms/dqn/dqn.py b/rllib/algorithms/dqn/dqn.py index 2368b733376f..e7d61a666e1b 100644 --- a/rllib/algorithms/dqn/dqn.py +++ b/rllib/algorithms/dqn/dqn.py @@ -616,7 +616,9 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: # Add the sampled experiences to the replay buffer. self.local_replay_buffer.add(episodes) # Reduce EnvRunner metrics over the n EnvRunners. - self.metrics.log_n_dicts(env_runner_results, key=ENV_RUNNER_RESULTS) + self.metrics.merge_and_log_n_dicts( + env_runner_results, key=ENV_RUNNER_RESULTS + ) self.metrics.log_value( NUM_ENV_STEPS_SAMPLED_LIFETIME, @@ -683,7 +685,9 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: mid: {TD_ERROR_KEY: np.concatenate(s, axis=0)} for mid, s in td_errors.items() } - self.metrics.log_n_dicts(learner_results, key=LEARNER_RESULTS) + self.metrics.merge_and_log_n_dicts( + learner_results, key=LEARNER_RESULTS + ) self.metrics.log_value( NUM_ENV_STEPS_TRAINED_LIFETIME, self.metrics.peek( @@ -722,7 +726,9 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: timestep=current_ts, ) # log the additional results as well. - self.metrics.log_n_dicts(additional_results, key=LEARNER_RESULTS) + self.metrics.merge_and_log_n_dicts( + additional_results, key=LEARNER_RESULTS + ) # Update weights and global_vars - after learning on the local worker - # on all remote workers. diff --git a/rllib/algorithms/ppo/ppo.py b/rllib/algorithms/ppo/ppo.py index 0390e639b009..ce491f9ca09d 100644 --- a/rllib/algorithms/ppo/ppo.py +++ b/rllib/algorithms/ppo/ppo.py @@ -456,7 +456,9 @@ def _training_step_new_api_stack(self) -> ResultDict: return {} # Reduce EnvRunner metrics over the n EnvRunners. - self.metrics.log_n_dicts(env_runner_results, key=ENV_RUNNER_RESULTS) + self.metrics.merge_and_log_n_dicts( + env_runner_results, key=ENV_RUNNER_RESULTS + ) # Log lifetime counts for env- and agent steps. self.metrics.log_dict( { @@ -483,7 +485,7 @@ def _training_step_new_api_stack(self) -> ResultDict: ), num_iters=self.config.num_sgd_iter, ) - self.metrics.log_n_dicts(learner_results, key=LEARNER_RESULTS) + self.metrics.merge_and_log_n_dicts(learner_results, key=LEARNER_RESULTS) self.metrics.log_dict( { NUM_ENV_STEPS_TRAINED_LIFETIME: self.metrics.peek( @@ -546,7 +548,7 @@ def _training_step_new_api_stack(self) -> ResultDict: sampled_kl_values=kl_dict, timestep=self.metrics.peek(NUM_ENV_STEPS_SAMPLED_LIFETIME), ) - self.metrics.log_n_dicts(additional_results, key=LEARNER_RESULTS) + self.metrics.merge_and_log_n_dicts(additional_results, key=LEARNER_RESULTS) return self.metrics.reduce() diff --git a/rllib/core/learner/learner_group.py b/rllib/core/learner/learner_group.py index b955ec78d1f1..acc42ef83dea 100644 --- a/rllib/core/learner/learner_group.py +++ b/rllib/core/learner/learner_group.py @@ -478,7 +478,7 @@ def _learner_update( results = tree.flatten_up_to( [[None] * len(r) for r in results], results ) - self._metrics_logger_old_and_hybrid_stack.log_n_dicts(results) + self._metrics_logger_old_and_hybrid_stack.merge_and_log_n_dicts(results) results = self._metrics_logger_old_and_hybrid_stack.reduce( # We are returning to a client (Algorithm) that does NOT make any # use of MetricsLogger (or Stats) -> Convert all values to non-Stats @@ -577,7 +577,7 @@ def additional_update( # the existing behavior of returning an already reduced dict (as if we had a # reduce_fn). if not self.config.enable_env_runner_and_connector_v2: - self._metrics_logger_old_and_hybrid_stack.log_n_dicts(results) + self._metrics_logger_old_and_hybrid_stack.merge_and_log_n_dicts(results) results = self._metrics_logger_old_and_hybrid_stack.reduce( return_stats_obj=False ) diff --git a/rllib/examples/evaluation/custom_evaluation.py b/rllib/examples/evaluation/custom_evaluation.py index 8d8e869b1146..d396ffee04df 100644 --- a/rllib/examples/evaluation/custom_evaluation.py +++ b/rllib/examples/evaluation/custom_evaluation.py @@ -143,7 +143,7 @@ def custom_eval_function( # You can compute metrics from the episodes manually, or use the Algorithm's # convenient MetricsLogger to store all evaluation metrics inside the main # algo. - algorithm.metrics.log_n_dicts( + algorithm.metrics.merge_and_log_n_dicts( env_runner_metrics, key=(EVALUATION_RESULTS, ENV_RUNNER_RESULTS) ) eval_results = algorithm.metrics.reduce( diff --git a/rllib/tuned_examples/dqn/cartpole_dqn.py b/rllib/tuned_examples/dqn/cartpole_dqn.py index 95e580a53b51..4d1f6e07381f 100644 --- a/rllib/tuned_examples/dqn/cartpole_dqn.py +++ b/rllib/tuned_examples/dqn/cartpole_dqn.py @@ -24,7 +24,7 @@ # Settings identical to old stack. model_config_dict={ "fcnet_hiddens": [256], - "fcnet_activation": "relu", + "fcnet_activation": "tanh", "epsilon": [(0, 1.0), (10000, 0.02)], "fcnet_bias_initializer": "zeros_", "post_fcnet_bias_initializer": "zeros_", @@ -40,6 +40,7 @@ "alpha": 0.6, "beta": 0.4, }, + n_step=3, double_q=True, num_atoms=1, noisy=False, diff --git a/rllib/tuned_examples/ppo/cartpole_ppo.py b/rllib/tuned_examples/ppo/cartpole_ppo.py index 3865c9991652..92ec15da7bec 100644 --- a/rllib/tuned_examples/ppo/cartpole_ppo.py +++ b/rllib/tuned_examples/ppo/cartpole_ppo.py @@ -19,7 +19,6 @@ enable_rl_module_and_learner=True, enable_env_runner_and_connector_v2=True, ) - .env_runners(num_env_runners=1) .environment("CartPole-v1") .rl_module( model_config_dict={ @@ -39,12 +38,13 @@ evaluation_num_env_runners=1, evaluation_interval=1, evaluation_parallel_to_training=True, + evaluation_config=PPOConfig.overrides(exploration=False), ) ) stop = { - f"{NUM_ENV_STEPS_SAMPLED_LIFETIME}": 100000, - f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 150.0, + f"{NUM_ENV_STEPS_SAMPLED_LIFETIME}": 200000, + f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 350.0, } diff --git a/rllib/utils/metrics/metrics_logger.py b/rllib/utils/metrics/metrics_logger.py index a329fd86fe0d..8de81deb0038 100644 --- a/rllib/utils/metrics/metrics_logger.py +++ b/rllib/utils/metrics/metrics_logger.py @@ -1,9 +1,10 @@ import logging -from typing import Any, Dict, Optional, Tuple, Union +from typing import Any, Dict, List, Optional, Tuple, Union import tree # pip install dm_tree from ray.rllib.utils import force_tuple +from ray.rllib.utils.deprecation import Deprecated from ray.rllib.utils.metrics.stats import Stats from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.util.annotations import PublicAPI @@ -283,9 +284,9 @@ def _map(path, stat_or_value): tree.map_structure_with_path(_map, stats_dict) - def log_n_dicts( + def merge_and_log_n_dicts( self, - stats_dicts, + stats_dicts: List[Dict[str, Any]], *, key: Optional[Union[str, Tuple[str]]] = None, reduce: Optional[str] = "mean", @@ -293,11 +294,120 @@ def log_n_dicts( ema_coeff: Optional[float] = None, clear_on_reduce: bool = False, ) -> None: - """TODO (sven): docstr + """Merges n dicts, generated by n parallel components, and logs the results. + + .. testcode:: + + from ray.rllib.utils.metrics.metrics_logger import MetricsLogger + from ray.rllib.utils.test_utils import check + + # Example: n Learners logging loss stats to be merged. + # Note that losses should usually be logged with a window=1 so they don't + # get smeared over time and instead provide an accurate picture of the + # current situation. + main_logger = MetricsLogger() + + logger_learner1 = MetricsLogger() + logger_learner1.log_value("loss", 0.1, window=1) + learner1_results = logger_learner1.reduce() + + logger_learner2 = MetricsLogger() + logger_learner2.log_value("loss", 0.2, window=1) + learner2_results = logger_learner2.reduce() + + # Merge the stats from both Learners. + main_logger.merge_and_log_n_dicts( + [learner1_results, learner2_results], + key="learners", + ) + check(main_logger.peek("learners", "loss"), 0.15) + + # Example: m EnvRunners logging episode returns to be merged. + main_logger = MetricsLogger() + + logger_env_runner1 = MetricsLogger() + logger_env_runner1.log_value("mean_ret", 100.0, window=3) + logger_env_runner1.log_value("mean_ret", 200.0) + logger_env_runner1.log_value("mean_ret", 300.0) + logger_env_runner1.log_value("mean_ret", 400.0) + env_runner1_results = logger_env_runner1.reduce() + + logger_env_runner2 = MetricsLogger() + logger_env_runner2.log_value("mean_ret", 150.0, window=3) + logger_env_runner2.log_value("mean_ret", 250.0) + logger_env_runner2.log_value("mean_ret", 350.0) + logger_env_runner2.log_value("mean_ret", 450.0) + env_runner2_results = logger_env_runner2.reduce() + + # Merge the stats from both EnvRunners. + main_logger.merge_and_log_n_dicts( + [env_runner1_results, env_runner2_results], + key="env_runners", + ) + # The expected procedure is as follows: + # The individual internal values lists of the two loggers are as follows: + # env runner 1: [100, 200, 300, 400] + # env runner 2: [150, 250, 350, 450] + # Move backwards from index=-1 (each time, loop through both env runners) + # index=-1 -> [400, 450] -> reduce-mean -> [425] -> repeat 2 times (number + # of env runners) -> [425, 425] + # index=-2 -> [300, 350] -> reduce-mean -> [325] -> repeat 2 times + # -> append -> [425, 425, 325, 325] -> STOP b/c we have reached >= window. + # reverse the list -> [325, 325, 425, 425] + check( + main_logger.stats["env_runners"]["mean_ret"].values, + [325, 325, 425, 425], + ) + check(main_logger.peek("env_runners", "mean_ret"), (325 + 425 + 425) / 3) + + # Example: Lifetime sum over n parallel components' stats. + main_logger = MetricsLogger() + + logger1 = MetricsLogger() + logger1.log_value("some_stat", 50, reduce="sum", window=None) + logger1.log_value("some_stat", 25, reduce="sum", window=None) + logger1_results = logger1.reduce() + + logger2 = MetricsLogger() + logger2.log_value("some_stat", 75, reduce="sum", window=None) + logger2_results = logger2.reduce() + + # Merge the stats from both Learners. + main_logger.merge_and_log_n_dicts([logger1_results, logger2_results]) + check(main_logger.peek("some_stat"), 150) + + # Example: Sum over n parallel components' stats with a window of 3. + main_logger = MetricsLogger() + + logger1 = MetricsLogger() + logger1.log_value("some_stat", 50, reduce="sum", window=3) + logger1.log_value("some_stat", 25, reduce="sum") + logger1.log_value("some_stat", 10, reduce="sum") + logger1.log_value("some_stat", 5, reduce="sum") + logger1_results = logger1.reduce() + + logger2 = MetricsLogger() + logger2.log_value("some_stat", 75, reduce="sum", window=3) + logger2.log_value("some_stat", 100, reduce="sum") + logger2_results = logger2.reduce() + + # Merge the stats from both Learners. + main_logger.merge_and_log_n_dicts([logger1_results, logger2_results]) + # The expected procedure is as follows: + # The individual internal values lists of the two loggers are as follows: + # env runner 1: [50, 25, 10, 5] + # env runner 2: [75, 100] + # Move backwards from index=-1 (each time, loop through both loggers) + # index=-1 -> [5, 100] -> leave as-is, b/c we are sum'ing -> [5, 100] + # index=-2 -> [10, 75] -> leave as-is -> [5, 100, 10, 75] -> STOP b/c we + # have reached >= window. + # reverse the list -> [75, 10, 100, 5] + check(main_logger.peek("some_stat"), 115) # last 3 items (window) get sum'd Args: - stats_dicts: - key: + stats_dicts: List of n stats dicts to be merged and then logged. + key: Optional top-level key under which to log all keys/key sequences + found in the n `stats_dicts`. reduce: The reduction method to apply, once `self.reduce()` is called. If None, will collect all logged values under `key` in a list (and also return that list upon calling `self.reduce()`). @@ -637,7 +747,8 @@ def reduce( MetricsLogger object. Component A now calls its n remote components, each of which returns an equivalent, reduced dict with `Stats` as leafs. Component A can then further log these n result dicts via its own MetricsLogger: - `logger.log_n_dicts([n returned result dicts from the remote components])`. + `logger.merge_and_log_n_dicts([n returned result dicts from the remote + components])`. .. testcode:: @@ -679,7 +790,7 @@ def reduce( # Now combine the 2 equivalent results into 1 end result dict. downstream_logger = MetricsLogger() - downstream_logger.log_n_dicts([result1, result2]) + downstream_logger.merge_and_log_n_dicts([result1, result2]) # What happens internally is that both values lists of the 2 components # are merged (concat'd) and randomly shuffled, then clipped at 10 (window # size). This is done such that no component has an "advantage" over the @@ -782,3 +893,7 @@ def _set_key(self, flat_key, stats): if key not in _dict: _dict[key] = {} _dict = _dict[key] + + @Deprecated(new="MetricsLogger.merge_and_log_n_dicts()", error=True) + def log_n_dicts(self, *args, **kwargs): + pass diff --git a/rllib/utils/metrics/stats.py b/rllib/utils/metrics/stats.py index 49793f11f4c3..eec5845fd1a9 100644 --- a/rllib/utils/metrics/stats.py +++ b/rllib/utils/metrics/stats.py @@ -299,44 +299,78 @@ def merge_in_parallel(self, *others: "Stats") -> None: Thereby, the newly incoming values of `others` are treated equally with respect to each other as well as with respect to the internal values of self. - Use this method to merge another Stats into this one that resulted from a - parallel run and metrics logging of `self` and n `others` (for example, n - Learner workers all returning a loss value). + Use this method to merge other `Stats` objects, which resulted from some + parallelly executed components, into this one. For example: n Learner workers + all returning a loss value in the form of `{"total_loss": [some value]}`. - The following examples demonstrate the parallel merging logic: + The following examples demonstrate the parallel merging logic for different + reduce- and window settings: .. testcode:: - from ray.rllib.utils.metrics.stats import Stats from ray.rllib.utils.test_utils import check - # Parallel-merge two mean stats (win=3). + # Parallel-merge two (reduce=mean) stats with window=3. stats = Stats(reduce="mean", window=3) stats1 = Stats(reduce="mean", window=3) + stats1.push(0) stats1.push(1) stats1.push(2) stats1.push(3) stats2 = Stats(reduce="mean", window=3) - stats1.push(4) - stats1.push(5) - stats1.push(6) + stats2.push(4000) + stats2.push(4) + stats2.push(5) + stats2.push(6) stats.merge_in_parallel(stats1, stats2) - check(stats.values, [3.5, 3.5, 3.5]) - - # Parallel-merge two max stats (win=3). + # Fill new merged-values list: + # - Start with index -1, moving to the start. + # - Thereby always reducing across the different Stats objects' at the + # current index. + # - The resulting reduced value (across Stats at current index) is then + # repeated AND + # added to the new merged-values list n times (where n is the number of + # Stats, across + # which we merge). + # - The merged-values list is reversed. + # Here: + # index -1: [3, 6] -> [4.5, 4.5] + # index -2: [2, 5] -> [4.5, 4.5, 3.5, 3.5] + # STOP after merged list contains >= 3 items (window size) + # reverse: [3.5, 3.5, 4.5, 4.5] + check(stats.values, [3.5, 3.5, 4.5, 4.5]) + check(stats.peek(), (3.5 + 4.5 + 4.5) / 3) # mean last 3 items (window) + + # Parallel-merge two (reduce=max) stats with window=3. stats = Stats(reduce="max", window=3) stats1 = Stats(reduce="max", window=3) stats1.push(1) stats1.push(2) stats1.push(3) stats2 = Stats(reduce="max", window=3) - stats1.push(4) - stats1.push(5) - stats1.push(6) + stats2.push(4) + stats2.push(5) + stats2.push(6) stats.merge_in_parallel(stats1, stats2) - check(stats.values, [6]) - - # Parallel-merge two min stats (win=4). + # Same here: Fill new merged-values list: + # - Start with index -1, moving to the start. + # - Thereby always reducing across the different Stats objects' at the + # current index. + # - The resulting reduced value (across Stats at current index) is then + # repeated AND + # added to the new merged-values list n times (where n is the number of + # Stats, across + # which we merge). + # - The merged-values list is reversed. + # Here: + # index -1: [3, 6] -> [6, 6] + # index -2: [2, 5] -> [6, 6, 5, 5] + # STOP after merged list contains >= 3 items (window size) + # reverse: [5, 5, 6, 6] + check(stats.values, [5, 5, 6, 6]) + check(stats.peek(), 6) # max is 6 + + # Parallel-merge two (reduce=min) stats with window=4. stats = Stats(reduce="min", window=4) stats1 = Stats(reduce="min", window=4) stats1.push(1) @@ -344,34 +378,55 @@ def merge_in_parallel(self, *others: "Stats") -> None: stats1.push(1) stats1.push(4) stats2 = Stats(reduce="min", window=4) - stats1.push(5) - stats1.push(0.5) - stats1.push(7) - stats1.push(8) + stats2.push(5) + stats2.push(0.5) + stats2.push(7) + stats2.push(8) stats.merge_in_parallel(stats1, stats2) - check(stats.values, [0.5]) - - # Parallel-merge two sum stats (no window). + # Same procedure: + # index -1: [4, 8] -> [4, 4] + # index -2: [1, 7] -> [4, 4, 1, 1] + # STOP after merged list contains >= 4 items (window size) + # reverse: [1, 1, 4, 4] + check(stats.values, [1, 1, 4, 4]) + check(stats.peek(), 1) # min is 1 + + # Parallel-merge two (reduce=sum) stats with no window. + # Note that when reduce="sum", we do NOT reduce across the indices of the + # parallel stats = Stats(reduce="sum") stats1 = Stats(reduce="sum") stats1.push(1) stats1.push(2) + stats1.push(0) stats1.push(3) stats2 = Stats(reduce="sum") - stats1.push(4) - stats1.push(5) - stats1.push(6) + stats2.push(4) + stats2.push(5) + stats2.push(6) + # index -1: [3, 6] -> [3, 6] (no reduction, leave values as-is) + # index -2: [0, 5] -> [3, 6, 0, 5] + # index -3: [2, 4] -> [3, 6, 0, 5, 2, 4] + # index -4: [1] -> [3, 6, 0, 5, 2, 4, 1] + # STOP after merged list contains >= 4 items (window size) + # reverse: [1, 4, 2, 5, 0, 6, 3] stats.merge_in_parallel(stats1, stats2) - check(stats.values, [21]) + check(stats.values, [1, 4, 2, 5, 0, 6, 3]) + check(stats.peek(), 21) - # Parallel-merge two "concat" stats (reduce=None; no win). + # Parallel-merge two "concat" (reduce=None) stats with no window. + # Note that when reduce=None, we do NOT reduce across the indices of the + # parallel stats = Stats(reduce=None, window=float("inf"), clear_on_reduce=True) stats1 = Stats(reduce=None, window=float("inf"), clear_on_reduce=True) stats1.push(1) stats2 = Stats(reduce=None, window=float("inf"), clear_on_reduce=True) - stats1.push(2) + stats2.push(2) + # index -1: [1, 2] -> [1, 2] (no reduction, leave values as-is) + # reverse: [2, 1] stats.merge_in_parallel(stats1, stats2) - check(stats.values, [1, 2]) + check(stats.values, [2, 1]) + check(stats.peek(), [2, 1]) Args: others: One or more other Stats objects that need to be parallely merged @@ -383,21 +438,39 @@ def merge_in_parallel(self, *others: "Stats") -> None: assert all(self._window == o._window for o in others) assert all(self._ema_coeff == o._ema_coeff for o in others) - # Extend `self`'s values by all `others`' values. - for o in others: - self.values.extend(o.values) - - # Reduce over the entire values (no matter the window size!) first. - store_win = self._window - self._window = None - reduced_value, new_values = self._reduced_values() - self._window = store_win - # x-fold the reduced_value over the actual window size and use the resulting - # list as new self.values. - if self._reduce_method == "mean": - self.values = [reduced_value] * self._window - else: - self.values = new_values + win = self._window or float("inf") + + # Take turns stepping through `self` and `*others` values, thereby moving + # backwards from last index to beginning and will up the resulting values list. + # Stop as soon as we reach the window size. + new_values = [] + tmp_values = [] + # Loop from index=-1 backward to index=start until our new_values list has + # at least a len of `win`. + for i in range(1, max(map(len, [self, *others])) + 1): + # Per index, loop through all involved stats, including `self` and add + # to `tmp_values`. + for stats in [self, *others]: + if len(stats) < i: + continue + tmp_values.append(stats.values[-i]) + + # Now reduce across `tmp_values` based on the reduce-settings of this Stats. + # TODO (sven) : explain why all this + if self._ema_coeff is not None: + new_values.extend([np.nanmean(tmp_values)] * len(tmp_values)) + elif self._reduce_method in [None, "sum"]: + new_values.extend(tmp_values) + else: + new_values.extend( + [self._reduced_values(values=tmp_values, window=float("inf"))[0]] + * len(tmp_values) + ) + tmp_values.clear() + if len(new_values) >= win: + break + + self.values = list(reversed(new_values)) def numpy(self, value: Any = None) -> "Stats": """Converts all of self's internal values to numpy (if a tensor).""" @@ -491,39 +564,47 @@ def similar_to(other: "Stats", init_value: Optional[Any] = None): clear_on_reduce=other._clear_on_reduce, ) - def _reduced_values(self) -> Tuple[Any, Any]: - """Runs a non-commited reduction procedure on the internal values list. + def _reduced_values(self, values=None, window=None) -> Tuple[Any, Any]: + """Runs a non-commited reduction procedure on given values (or `self.values`). + + Note that this method does NOT alter any state of `self` or the possibly + provided list of `values`. It only returns new values as they should be + adopted after a possible, actual reduction step. + + Args: + values: The list of values to reduce. If not None, use `self.values` + window: A possible override window setting to use (instead of + `self._window`). Use float('inf') here for an infinite window size. Returns: A tuple containing 1) the reduced value and 2) the new internal values list to be used. """ + values = values if values is not None else self.values + window = window if window is not None else self._window + + # Apply the window (if provided and not inf). + values = ( + values if window is None or window == float("inf") else values[-window:] + ) + # No reduction method. Return list as-is OR reduce list to len=window. if self._reduce_method is None: - # No window -> return all internal values. - if self._window is None or self._window == float("inf"): - return self.values, self.values - # Window -> return shortened internal values list. - else: - return self.values[-self._window :], self.values[-self._window :] + return values, values # Special case: Internal values list is empty -> return NaN. - elif len(self.values) == 0: + elif len(values) == 0: return float("nan"), [] - # Do EMA (always a "mean" reduction). - elif self._ema_coeff is not None: + + # Do EMA (always a "mean" reduction; possibly using a window). + if self._ema_coeff is not None: # Perform EMA reduction over all values in internal values list. - mean_value = self.values[0] - for v in self.values[1:]: + mean_value = values[0] + for v in values[1:]: mean_value = self._ema_coeff * v + (1.0 - self._ema_coeff) * mean_value - return mean_value, [mean_value] + return mean_value, values # Do non-EMA reduction (possibly using a window). else: - values = ( - self.values - if self._window is None or self._window == float("inf") - else self.values[-self._window :] - ) # Use the numpy/torch "nan"-prefix to ignore NaN's in our value lists. if torch and torch.is_tensor(values[0]): reduce_meth = getattr(torch, "nan" + self._reduce_method) @@ -548,16 +629,13 @@ def _reduced_values(self) -> Tuple[Any, Any]: # For window=None|inf (infinite window) and reduce != mean, we don't have to # keep any values, except the last (reduced) one. - if ( - self._window is None or self._window == float("inf") - ) and self._reduce_method != "mean": + if window in [None, float("inf")] and self._reduce_method != "mean": # TODO (sven): What if out values are torch tensors? In this case, we # would have to do reduction using `torch` above (not numpy) and only # then return the python primitive AND put the reduced new torch # tensor in `new_values`. - new_values = [reduced] + return reduced, [reduced] # In all other cases, keep the values that were also used for the reduce # operation. else: - new_values = values - return reduced, new_values + return reduced, values From ce47bcaf2e263a069df706d7481eb29a4e3b9b9d Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Tue, 28 May 2024 12:00:49 -0700 Subject: [PATCH 29/65] [ci][doc/01] move module walking class to ray_ci (#45527) Move module walking class and the logic to find class/function api to ray_ci, so that it can be tested. The `Module` class walks through all its children attribute, check if an attribute is an API (by checking a special field name) and record if it is. Test: - CI --------- Signed-off-by: can --- ci/ray_ci/doc/BUILD.bazel | 30 ++++++++++++++ ci/ray_ci/doc/api.py | 21 ++++++++++ ci/ray_ci/doc/mock_module.py | 42 +++++++++++++++++++ ci/ray_ci/doc/module.py | 78 ++++++++++++++++++++++++++++++++++++ ci/ray_ci/doc/test_module.py | 20 +++++++++ 5 files changed, 191 insertions(+) create mode 100644 ci/ray_ci/doc/BUILD.bazel create mode 100644 ci/ray_ci/doc/api.py create mode 100644 ci/ray_ci/doc/mock_module.py create mode 100644 ci/ray_ci/doc/module.py create mode 100644 ci/ray_ci/doc/test_module.py diff --git a/ci/ray_ci/doc/BUILD.bazel b/ci/ray_ci/doc/BUILD.bazel new file mode 100644 index 000000000000..85feb3222ad9 --- /dev/null +++ b/ci/ray_ci/doc/BUILD.bazel @@ -0,0 +1,30 @@ +load("@rules_python//python:defs.bzl", "py_library", "py_test") +load("@py_deps_buildkite//:requirements.bzl", ci_require = "requirement") + +py_library( + name = "doc", + srcs = glob( + ["*.py"], + exclude = [ + "test_*.py", + "cmd_*.py", + "mock_module.py", + ], + ), + visibility = ["//ci/ray_ci/doc:__subpackages__"], +) + +py_test( + name = "test_module", + size = "small", + srcs = ["test_module.py", "mock_module.py"], + exec_compatible_with = ["//:hermetic_python"], + tags = [ + "ci_unit", + "team:ci", + ], + deps = [ + ":doc", + ci_require("pytest"), + ], +) diff --git a/ci/ray_ci/doc/api.py b/ci/ray_ci/doc/api.py new file mode 100644 index 000000000000..25996fa7d436 --- /dev/null +++ b/ci/ray_ci/doc/api.py @@ -0,0 +1,21 @@ +from enum import Enum +from dataclasses import dataclass + + +class AnnotationType(Enum): + PUBLIC_API = "PublicAPI" + DEVELOPER_API = "DeveloperAPI" + DEPRECATED = "Deprecated" + UNKNOWN = "Unknown" + + +class CodeType(Enum): + CLASS = "Class" + FUNCTION = "Function" + + +@dataclass +class API: + name: str + annotation_type: AnnotationType + code_type: CodeType diff --git a/ci/ray_ci/doc/mock_module.py b/ci/ray_ci/doc/mock_module.py new file mode 100644 index 000000000000..39328ba83419 --- /dev/null +++ b/ci/ray_ci/doc/mock_module.py @@ -0,0 +1,42 @@ +from ci.ray_ci.doc.api import AnnotationType + + +def PublicAPI(*args, **kwargs): + if len(args) == 1 and len(kwargs) == 0 and callable(args[0]): + return PublicAPI()(args[0]) + + def wrap(obj): + obj._annotated = None + obj._annotated_type = AnnotationType.PUBLIC_API + return obj + + return wrap + + +def Deprecated(*args, **kwargs): + if len(args) == 1 and len(kwargs) == 0 and callable(args[0]): + return Deprecated()(args[0]) + + def wrap(obj): + obj._annotated = None + obj._annotated_type = AnnotationType.DEPRECATED + return obj + + return wrap + + +@PublicAPI +class MockClass: + """ + This class is used for testing purpose only. It should not be used in production. + """ + + pass + + +@Deprecated +def mock_function(): + """ + This function is used for testing purpose only. It should not be used in production. + """ + pass diff --git a/ci/ray_ci/doc/module.py b/ci/ray_ci/doc/module.py new file mode 100644 index 000000000000..deacb3d28b55 --- /dev/null +++ b/ci/ray_ci/doc/module.py @@ -0,0 +1,78 @@ +import importlib +import inspect +from types import ModuleType +from typing import List + +from ci.ray_ci.doc.api import API, AnnotationType, CodeType + + +class Module: + def __init__(self, module: str): + self._module = importlib.import_module(module) + self._visited = set() + self._apis = [] + + def walk(self) -> None: + self._walk(self._module) + + def get_apis(self) -> List[API]: + self.walk() + return self._apis + + def _walk(self, module: ModuleType) -> None: + """ + Depth-first search through the module and its children to find annotated classes + and functions. + """ + if module in self._visited: + return + self._visited.add(module.__hash__) + + if not self._is_valid_child(module): + return + + for child in dir(module): + attribute = getattr(module, child) + + if inspect.ismodule(attribute): + self._walk(attribute) + if inspect.isclass(attribute): + if self._is_api(attribute): + self._apis.append( + API( + name=self._fullname(attribute), + annotation_type=self._get_annotation_type(attribute), + code_type=CodeType.CLASS, + ) + ) + self._walk(attribute) + if inspect.isfunction(attribute): + if self._is_api(attribute): + self._apis.append( + API( + name=self._fullname(attribute), + annotation_type=self._get_annotation_type(attribute), + code_type=CodeType.FUNCTION, + ) + ) + + return + + def _fullname(self, module: ModuleType) -> str: + return f"{module.__module__}.{module.__qualname__}" + + def _is_valid_child(self, module: ModuleType) -> bool: + """ + This module is a valid child of the top level module if it is the top level + module itself, or its module name starts with the top level module name. + """ + module = inspect.getmodule(module) + if not hasattr(module, "__name__"): + return False + return module.__name__.startswith(self._module.__name__) + + def _is_api(self, module: ModuleType) -> bool: + return self._is_valid_child(module) and hasattr(module, "_annotated") + + def _get_annotation_type(self, module: ModuleType) -> AnnotationType: + return AnnotationType(module._annotated_type.value) diff --git a/ci/ray_ci/doc/test_module.py b/ci/ray_ci/doc/test_module.py new file mode 100644 index 000000000000..a55e41c2fd25 --- /dev/null +++ b/ci/ray_ci/doc/test_module.py @@ -0,0 +1,20 @@ +import sys +import pytest + +from ci.ray_ci.doc.module import Module +from ci.ray_ci.doc.api import AnnotationType, CodeType + + +def test_walk(): + module = Module("ci.ray_ci.doc.mock_module") + apis = module.get_apis() + assert apis[0].name == "ci.ray_ci.doc.mock_module.MockClass" + assert apis[0].annotation_type.value == AnnotationType.PUBLIC_API.value + assert apis[0].code_type.value == CodeType.CLASS.value + assert apis[1].name == "ci.ray_ci.doc.mock_module.mock_function" + assert apis[1].annotation_type.value == AnnotationType.DEPRECATED.value + assert apis[1].code_type.value == CodeType.FUNCTION.value + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) From c3e6ecab361eef84f3ce3c8940ec413c88486db0 Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Tue, 28 May 2024 12:16:29 -0700 Subject: [PATCH 30/65] [Data] Add `metadata` and `block_refs` properties to `RefBundle` (#45567) RefBundle stores data as a List[Tuple[ObjectRef, BlockMetadata]]. Often, we'll need to access either just the object references (List[ObjectRef]) or the metadata (List[BlockMetadata]). To avoid boilerplate code to access this data, this PR adds properties to separately access the object references and metadata. Signed-off-by: Balaji Veeramani --- .../interfaces/op_runtime_metrics.py | 6 ++-- .../execution/interfaces/ref_bundle.py | 28 +++++++++++++------ .../data/_internal/execution/legacy_compat.py | 5 ++-- .../execution/operators/input_data_buffer.py | 2 +- .../execution/operators/map_operator.py | 3 +- .../operators/task_pool_map_operator.py | 3 +- .../ray/data/_internal/planner/aggregate.py | 5 ++-- .../pull_based_shuffle_task_scheduler.py | 3 +- .../push_based_shuffle_task_scheduler.py | 3 +- .../_internal/planner/randomize_blocks.py | 3 +- python/ray/data/_internal/planner/sort.py | 5 ++-- python/ray/data/tests/test_operators.py | 22 ++++++++------- python/ray/data/tests/test_split.py | 2 +- .../data/tests/test_streaming_integration.py | 8 +++--- 14 files changed, 51 insertions(+), 47 deletions(-) diff --git a/python/ray/data/_internal/execution/interfaces/op_runtime_metrics.py b/python/ray/data/_internal/execution/interfaces/op_runtime_metrics.py index 5ea4f4cdc71e..51b7ab9f30e3 100644 --- a/python/ray/data/_internal/execution/interfaces/op_runtime_metrics.py +++ b/python/ray/data/_internal/execution/interfaces/op_runtime_metrics.py @@ -512,12 +512,10 @@ def on_task_finished(self, task_index: int, exception: Optional[Exception]): input_size, ) - blocks = [input[0] for input in inputs.blocks] - metadata = [input[1] for input in inputs.blocks] ctx = ray.data.context.DataContext.get_current() if ctx.enable_get_object_locations_for_metrics: - locations = ray.experimental.get_object_locations(blocks) - for block, meta in zip(blocks, metadata): + locations = ray.experimental.get_object_locations(inputs.block_refs) + for block, meta in inputs.blocks: if locations[block].get("did_spill", False): assert meta.size_bytes is not None self.obj_store_mem_spilled += meta.size_bytes diff --git a/python/ray/data/_internal/execution/interfaces/ref_bundle.py b/python/ray/data/_internal/execution/interfaces/ref_bundle.py index 6304f48432a8..c8996a1c422b 100644 --- a/python/ray/data/_internal/execution/interfaces/ref_bundle.py +++ b/python/ray/data/_internal/execution/interfaces/ref_bundle.py @@ -1,5 +1,5 @@ from dataclasses import dataclass -from typing import Optional, Tuple +from typing import List, Optional, Tuple import ray from .common import NodeIdStr @@ -58,19 +58,29 @@ def __setattr__(self, key, value): raise ValueError(f"The `{key}` field of RefBundle cannot be updated.") object.__setattr__(self, key, value) + @property + def block_refs(self) -> List[BlockMetadata]: + """List of block references in this bundle.""" + return [block_ref for block_ref, _ in self.blocks] + + @property + def metadata(self) -> List[BlockMetadata]: + """List of block metadata in this bundle.""" + return [metadata for _, metadata in self.blocks] + def num_rows(self) -> Optional[int]: """Number of rows present in this bundle, if known.""" total = 0 - for b in self.blocks: - if b[1].num_rows is None: + for m in self.metadata: + if m.num_rows is None: return None else: - total += b[1].num_rows + total += m.num_rows return total def size_bytes(self) -> int: """Size of the blocks of this bundle in bytes.""" - return sum(b[1].size_bytes for b in self.blocks) + return sum(m.size_bytes for m in self.metadata) def destroy_if_owned(self) -> int: """Clears the object store memory for these blocks if owned. @@ -79,8 +89,10 @@ def destroy_if_owned(self) -> int: The number of bytes freed. """ should_free = self.owns_blocks and DataContext.get_current().eager_free - for b in self.blocks: - trace_deallocation(b[0], "RefBundle.destroy_if_owned", free=should_free) + for block_ref in self.block_refs: + trace_deallocation( + block_ref, "RefBundle.destroy_if_owned", free=should_free + ) return self.size_bytes() if should_free else 0 def get_cached_location(self) -> Optional[NodeIdStr]: @@ -91,7 +103,7 @@ def get_cached_location(self) -> Optional[NodeIdStr]: if self._cached_location is None: # Only consider the first block in the bundle for now. TODO(ekl) consider # taking into account other blocks. - ref = self.blocks[0][0] + ref = self.block_refs[0] # This call is pretty fast for owned objects (~5k/s), so we don't need to # batch it for now. locs = ray.experimental.get_object_locations([ref]) diff --git a/python/ray/data/_internal/execution/legacy_compat.py b/python/ray/data/_internal/execution/legacy_compat.py index c216617eeb0f..aa4e7ba64ee5 100644 --- a/python/ray/data/_internal/execution/legacy_compat.py +++ b/python/ray/data/_internal/execution/legacy_compat.py @@ -204,9 +204,8 @@ def _bundles_to_block_list(bundles: Iterator[RefBundle]) -> BlockList: for ref_bundle in bundles: if not ref_bundle.owns_blocks: owns_blocks = False - for block, meta in ref_bundle.blocks: - blocks.append(block) - metadata.append(meta) + blocks.extend(ref_bundle.block_refs) + metadata.extend(ref_bundle.metadata) return BlockList(blocks, metadata, owned_by_consumer=owns_blocks) diff --git a/python/ray/data/_internal/execution/operators/input_data_buffer.py b/python/ray/data/_internal/execution/operators/input_data_buffer.py index ba781b660713..cc0447d8d26e 100644 --- a/python/ray/data/_internal/execution/operators/input_data_buffer.py +++ b/python/ray/data/_internal/execution/operators/input_data_buffer.py @@ -80,7 +80,7 @@ def _initialize_metadata(self): self._num_output_bundles = len(self._input_data) block_metadata = [] for bundle in self._input_data: - block_metadata.extend([m for (_, m) in bundle.blocks]) + block_metadata.extend(bundle.metadata) self._stats = { "input": block_metadata, } diff --git a/python/ray/data/_internal/execution/operators/map_operator.py b/python/ray/data/_internal/execution/operators/map_operator.py index 075e28bc6c1c..931dbd4aeb7c 100644 --- a/python/ray/data/_internal/execution/operators/map_operator.py +++ b/python/ray/data/_internal/execution/operators/map_operator.py @@ -370,8 +370,7 @@ def _get_next_inner(self) -> RefBundle: assert self._started bundle = self._output_queue.get_next() self._metrics.on_output_dequeued(bundle) - for _, meta in bundle.blocks: - self._output_metadata.append(meta) + self._output_metadata.extend(bundle.metadata) return bundle @abstractmethod diff --git a/python/ray/data/_internal/execution/operators/task_pool_map_operator.py b/python/ray/data/_internal/execution/operators/task_pool_map_operator.py index 8f4fc7fee2b8..42fa68030f48 100644 --- a/python/ray/data/_internal/execution/operators/task_pool_map_operator.py +++ b/python/ray/data/_internal/execution/operators/task_pool_map_operator.py @@ -63,7 +63,6 @@ def __init__( def _add_bundled_input(self, bundle: RefBundle): # Submit the task as a normal Ray task. map_task = cached_remote_fn(_map_task, num_returns="streaming") - input_blocks = [block for block, _ in bundle.blocks] ctx = TaskContext( task_idx=self._next_data_task_idx, target_max_block_size=self.actual_target_max_block_size, @@ -84,7 +83,7 @@ def _add_bundled_input(self, bundle: RefBundle): self._map_transformer_ref, data_context, ctx, - *input_blocks, + *bundle.block_refs, ) self._submit_data_task(gen, bundle) diff --git a/python/ray/data/_internal/planner/aggregate.py b/python/ray/data/_internal/planner/aggregate.py index 6cd86ad3938d..d31afc5a7c00 100644 --- a/python/ray/data/_internal/planner/aggregate.py +++ b/python/ray/data/_internal/planner/aggregate.py @@ -39,9 +39,8 @@ def fn( blocks = [] metadata = [] for ref_bundle in refs: - for block, block_metadata in ref_bundle.blocks: - blocks.append(block) - metadata.append(block_metadata) + blocks.extend(ref_bundle.block_refs) + metadata.extend(ref_bundle.metadata) if len(blocks) == 0: return (blocks, {}) unified_schema = unify_block_metadata_schema(metadata) diff --git a/python/ray/data/_internal/planner/exchange/pull_based_shuffle_task_scheduler.py b/python/ray/data/_internal/planner/exchange/pull_based_shuffle_task_scheduler.py index 1176202b9758..b2cf448f030d 100644 --- a/python/ray/data/_internal/planner/exchange/pull_based_shuffle_task_scheduler.py +++ b/python/ray/data/_internal/planner/exchange/pull_based_shuffle_task_scheduler.py @@ -41,8 +41,7 @@ def execute( # eagerly release the blocks' memory. input_blocks_list = [] for ref_bundle in refs: - for block, _ in ref_bundle.blocks: - input_blocks_list.append(block) + input_blocks_list.extend(ref_bundle.block_refs) input_num_blocks = len(input_blocks_list) input_owned = all(b.owns_blocks for b in refs) diff --git a/python/ray/data/_internal/planner/exchange/push_based_shuffle_task_scheduler.py b/python/ray/data/_internal/planner/exchange/push_based_shuffle_task_scheduler.py index 2c5130fe8933..cafd5f31f4da 100644 --- a/python/ray/data/_internal/planner/exchange/push_based_shuffle_task_scheduler.py +++ b/python/ray/data/_internal/planner/exchange/push_based_shuffle_task_scheduler.py @@ -457,8 +457,7 @@ def execute( # processed concurrently. input_blocks_list = [] for ref_bundle in refs: - for block, _ in ref_bundle.blocks: - input_blocks_list.append(block) + input_blocks_list.extend(ref_bundle.block_refs) input_owned = all(b.owns_blocks for b in refs) if map_ray_remote_args is None: diff --git a/python/ray/data/_internal/planner/randomize_blocks.py b/python/ray/data/_internal/planner/randomize_blocks.py index b2fc9c950c0e..835017f2cafd 100644 --- a/python/ray/data/_internal/planner/randomize_blocks.py +++ b/python/ray/data/_internal/planner/randomize_blocks.py @@ -22,8 +22,7 @@ def fn( nonlocal op blocks_with_metadata = [] for ref_bundle in refs: - for block, meta in ref_bundle.blocks: - blocks_with_metadata.append((block, meta)) + blocks_with_metadata.extend(ref_bundle.blocks) if len(blocks_with_metadata) == 0: return refs, {op._name: []} diff --git a/python/ray/data/_internal/planner/sort.py b/python/ray/data/_internal/planner/sort.py index fddb80a72dde..3ca5af144006 100644 --- a/python/ray/data/_internal/planner/sort.py +++ b/python/ray/data/_internal/planner/sort.py @@ -32,9 +32,8 @@ def fn( blocks = [] metadata = [] for ref_bundle in refs: - for block, block_metadata in ref_bundle.blocks: - blocks.append(block) - metadata.append(block_metadata) + blocks.extend(ref_bundle.block_refs) + metadata.extend(ref_bundle.metadata) if len(blocks) == 0: return (blocks, {}) sort_key.validate_schema(unify_block_metadata_schema(metadata)) diff --git a/python/ray/data/tests/test_operators.py b/python/ray/data/tests/test_operators.py index b48d94c79fb7..54ea30c17234 100644 --- a/python/ray/data/tests/test_operators.py +++ b/python/ray/data/tests/test_operators.py @@ -44,8 +44,8 @@ def _get_blocks(bundle: RefBundle, output_list: List[Block]): - for block, _ in bundle.blocks: - output_list.append(list(ray.get(block)["id"])) + for block_ref in bundle.block_refs: + output_list.append(list(ray.get(block_ref)["id"])) def _mul2_transform(block_iter: Iterable[Block], ctx) -> Iterable[Block]: @@ -196,9 +196,11 @@ def test_split_operator(ray_start_regular_shared, equal, chunk_size): while op.has_next(): ref = op.get_next() assert ref.owns_blocks, ref - for block, _ in ref.blocks: + for block_ref in ref.block_refs: assert ref.output_split_idx is not None - output_splits[ref.output_split_idx].extend(list(ray.get(block)["id"])) + output_splits[ref.output_split_idx].extend( + list(ray.get(block_ref)["id"]) + ) op.all_inputs_done() expected_splits = [[] for _ in range(num_splits)] @@ -234,8 +236,8 @@ def test_split_operator_random(ray_start_regular_shared, equal, random_seed): while op.has_next(): ref = op.get_next() assert ref.owns_blocks, ref - for block, _ in ref.blocks: - output_splits[ref.output_split_idx].extend(list(ray.get(block)["id"])) + for block_ref in ref.block_refs: + output_splits[ref.output_split_idx].extend(list(ray.get(block_ref)["id"])) if equal: actual = [len(output_splits[i]) for i in range(3)] expected = [num_inputs // 3] * 3 @@ -271,8 +273,8 @@ def get_bundle_loc(bundle): while op.has_next(): ref = op.get_next() assert ref.owns_blocks, ref - for block, _ in ref.blocks: - output_splits[ref.output_split_idx].extend(list(ray.get(block)["id"])) + for block_ref in ref.block_refs: + output_splits[ref.output_split_idx].extend(list(ray.get(block_ref)["id"])) total = 0 for i in range(2): @@ -583,8 +585,8 @@ def test_limit_operator(ray_start_regular_shared): def _get_bundles(bundle: RefBundle): output = [] - for block, _ in bundle.blocks: - output.extend(list(ray.get(block)["id"])) + for block_ref in bundle.block_refs: + output.extend(list(ray.get(block_ref)["id"])) return output diff --git a/python/ray/data/tests/test_split.py b/python/ray/data/tests/test_split.py index 8fe481338baf..db729ba93f24 100644 --- a/python/ray/data/tests/test_split.py +++ b/python/ray/data/tests/test_split.py @@ -667,7 +667,7 @@ def equalize_helper(input_block_lists: List[List[List[Any]]]): result_block_lists = [] for bundle in result: block_list = [] - for block_ref, _ in bundle.blocks: + for block_ref in bundle.block_refs: block = ray.get(block_ref) block_accessor = BlockAccessor.for_block(block) block_list.append(list(block_accessor.to_default()["id"])) diff --git a/python/ray/data/tests/test_streaming_integration.py b/python/ray/data/tests/test_streaming_integration.py index a317af32ef83..9f2672123c06 100644 --- a/python/ray/data/tests/test_streaming_integration.py +++ b/python/ray/data/tests/test_streaming_integration.py @@ -43,8 +43,8 @@ def map_fn(block_iter, _): def ref_bundles_to_list(bundles: List[RefBundle]) -> List[List[Any]]: output = [] for bundle in bundles: - for block, _ in bundle.blocks: - output.append(list(ray.get(block)["id"])) + for block_ref in bundle.block_refs: + output.append(list(ray.get(block_ref)["id"])) return output @@ -144,8 +144,8 @@ def run(self): def get_outputs(out: List[RefBundle]): outputs = [] for bundle in out: - for block, _ in bundle.blocks: - ids: pd.Series = ray.get(block)["id"] + for block_ref in bundle.block_refs: + ids: pd.Series = ray.get(block_ref)["id"] outputs.extend(ids.values) return outputs From 73f6fb26034251e39ed0aad1765443b8f2df99a0 Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Tue, 28 May 2024 12:16:51 -0700 Subject: [PATCH 31/65] [Data] Remove dead code from `ray.data._internal.compute` (#45564) Signed-off-by: Balaji Veeramani --- python/ray/data/_internal/compute.py | 129 +-------------------------- 1 file changed, 2 insertions(+), 127 deletions(-) diff --git a/python/ray/data/_internal/compute.py b/python/ray/data/_internal/compute.py index 10a477589922..c5b1612ffe3f 100644 --- a/python/ray/data/_internal/compute.py +++ b/python/ray/data/_internal/compute.py @@ -1,19 +1,8 @@ import logging -import math -from typing import Any, Callable, Iterable, List, Optional, Tuple, TypeVar, Union +from typing import Any, Callable, Iterable, Optional, TypeVar, Union -from ray.data._internal.delegating_block_builder import DelegatingBlockBuilder from ray.data._internal.execution.interfaces import TaskContext -from ray.data.block import ( - Block, - BlockAccessor, - BlockExecStats, - BlockMetadata, - BlockPartition, - UserDefinedFunction, -) -from ray.data.context import DataContext -from ray.types import ObjectRef +from ray.data.block import Block, UserDefinedFunction from ray.util.annotations import DeveloperAPI, PublicAPI logger = logging.getLogger(__name__) @@ -161,117 +150,3 @@ def is_task_compute(compute_spec: Union[str, ComputeStrategy]) -> bool: or compute_spec == "tasks" or isinstance(compute_spec, TaskPoolStrategy) ) - - -def _map_block_split( - block_fn: BlockTransform, - input_files: List[str], - fn: Optional[UserDefinedFunction], - num_blocks: int, - *blocks_and_fn_args: Union[Block, Any], - **fn_kwargs, -) -> BlockPartition: - stats = BlockExecStats.builder() - blocks, fn_args = blocks_and_fn_args[:num_blocks], blocks_and_fn_args[num_blocks:] - if fn is not None: - fn_args = (fn,) + fn_args - new_metas = [] - for new_block in block_fn(blocks, *fn_args, **fn_kwargs): - accessor = BlockAccessor.for_block(new_block) - new_meta = BlockMetadata( - num_rows=accessor.num_rows(), - size_bytes=accessor.size_bytes(), - schema=accessor.schema(), - input_files=input_files, - exec_stats=stats.build(), - ) - yield new_block - new_metas.append(new_meta) - stats = BlockExecStats.builder() - yield new_metas - - -def _map_block_nosplit( - block_fn: BlockTransform, - input_files: List[str], - fn: Optional[UserDefinedFunction], - num_blocks: int, - *blocks_and_fn_args: Union[Block, Any], - **fn_kwargs, -) -> Tuple[Block, BlockMetadata]: - stats = BlockExecStats.builder() - builder = DelegatingBlockBuilder() - blocks, fn_args = blocks_and_fn_args[:num_blocks], blocks_and_fn_args[num_blocks:] - if fn is not None: - fn_args = (fn,) + fn_args - for new_block in block_fn(blocks, *fn_args, **fn_kwargs): - builder.add_block(new_block) - new_block = builder.build() - accessor = BlockAccessor.for_block(new_block) - return new_block, accessor.get_metadata( - input_files=input_files, exec_stats=stats.build() - ) - - -def _bundle_blocks_up_to_size( - blocks: List[Tuple[ObjectRef[Block], BlockMetadata]], - target_size: int, -) -> List[Tuple[Tuple[ObjectRef[Block]], Tuple[BlockMetadata]]]: - """Group blocks into bundles that are up to (but not exceeding) the provided target - size. - """ - block_bundles: List[List[Tuple[ObjectRef[Block], BlockMetadata]]] = [] - curr_bundle: List[Tuple[ObjectRef[Block], BlockMetadata]] = [] - curr_bundle_size = 0 - for b, m in blocks: - num_rows = m.num_rows - if num_rows is None: - num_rows = float("inf") - if curr_bundle_size > 0 and curr_bundle_size + num_rows > target_size: - block_bundles.append(curr_bundle) - curr_bundle = [] - curr_bundle_size = 0 - curr_bundle.append((b, m)) - curr_bundle_size += num_rows - if curr_bundle: - block_bundles.append(curr_bundle) - if len(blocks) / len(block_bundles) >= 10: - logger.warning( - f"`batch_size` is set to {target_size}, which reduces parallelism from " - f"{len(blocks)} to {len(block_bundles)}. If the performance is worse than " - "expected, this may indicate that the batch size is too large or the " - "input block size is too small. To reduce batch size, consider decreasing " - "`batch_size` or use the default in `map_batches`. To increase input " - "block size, consider decreasing `parallelism` in read." - ) - return [tuple(zip(*block_bundle)) for block_bundle in block_bundles] - - -def _check_batch_size( - blocks_and_meta: List[Tuple[ObjectRef[Block], BlockMetadata]], - batch_size: int, - name: str, -): - """Log a warning if the provided batch size exceeds the configured target max block - size. - """ - batch_size_bytes = None - for _, meta in blocks_and_meta: - if meta.num_rows and meta.size_bytes: - batch_size_bytes = math.ceil(batch_size * (meta.size_bytes / meta.num_rows)) - break - context = DataContext.get_current() - if ( - batch_size_bytes is not None - and batch_size_bytes > context.target_max_block_size - ): - logger.warning( - f"Requested batch size {batch_size} results in batches of " - f"{batch_size_bytes} bytes for {name} tasks, which is larger than the " - f"configured target max block size {context.target_max_block_size}. This " - "may result in out-of-memory errors for certain workloads, and you may " - "want to decrease your batch size or increase the configured target max " - "block size, e.g.: " - "from ray.data.context import DataContext; " - "DataContext.get_current().target_max_block_size = 4_000_000_000" - ) From b54ee6863812c65c9a5a886d8054e7534f4c1e5e Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Tue, 28 May 2024 12:17:48 -0700 Subject: [PATCH 32/65] [Data] Add default parameter values for `BlockAccessor.get_metadata` (#45566) BlockAccessor.get_metadata has two required arguments: input_files and exec_stats. However, we usually pass None to these arguments. To simplify the code, this PR makes the arguments optional and uses None as the default value. Signed-off-by: Balaji Veeramani --- python/ray/data/_internal/arrow_block.py | 4 ++-- .../_internal/execution/operators/map_operator.py | 2 +- .../_internal/execution/operators/zip_operator.py | 2 +- python/ray/data/_internal/execution/util.py | 2 +- python/ray/data/_internal/pandas_block.py | 8 ++------ .../planner/exchange/aggregate_task_spec.py | 4 +--- .../_internal/planner/exchange/sort_task_spec.py | 4 +--- .../exchange/split_repartition_task_scheduler.py | 2 +- python/ray/data/_internal/util.py | 12 +++--------- python/ray/data/block.py | 4 +++- python/ray/data/read_api.py | 9 ++------- python/ray/data/tests/test_split.py | 6 ++---- 12 files changed, 20 insertions(+), 39 deletions(-) diff --git a/python/ray/data/_internal/arrow_block.py b/python/ray/data/_internal/arrow_block.py index b05de9ed9344..dba739f75644 100644 --- a/python/ray/data/_internal/arrow_block.py +++ b/python/ray/data/_internal/arrow_block.py @@ -559,7 +559,7 @@ def merge_sorted_blocks( blocks = TableBlockAccessor.normalize_block_types(blocks, "arrow") concat_and_sort = get_concat_and_sort_transform(DataContext.get_current()) ret = concat_and_sort(blocks, sort_key) - return ret, ArrowBlockAccessor(ret).get_metadata(None, exec_stats=stats.build()) + return ret, ArrowBlockAccessor(ret).get_metadata(exec_stats=stats.build()) @staticmethod def aggregate_combined_blocks( @@ -672,7 +672,7 @@ def gen(): break ret = builder.build() - return ret, ArrowBlockAccessor(ret).get_metadata(None, exec_stats=stats.build()) + return ret, ArrowBlockAccessor(ret).get_metadata(exec_stats=stats.build()) def block_type(self) -> BlockType: return BlockType.ARROW diff --git a/python/ray/data/_internal/execution/operators/map_operator.py b/python/ray/data/_internal/execution/operators/map_operator.py index 931dbd4aeb7c..0c8023ef5397 100644 --- a/python/ray/data/_internal/execution/operators/map_operator.py +++ b/python/ray/data/_internal/execution/operators/map_operator.py @@ -425,7 +425,7 @@ def _map_task( map_transformer.set_target_max_block_size(ctx.target_max_block_size) for b_out in map_transformer.apply_transform(iter(blocks), ctx): # TODO(Clark): Add input file propagation from input blocks. - m_out = BlockAccessor.for_block(b_out).get_metadata([], None) + m_out = BlockAccessor.for_block(b_out).get_metadata() m_out.exec_stats = stats.build() m_out.exec_stats.udf_time_s = map_transformer.udf_time() m_out.exec_stats.task_idx = ctx.task_idx diff --git a/python/ray/data/_internal/execution/operators/zip_operator.py b/python/ray/data/_internal/execution/operators/zip_operator.py index a0092d93b62d..d93e29d6ac84 100644 --- a/python/ray/data/_internal/execution/operators/zip_operator.py +++ b/python/ray/data/_internal/execution/operators/zip_operator.py @@ -240,7 +240,7 @@ def _zip_one_block( # Zip block and other blocks. result = BlockAccessor.for_block(block).zip(other_block) br = BlockAccessor.for_block(result) - return result, br.get_metadata(input_files=[], exec_stats=stats.build()) + return result, br.get_metadata(exec_stats=stats.build()) def _get_num_rows_and_bytes(block: Block) -> Tuple[int, int]: diff --git a/python/ray/data/_internal/execution/util.py b/python/ray/data/_internal/execution/util.py index 2e10895d5168..d3bf3d9f1f54 100644 --- a/python/ray/data/_internal/execution/util.py +++ b/python/ray/data/_internal/execution/util.py @@ -25,7 +25,7 @@ def make_ref_bundles(simple_data: List[List[Any]]) -> List["RefBundle"]: [ ( ray.put(block), - BlockAccessor.for_block(block).get_metadata([], None), + BlockAccessor.for_block(block).get_metadata(), ) ], owns_blocks=True, diff --git a/python/ray/data/_internal/pandas_block.py b/python/ray/data/_internal/pandas_block.py index 599ebff222c9..3d40c97221a1 100644 --- a/python/ray/data/_internal/pandas_block.py +++ b/python/ray/data/_internal/pandas_block.py @@ -514,9 +514,7 @@ def merge_sorted_blocks( ret = pd.concat(blocks, ignore_index=True) columns, ascending = sort_key.to_pandas_sort_args() ret = ret.sort_values(by=columns, ascending=ascending) - return ret, PandasBlockAccessor(ret).get_metadata( - None, exec_stats=stats.build() - ) + return ret, PandasBlockAccessor(ret).get_metadata(exec_stats=stats.build()) @staticmethod def aggregate_combined_blocks( @@ -628,9 +626,7 @@ def gen(): break ret = builder.build() - return ret, PandasBlockAccessor(ret).get_metadata( - None, exec_stats=stats.build() - ) + return ret, PandasBlockAccessor(ret).get_metadata(exec_stats=stats.build()) def block_type(self) -> BlockType: return BlockType.PANDAS diff --git a/python/ray/data/_internal/planner/exchange/aggregate_task_spec.py b/python/ray/data/_internal/planner/exchange/aggregate_task_spec.py index be621a958914..45f813ada13a 100644 --- a/python/ray/data/_internal/planner/exchange/aggregate_task_spec.py +++ b/python/ray/data/_internal/planner/exchange/aggregate_task_spec.py @@ -55,9 +55,7 @@ def map( SortKey(key), ) parts = [BlockAccessor.for_block(p).combine(key, aggs) for p in partitions] - meta = BlockAccessor.for_block(block).get_metadata( - input_files=None, exec_stats=stats.build() - ) + meta = BlockAccessor.for_block(block).get_metadata(exec_stats=stats.build()) return parts + [meta] @staticmethod diff --git a/python/ray/data/_internal/planner/exchange/sort_task_spec.py b/python/ray/data/_internal/planner/exchange/sort_task_spec.py index e6cbc8aaf72f..d4b8f0b04391 100644 --- a/python/ray/data/_internal/planner/exchange/sort_task_spec.py +++ b/python/ray/data/_internal/planner/exchange/sort_task_spec.py @@ -131,9 +131,7 @@ def map( ) -> List[Union[BlockMetadata, Block]]: stats = BlockExecStats.builder() out = BlockAccessor.for_block(block).sort_and_partition(boundaries, sort_key) - meta = BlockAccessor.for_block(block).get_metadata( - input_files=None, exec_stats=stats.build() - ) + meta = BlockAccessor.for_block(block).get_metadata(exec_stats=stats.build()) return out + [meta] @staticmethod diff --git a/python/ray/data/_internal/planner/exchange/split_repartition_task_scheduler.py b/python/ray/data/_internal/planner/exchange/split_repartition_task_scheduler.py index 946634d3d0d2..cc21f699667b 100644 --- a/python/ray/data/_internal/planner/exchange/split_repartition_task_scheduler.py +++ b/python/ray/data/_internal/planner/exchange/split_repartition_task_scheduler.py @@ -117,7 +117,7 @@ def execute( builder = PandasBlockBuilder() empty_block = builder.build() empty_meta = BlockAccessor.for_block(empty_block).get_metadata( - input_files=None, exec_stats=None + exec_stats=None ) # No stats for empty block. empty_block_refs, empty_metadata = zip( *[(ray.put(empty_block), empty_meta) for _ in range(num_empty_blocks)] diff --git a/python/ray/data/_internal/util.py b/python/ray/data/_internal/util.py index c91f34f1214b..8aa0c95fa50c 100644 --- a/python/ray/data/_internal/util.py +++ b/python/ray/data/_internal/util.py @@ -653,9 +653,7 @@ def pandas_df_to_arrow_block(df: "pandas.DataFrame") -> "Block": stats = BlockExecStats.builder() return ( block, - BlockAccessor.for_block(block).get_metadata( - input_files=None, exec_stats=stats.build() - ), + BlockAccessor.for_block(block).get_metadata(exec_stats=stats.build()), ) @@ -666,9 +664,7 @@ def ndarray_to_block(ndarray: np.ndarray, ctx: DataContext) -> "Block": stats = BlockExecStats.builder() block = BlockAccessor.batch_to_block({"data": ndarray}) - metadata = BlockAccessor.for_block(block).get_metadata( - input_files=None, exec_stats=stats.build() - ) + metadata = BlockAccessor.for_block(block).get_metadata(exec_stats=stats.build()) return block, metadata @@ -678,9 +674,7 @@ def get_table_block_metadata( from ray.data.block import BlockAccessor, BlockExecStats stats = BlockExecStats.builder() - return BlockAccessor.for_block(table).get_metadata( - input_files=None, exec_stats=stats.build() - ) + return BlockAccessor.for_block(table).get_metadata(exec_stats=stats.build()) def unify_block_metadata_schema( diff --git a/python/ray/data/block.py b/python/ray/data/block.py index 1ad73760b6c7..1f8c156c8578 100644 --- a/python/ray/data/block.py +++ b/python/ray/data/block.py @@ -331,7 +331,9 @@ def schema(self) -> Union[type, "pyarrow.lib.Schema"]: raise NotImplementedError def get_metadata( - self, input_files: List[str], exec_stats: Optional[BlockExecStats] + self, + input_files: Optional[List[str]] = None, + exec_stats: Optional[BlockExecStats] = None, ) -> BlockMetadata: """Create a metadata object from this block.""" return BlockMetadata( diff --git a/python/ray/data/read_api.py b/python/ray/data/read_api.py index d30045cb3085..d3343ccd16f7 100644 --- a/python/ray/data/read_api.py +++ b/python/ray/data/read_api.py @@ -122,10 +122,7 @@ def from_blocks(blocks: List[Block]): A :class:`~ray.data.Dataset` holding the blocks. """ block_refs = [ray.put(block) for block in blocks] - metadata = [ - BlockAccessor.for_block(block).get_metadata(input_files=None, exec_stats=None) - for block in blocks - ] + metadata = [BlockAccessor.for_block(block).get_metadata() for block in blocks] from_blocks_op = FromBlocks(block_refs, metadata) logical_plan = LogicalPlan(from_blocks_op) return MaterializedDataset( @@ -207,9 +204,7 @@ def from_items( block = builder.build() blocks.append(ray.put(block)) metadata.append( - BlockAccessor.for_block(block).get_metadata( - input_files=None, exec_stats=stats.build() - ) + BlockAccessor.for_block(block).get_metadata(exec_stats=stats.build()) ) from_items_op = FromItems(blocks, metadata) diff --git a/python/ray/data/tests/test_split.py b/python/ray/data/tests/test_split.py index db729ba93f24..c9cf6f7f537b 100644 --- a/python/ray/data/tests/test_split.py +++ b/python/ray/data/tests/test_split.py @@ -93,7 +93,7 @@ def _test_equal_split_balanced(block_sizes, num_splits): for block_size in block_sizes: block = pd.DataFrame({"id": list(range(total_rows, total_rows + block_size))}) blocks.append(ray.put(block)) - metadata.append(BlockAccessor.for_block(block).get_metadata(None, None)) + metadata.append(BlockAccessor.for_block(block).get_metadata()) blk = (blocks[-1], metadata[-1]) ref_bundles.append(RefBundle((blk,), owns_blocks=True)) total_rows += block_size @@ -501,9 +501,7 @@ def _create_meta(num_rows): def _create_block_and_metadata(data: Any) -> Tuple[ObjectRef[Block], BlockMetadata]: block = pd.DataFrame({"id": data}) - metadata = BlockAccessor.for_block(block).get_metadata( - input_files=[], exec_stats=None - ) + metadata = BlockAccessor.for_block(block).get_metadata() return (ray.put(block), metadata) From bc76b19c312780c5540483063a8f08628f1500a9 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Tue, 28 May 2024 12:47:22 -0700 Subject: [PATCH 33/65] [ci][doc/03] add a command to query and validate apis for ray packages (#45531) As title, a command to: - Query apis for a given ray package - Print out the information for these APIs Test: - CI --------- Signed-off-by: can --- ci/lint/check_api_discrepancy.py | 73 ---------------------- ci/lint/lint.sh | 3 +- ci/ray_ci/doc/BUILD.bazel | 6 ++ ci/ray_ci/doc/cmd_check_api_discrepancy.py | 23 +++++++ 4 files changed, 31 insertions(+), 74 deletions(-) delete mode 100755 ci/lint/check_api_discrepancy.py create mode 100644 ci/ray_ci/doc/cmd_check_api_discrepancy.py diff --git a/ci/lint/check_api_discrepancy.py b/ci/lint/check_api_discrepancy.py deleted file mode 100755 index 2ecc0db68ae9..000000000000 --- a/ci/lint/check_api_discrepancy.py +++ /dev/null @@ -1,73 +0,0 @@ -#!/usr/bin/env python - -import click -import inspect -from typing import Dict, Set - -import ray -from ray.util.annotations import _is_annotated, _get_annotation_type - - -VISITED = set() -ANNOTATED_CLASSES = {} -ANNOTATED_FUNCTIONS = {} - - -@click.command() -def main() -> None: - """ - This script checks for annotated classes and functions in a module, and finds - discrepancies between the annotations and the documentation. - """ - walk(ray.data) # TODO(can): parameterize the module to walk - _print(ANNOTATED_CLASSES, "classes") - _print(ANNOTATED_FUNCTIONS, "functions") - - -def _print(annotated: Dict[str, Set[str]], prefix: str) -> None: - for type, objs in annotated.items(): - print(f"{prefix} {type}".upper()) - for obj in sorted(objs): - print(obj) - print("\n") - - -def walk(module) -> None: - """ - Depth-first search through the module and its children to find annotated classes - and functions. - """ - if module in VISITED: - return - VISITED.add(module) - if not _fullname(module).startswith("ray.data"): - return - - for child in dir(module): - attribute = getattr(module, child) - - if inspect.ismodule(attribute): - walk(attribute) - if inspect.isclass(attribute): - _add_if_annotated(attribute, ANNOTATED_CLASSES) - walk(attribute) - if inspect.isfunction(attribute) and _is_annotated(attribute): - _add_if_annotated(attribute, ANNOTATED_FUNCTIONS) - return - - -def _add_if_annotated(attribute, result: Dict[str, Set[str]]): - if not _is_annotated(attribute): - return - type = _get_annotation_type(attribute) - if type not in result: - result[type] = set() - result[type].add(_fullname(attribute)) - - -def _fullname(attr): - return inspect.getmodule(attr).__name__ + "." + attr.__name__ - - -if __name__ == "__main__": - main() diff --git a/ci/lint/lint.sh b/ci/lint/lint.sh index 0a3403e899fd..c17766d105e9 100755 --- a/ci/lint/lint.sh +++ b/ci/lint/lint.sh @@ -64,7 +64,8 @@ api_annotations() { api_discrepancy() { # shellcheck disable=SC2102 RAY_DISABLE_EXTRA_CPP=1 pip install -e python/[all] - ./ci/lint/check_api_discrepancy.py + # TODO(can): run this check with other ray packages + bazel run //ci/ray_ci/doc:cmd_check_api_discrepancy -- ray.data } documentation_style() { diff --git a/ci/ray_ci/doc/BUILD.bazel b/ci/ray_ci/doc/BUILD.bazel index 85feb3222ad9..8c34c44acf15 100644 --- a/ci/ray_ci/doc/BUILD.bazel +++ b/ci/ray_ci/doc/BUILD.bazel @@ -1,6 +1,12 @@ load("@rules_python//python:defs.bzl", "py_library", "py_test") load("@py_deps_buildkite//:requirements.bzl", ci_require = "requirement") +py_binary( + name = "cmd_check_api_discrepancy", + srcs = ["cmd_check_api_discrepancy.py"], + deps = [":doc"], +) + py_library( name = "doc", srcs = glob( diff --git a/ci/ray_ci/doc/cmd_check_api_discrepancy.py b/ci/ray_ci/doc/cmd_check_api_discrepancy.py new file mode 100644 index 000000000000..e55f2364ea61 --- /dev/null +++ b/ci/ray_ci/doc/cmd_check_api_discrepancy.py @@ -0,0 +1,23 @@ +import click + +from ci.ray_ci.doc.module import Module + + +@click.command() +@click.argument("module", required=True, type=str) +def main(module: str) -> None: + """ + This script checks for annotated classes and functions in a module, and finds + discrepancies between the annotations and the documentation. + """ + module = Module(module) + for api in module.get_apis(): + print( + f"API: {api.name}, " + f"Annotation Type: {api.annotation_type}, " + f"Code Type: {api.code_type}" + ) + + +if __name__ == "__main__": + main() From ce5a90f2fd0213ec2dbe0122a35a5660cb42e24e Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Tue, 28 May 2024 23:39:35 +0200 Subject: [PATCH 34/65] [RLlib] Fix frame stacking connector for experiments with more than 1 Learner. (#45588) --- rllib/algorithms/algorithm.py | 14 ++++ rllib/algorithms/dqn/dqn.py | 19 ++++-- rllib/algorithms/sac/sac.py | 18 +++-- rllib/env/multi_agent_env_runner.py | 2 +- rllib/env/multi_agent_episode.py | 9 ++- rllib/env/single_agent_episode.py | 2 + rllib/examples/connectors/frame_stacking.py | 31 +++++---- .../envs/classes/debug_counter_env.py | 23 +++++-- rllib/tuned_examples/dqn/cartpole_dqn.py | 7 +- .../dqn/multi_agent_cartpole_dqn.py | 66 +++++++++++++++++++ .../sac/multi_agent_pendulum_sac.py | 44 ++++++------- rllib/tuned_examples/sac/pendulum_sac.py | 42 +++++------- .../multi_agent_episode_replay_buffer.py | 55 +++++++--------- 13 files changed, 211 insertions(+), 121 deletions(-) create mode 100644 rllib/tuned_examples/dqn/multi_agent_cartpole_dqn.py diff --git a/rllib/algorithms/algorithm.py b/rllib/algorithms/algorithm.py index 6c4e771f0ac4..6b71fd59dd0c 100644 --- a/rllib/algorithms/algorithm.py +++ b/rllib/algorithms/algorithm.py @@ -3413,6 +3413,20 @@ def _compile_iteration_results_new_api_stack( ): # Return dict (shallow copy of `train_results`). results: ResultDict = train_results.copy() + + # TODO (sven): Fix Tune, instead, to be tolerant against possibly missing result + # keys. Otherwise, we'll have to guess here, what "popular" keys users use in + # order to protect them from running into Tune KeyErrors. + if ENV_RUNNER_RESULTS not in results: + results[ENV_RUNNER_RESULTS] = {} + for must_have in [ + EPISODE_RETURN_MEAN, + EPISODE_RETURN_MIN, + EPISODE_RETURN_MAX, + ]: + if must_have not in results[ENV_RUNNER_RESULTS]: + results[ENV_RUNNER_RESULTS][must_have] = np.nan + # Evaluation results. if eval_results: results.update(eval_results) diff --git a/rllib/algorithms/dqn/dqn.py b/rllib/algorithms/dqn/dqn.py index e7d61a666e1b..2674f99dbbd5 100644 --- a/rllib/algorithms/dqn/dqn.py +++ b/rllib/algorithms/dqn/dqn.py @@ -11,7 +11,7 @@ from collections import defaultdict import logging -from typing import Any, Callable, Dict, List, Optional, Type, Union +from typing import Any, Callable, Dict, List, Optional, Tuple, Type, Union import numpy as np from ray.rllib.algorithms.algorithm import Algorithm @@ -127,7 +127,7 @@ def __init__(self, algo_class=None): # Overrides of AlgorithmConfig defaults # `env_runners()` # Set to `self.n_step`, if 'auto'. - self.rollout_fragment_length = "auto" + self.rollout_fragment_length: Union[int, str] = "auto" self.exploration_config = { "type": "EpsilonGreedy", "initial_epsilon": 1.0, @@ -235,7 +235,7 @@ def training( dueling: Optional[bool] = NotProvided, hiddens: Optional[int] = NotProvided, double_q: Optional[bool] = NotProvided, - n_step: Optional[int] = NotProvided, + n_step: Optional[Union[int, Tuple[int, int]]] = NotProvided, before_learn_on_batch: Callable[ [Type[MultiAgentBatch], List[Type[Policy]], Type[int]], Type[MultiAgentBatch], @@ -311,7 +311,12 @@ def training( hiddens: Dense-layer setup for each the advantage branch and the value branch double_q: Whether to use double DQN. - n_step: N-step for Q-learning. + n_step: N-step target updates. If >1, sars' tuples in trajectories will be + postprocessed to become sa[discounted sum of R][s t+n] tuples. An + integer will be interpreted as a fixed n-step value. If a tuple of 2 + ints is provided here, the n-step value will be drawn for each sample(!) + in the train batch from a uniform distribution over the closed interval + defined by `[n_step[0], n_step[1]]`. before_learn_on_batch: Callback to run before learning on a multi-agent batch of experiences. training_intensity: The intensity with which to update the model (vs @@ -477,7 +482,11 @@ def validate(self) -> None: @override(AlgorithmConfig) def get_rollout_fragment_length(self, worker_index: int = 0) -> int: if self.rollout_fragment_length == "auto": - return self.n_step + return ( + self.n_step[1] + if isinstance(self.n_step, (tuple, list)) + else self.n_step + ) else: return self.rollout_fragment_length diff --git a/rllib/algorithms/sac/sac.py b/rllib/algorithms/sac/sac.py index 8ab0c89cf3ad..fed9ae2c02ef 100644 --- a/rllib/algorithms/sac/sac.py +++ b/rllib/algorithms/sac/sac.py @@ -1,5 +1,5 @@ import logging -from typing import Any, Dict, Optional, Type, Union +from typing import Any, Dict, Optional, Tuple, Type, Union from ray.rllib.algorithms.algorithm_config import AlgorithmConfig, NotProvided from ray.rllib.algorithms.dqn.dqn import DQN @@ -86,6 +86,7 @@ def __init__(self, algo_class=None): self.target_network_update_freq = 0 # .env_runners() + # Set to `self.n_step`, if 'auto'. self.rollout_fragment_length = "auto" self.compress_observations = False self.exploration_config = { @@ -127,7 +128,7 @@ def training( tau: Optional[float] = NotProvided, initial_alpha: Optional[float] = NotProvided, target_entropy: Optional[Union[str, float]] = NotProvided, - n_step: Optional[int] = NotProvided, + n_step: Optional[Union[int, Tuple[int, int]]] = NotProvided, store_buffer_in_checkpoints: Optional[bool] = NotProvided, replay_buffer_config: Optional[Dict[str, Any]] = NotProvided, training_intensity: Optional[float] = NotProvided, @@ -169,9 +170,10 @@ def training( automatically. n_step: N-step target updates. If >1, sars' tuples in trajectories will be postprocessed to become sa[discounted sum of R][s t+n] tuples. An - integer will be interpreted as a fixed n-step value. In case of a tuple - the n-step value will be drawn for each sample in the train batch from - a uniform distribution over the interval defined by the 'n-step'-tuple. + integer will be interpreted as a fixed n-step value. If a tuple of 2 + ints is provided here, the n-step value will be drawn for each sample(!) + in the train batch from a uniform distribution over the closed interval + defined by `[n_step[0], n_step[1]]`. store_buffer_in_checkpoints: Set this to True, if you want the contents of your buffer(s) to be stored in any saved checkpoints as well. Warnings will be created if: @@ -362,7 +364,11 @@ def validate(self) -> None: @override(AlgorithmConfig) def get_rollout_fragment_length(self, worker_index: int = 0) -> int: if self.rollout_fragment_length == "auto": - return self.n_step[1] if isinstance(self.n_step, tuple) else self.n_step + return ( + self.n_step[1] + if isinstance(self.n_step, (tuple, list)) + else self.n_step + ) else: return self.rollout_fragment_length diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index 857b56081fd1..077adbdad589 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -139,7 +139,7 @@ def sample( assert not (num_timesteps is not None and num_episodes is not None) # If no execution details are provided, use the config to try to infer the - # desired timesteps/episodes to sample and exploration behavior. + # desired timesteps/episodes to sample and the exploration behavior. if explore is None: explore = self.config.explore if num_timesteps is None and num_episodes is None: diff --git a/rllib/env/multi_agent_episode.py b/rllib/env/multi_agent_episode.py index a59cec2d7a63..5ce2f44ca2b3 100644 --- a/rllib/env/multi_agent_episode.py +++ b/rllib/env/multi_agent_episode.py @@ -873,7 +873,12 @@ def concat_episode(self, other: "MultiAgentEpisode") -> None: ) # Concatenate the env- to agent-timestep mappings. - self.env_t_to_agent_t[agent_id].extend(other.env_t_to_agent_t[agent_id]) + j = self.env_t + for i, val in enumerate(other.env_t_to_agent_t[agent_id][1:]): + if val == self.SKIP_ENV_TS_TAG: + self.env_t_to_agent_t[agent_id].append(self.SKIP_ENV_TS_TAG) + else: + self.env_t_to_agent_t[agent_id].append(i + 1 + j) # Otherwise, the agent is only in `self` and not done. All data is stored # already -> skip @@ -1883,7 +1888,7 @@ def get_agents_to_act(self) -> Set[AgentID]: `env.step()` call. Returns: - A set of AgentIDs that are suposed to send actions to the next `env.step()` + A set of AgentIDs that are supposed to send actions to the next `env.step()` call. """ return { diff --git a/rllib/env/single_agent_episode.py b/rllib/env/single_agent_episode.py index 1fdb8c0ca707..19eeeb046e9a 100644 --- a/rllib/env/single_agent_episode.py +++ b/rllib/env/single_agent_episode.py @@ -1508,6 +1508,7 @@ def slice(self, slice_: slice) -> "SingleAgentEpisode": lookback=self.observations.lookback, space=self.observation_space, ), + observation_space=self.observation_space, infos=InfiniteLookbackBuffer( data=self.get_infos( slice(start - self.infos.lookback, stop + 1, step), @@ -1523,6 +1524,7 @@ def slice(self, slice_: slice) -> "SingleAgentEpisode": lookback=self.actions.lookback, space=self.action_space, ), + action_space=self.action_space, rewards=InfiniteLookbackBuffer( data=self.get_rewards( slice(start - self.rewards.lookback, stop, step), diff --git a/rllib/examples/connectors/frame_stacking.py b/rllib/examples/connectors/frame_stacking.py index 9c316e591d3a..6abce5582b0b 100644 --- a/rllib/examples/connectors/frame_stacking.py +++ b/rllib/examples/connectors/frame_stacking.py @@ -1,15 +1,3 @@ -import gymnasium as gym - -from ray.rllib.connectors.env_to_module.frame_stacking import FrameStackingEnvToModule -from ray.rllib.connectors.learner.frame_stacking import FrameStackingLearner -from ray.rllib.env.wrappers.atari_wrappers import wrap_atari_for_new_api_stack -from ray.rllib.examples.envs.classes.multi_agent import make_multi_agent -from ray.rllib.utils.test_utils import ( - add_rllib_example_script_args, - run_rllib_example_script_experiment, -) -from ray.tune.registry import get_trainable_cls - """ Example using connectors (V2) for frame-stacking in Atari environments. How to run this script @@ -23,6 +11,17 @@ `--wandb-key=[your WandB API key] --wandb-project=[some project name] --wandb-run-name=[optional: WandB run name (within the defined project)]` """ +import gymnasium as gym + +from ray.rllib.connectors.env_to_module.frame_stacking import FrameStackingEnvToModule +from ray.rllib.connectors.learner.frame_stacking import FrameStackingLearner +from ray.rllib.env.wrappers.atari_wrappers import wrap_atari_for_new_api_stack +from ray.rllib.examples.envs.classes.multi_agent import make_multi_agent +from ray.rllib.utils.test_utils import ( + add_rllib_example_script_args, + run_rllib_example_script_experiment, +) +from ray.tune.registry import get_trainable_cls # Read in common example script command line arguments. parser = add_rllib_example_script_args( @@ -138,7 +137,7 @@ def _env_creator(cfg): ) ) if args.enable_new_api_stack: - base_config = base_config.rl_module( + base_config.rl_module( model_config_dict=dict( { "vf_share_layers": True, @@ -147,11 +146,11 @@ def _env_creator(cfg): "post_fcnet_hiddens": [256], "uses_new_env_runners": True, }, - ), + ) ) else: - base_config = base_config.training( - { + base_config.training( + model={ "vf_share_layers": True, "conv_filters": [[16, 4, 2], [32, 4, 2], [64, 4, 2], [128, 4, 2]], "conv_activation": "relu", diff --git a/rllib/examples/envs/classes/debug_counter_env.py b/rllib/examples/envs/classes/debug_counter_env.py index 24d95e56b3bf..69dc0870f62a 100644 --- a/rllib/examples/envs/classes/debug_counter_env.py +++ b/rllib/examples/envs/classes/debug_counter_env.py @@ -38,15 +38,30 @@ def __init__(self, config): super().__init__() self.num_agents = config["num_agents"] self.base_episode_len = config.get("base_episode_len", 103) - # Actions are always: - # (episodeID, envID) as floats. - self.action_space = gym.spaces.Box(-float("inf"), float("inf"), shape=(2,)) + # Observation dims: # 0=agent ID. # 1=episode ID (0.0 for obs after reset). # 2=env ID (0.0 for obs after reset). # 3=ts (of the agent). - self.observation_space = gym.spaces.Box(float("-inf"), float("inf"), (4,)) + self.observation_space = gym.spaces.Dict( + { + aid: gym.spaces.Box(float("-inf"), float("inf"), (4,)) + for aid in range(self.num_agents) + } + ) + self._obs_space_in_preferred_format = True + + # Actions are always: + # (episodeID, envID) as floats. + self.action_space = gym.spaces.Dict( + { + aid: gym.spaces.Box(-float("inf"), float("inf"), shape=(2,)) + for aid in range(self.num_agents) + } + ) + self._action_space_in_preferred_format = True + self.timesteps = [0] * self.num_agents self.terminateds = set() self.truncateds = set() diff --git a/rllib/tuned_examples/dqn/cartpole_dqn.py b/rllib/tuned_examples/dqn/cartpole_dqn.py index 4d1f6e07381f..6f64fba05a6f 100644 --- a/rllib/tuned_examples/dqn/cartpole_dqn.py +++ b/rllib/tuned_examples/dqn/cartpole_dqn.py @@ -15,11 +15,6 @@ config = ( DQNConfig() .environment(env="CartPole-v1") - .framework(framework="torch") - .api_stack( - enable_rl_module_and_learner=True, - enable_env_runner_and_connector_v2=True, - ) .rl_module( # Settings identical to old stack. model_config_dict={ @@ -63,7 +58,7 @@ ) stop = { - f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 450.0, + f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 500.0, NUM_ENV_STEPS_SAMPLED_LIFETIME: 100000, } diff --git a/rllib/tuned_examples/dqn/multi_agent_cartpole_dqn.py b/rllib/tuned_examples/dqn/multi_agent_cartpole_dqn.py new file mode 100644 index 000000000000..f66b0eb1b35d --- /dev/null +++ b/rllib/tuned_examples/dqn/multi_agent_cartpole_dqn.py @@ -0,0 +1,66 @@ +from ray.rllib.algorithms.dqn import DQNConfig +from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole +from ray.rllib.utils.metrics import ( + ENV_RUNNER_RESULTS, + EPISODE_RETURN_MEAN, + NUM_ENV_STEPS_SAMPLED_LIFETIME, +) +from ray.tune.registry import register_env + +from ray.rllib.utils.test_utils import add_rllib_example_script_args + +parser = add_rllib_example_script_args() +parser.set_defaults(num_agents=2) +# Use `parser` to add your own custom command line options to this script +# and (if needed) use their values to set up `config` below. +args = parser.parse_args() + +register_env( + "multi_agent_cartpole", + lambda _: MultiAgentCartPole({"num_agents": args.num_agents}), +) + +config = ( + DQNConfig() + .environment(env="multi_agent_cartpole") + .training( + # Settings identical to old stack. + train_batch_size_per_learner=32, + replay_buffer_config={ + "type": "MultiAgentEpisodeReplayBuffer", + "capacity": 50000, + }, + n_step=3, + double_q=True, + num_atoms=1, + noisy=False, + dueling=True, + ) + .rl_module( + model_config_dict={ + "fcnet_hiddens": [256], + "fcnet_activation": "tanh", + "epsilon": [(0, 1.0), (10000, 0.02)], + "fcnet_bias_initializer": "zeros_", + "post_fcnet_bias_initializer": "zeros_", + "post_fcnet_hiddens": [256], + }, + ) +) + +if args.num_agents: + config.multi_agent( + policy_mapping_fn=lambda aid, *arg, **kw: f"p{aid}", + policies={f"p{i}" for i in range(args.num_agents)}, + ) + +stop = { + NUM_ENV_STEPS_SAMPLED_LIFETIME: 500000, + # `episode_return_mean` is the sum of all agents/policies' returns. + f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 250.0 * args.num_agents, +} + +if __name__ == "__main__": + from ray.rllib.utils.test_utils import run_rllib_example_script_experiment + + run_rllib_example_script_experiment(config, args, stop=stop) diff --git a/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py b/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py index a70ca0b9d62b..077ad94baec5 100644 --- a/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py +++ b/rllib/tuned_examples/sac/multi_agent_pendulum_sac.py @@ -1,46 +1,31 @@ from ray.rllib.algorithms.sac import SACConfig from ray.rllib.examples.envs.classes.multi_agent import MultiAgentPendulum +from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.metrics import ( ENV_RUNNER_RESULTS, EPISODE_RETURN_MEAN, NUM_ENV_STEPS_SAMPLED_LIFETIME, ) +from ray.rllib.utils.test_utils import add_rllib_example_script_args from ray.tune.registry import register_env -from ray.rllib.utils.test_utils import add_rllib_example_script_args +torch, nn = try_import_torch() parser = add_rllib_example_script_args() +parser.set_defaults(num_agents=2) # Use `parser` to add your own custom command line options to this script # and (if needed) use their values to set up `config` below. args = parser.parse_args() register_env( "multi_agent_pendulum", - lambda _: MultiAgentPendulum({"num_agents": args.num_agents or 2}), + lambda _: MultiAgentPendulum({"num_agents": args.num_agents}), ) config = ( SACConfig() .environment(env="multi_agent_pendulum") - .rl_module( - model_config_dict={ - "fcnet_hiddens": [256, 256], - "fcnet_activation": "relu", - "post_fcnet_hiddens": [], - "post_fcnet_activation": None, - "post_fcnet_weights_initializer": "orthogonal_", - "post_fcnet_weights_initializer_config": {"gain": 0.01}, - } - ) - .api_stack( - enable_rl_module_and_learner=True, - enable_env_runner_and_connector_v2=True, - ) - .env_runners( - rollout_fragment_length=1, - num_env_runners=2, - num_envs_per_env_runner=1, - ) + .env_runners(num_env_runners=2) .training( initial_alpha=1.001, lr=3e-4, @@ -53,7 +38,18 @@ "type": "MultiAgentEpisodeReplayBuffer", "capacity": 100000, }, - num_steps_sampled_before_learning_starts=256, + num_steps_sampled_before_learning_starts=1024, + ) + .rl_module( + model_config_dict={ + "fcnet_hiddens": [256, 256], + "fcnet_activation": "tanh", + "fcnet_weights_initializer": nn.init.xavier_uniform_, + # "post_fcnet_hiddens": [], + # "post_fcnet_activation": None, + # "post_fcnet_weights_initializer": nn.init.orthogonal_, + # "post_fcnet_weights_initializer_config": {"gain": 0.01}, + } ) .reporting( metrics_num_episodes_for_smoothing=5, @@ -61,7 +57,7 @@ ) ) -if args.num_agents: +if args.num_agents > 0: config.multi_agent( policy_mapping_fn=lambda aid, *arg, **kw: f"p{aid}", policies={f"p{i}" for i in range(args.num_agents)}, @@ -70,7 +66,7 @@ stop = { NUM_ENV_STEPS_SAMPLED_LIFETIME: 500000, # `episode_return_mean` is the sum of all agents/policies' returns. - f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": -400.0 * (args.num_agents or 2), + f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": -400.0 * args.num_agents, } if __name__ == "__main__": diff --git a/rllib/tuned_examples/sac/pendulum_sac.py b/rllib/tuned_examples/sac/pendulum_sac.py index b066c66a312d..a461f042e3c0 100644 --- a/rllib/tuned_examples/sac/pendulum_sac.py +++ b/rllib/tuned_examples/sac/pendulum_sac.py @@ -1,34 +1,17 @@ from ray.rllib.algorithms.sac.sac import SACConfig -from ray.rllib.utils.metrics import ( - ENV_RUNNER_RESULTS, - EPISODE_RETURN_MEAN, - NUM_ENV_STEPS_SAMPLED_LIFETIME, -) from ray.rllib.utils.test_utils import add_rllib_example_script_args -parser = add_rllib_example_script_args() +parser = add_rllib_example_script_args( + default_timesteps=20000, + default_reward=-250.0, +) # Use `parser` to add your own custom command line options to this script # and (if needed) use their values toset up `config` below. args = parser.parse_args() config = ( SACConfig() - # Enable new API stack and use EnvRunner. - .api_stack( - enable_rl_module_and_learner=True, - enable_env_runner_and_connector_v2=True, - ) .environment(env="Pendulum-v1") - .rl_module( - model_config_dict={ - "fcnet_hiddens": [256, 256], - "fcnet_activation": "relu", - "post_fcnet_hiddens": [], - "post_fcnet_activation": None, - "post_fcnet_weights_initializer": "orthogonal_", - "post_fcnet_weights_initializer_config": {"gain": 0.01}, - } - ) .training( initial_alpha=1.001, lr=3e-4, @@ -45,19 +28,24 @@ }, num_steps_sampled_before_learning_starts=256, ) + .rl_module( + model_config_dict={ + "fcnet_hiddens": [256, 256], + "fcnet_activation": "relu", + "post_fcnet_hiddens": [], + "post_fcnet_activation": None, + "post_fcnet_weights_initializer": "orthogonal_", + "post_fcnet_weights_initializer_config": {"gain": 0.01}, + } + ) .reporting( metrics_num_episodes_for_smoothing=5, min_sample_timesteps_per_iteration=1000, ) ) -stop = { - NUM_ENV_STEPS_SAMPLED_LIFETIME: 20000, - f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": -250.0, -} - if __name__ == "__main__": from ray.rllib.utils.test_utils import run_rllib_example_script_experiment - run_rllib_example_script_experiment(config, args, stop=stop) + run_rllib_example_script_experiment(config, args) diff --git a/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py b/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py index 9b1af8e86cff..310791901f50 100644 --- a/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py +++ b/rllib/utils/replay_buffers/multi_agent_episode_replay_buffer.py @@ -131,7 +131,7 @@ def add( """ episodes: List["MultiAgentEpisode"] = force_list(episodes) - new_episode_ids: List[str] = {eps.id_ for eps in episodes} + new_episode_ids: Set[str] = {eps.id_ for eps in episodes} total_env_timesteps = sum([eps.env_steps() for eps in episodes]) self._num_timesteps += total_env_timesteps self._num_timesteps_added += total_env_timesteps @@ -455,7 +455,7 @@ def _sample_independent( self, batch_size_B: Optional[int], batch_length_T: Optional[int], - n_step: Optional[Union[int, Tuple]], + n_step: Optional[Union[int, Tuple[int, int]]], gamma: float, include_infos: bool, include_extra_model_outputs: bool, @@ -465,7 +465,7 @@ def _sample_independent( actual_n_step = n_step or 1 # Sample the n-step if necessary. - random_n_step = isinstance(n_step, tuple) + random_n_step = isinstance(n_step, (tuple, list)) sampled_episodes = [] # TODO (simon): Ensure that the module has data and if not, skip it. @@ -828,49 +828,44 @@ def _update_module_counters(self, ma_episode: MultiAgentEpisode) -> None: # Only add if the agent has stepped in the episode (chunk). if agent_steps > 0: # Receive the corresponding module ID. - # TODO (sven, simon): Is there always a mapping? What if not? - # Is then module_id == agent_id? - module_id = ma_episode._agent_to_module_mapping[agent_id] + module_id = ma_episode.module_for(agent_id) self._num_module_timesteps[module_id] += agent_steps self._num_module_timesteps_added[module_id] += agent_steps # Also add to the module episode counter. self._num_module_episodes[module_id] += 1 def _add_new_module_indices( - self, ma_episode: MultiAgentEpisode, episode_idx: int, exists: bool = True + self, + ma_episode: MultiAgentEpisode, + episode_idx: int, + ma_episode_exists: bool = True, ) -> None: """Adds the module indices for new episode chunks. Args: - multi_agent_episode: The multi-agent episode to add the module indices for. + ma_episode: The multi-agent episode to add the module indices for. episode_idx: The index of the episode in the `self.episodes`. + ma_episode_exists: Whether `ma_episode` is already in this buffer (with a + predecessor chunk to which we'll concatenate `ma_episode` later). """ + existing_ma_episode = None + if ma_episode_exists: + existing_ma_episode = self.episodes[ + self.episode_id_to_index[ma_episode.id_] - self._num_episodes_evicted + ] for agent_id in ma_episode.agent_ids: + existing_sa_eps_len = 0 + # Get the corresponding module id. - module_id = ma_episode._agent_to_module_mapping[agent_id] + module_id = ma_episode.module_for(agent_id) # Get the module episode. module_eps = ma_episode.agent_episodes[agent_id] - # Check if the module episode is already in the buffer. - if exists: - old_ma_episode = self.episodes[ - self.episode_id_to_index[ma_episode.id_] - - self._num_episodes_evicted - ] - # Is the agent episode already in the buffer? - sa_episode_in_buffer = agent_id in old_ma_episode.agent_episodes - else: - # This agent episode is new. The agent might have just entered - # the environment. - sa_episode_in_buffer = False - if sa_episode_in_buffer: - existing_eps_len = len( - self.episodes[ - episode_idx - self._num_episodes_evicted - ].agent_episodes[agent_id] - ) - else: - existing_eps_len = 0 + + # Is the agent episode already in the buffer's existing `ma_episode`? + if ma_episode_exists and agent_id in existing_ma_episode.agent_episodes: + existing_sa_eps_len = len(existing_ma_episode.agent_episodes[agent_id]) + # Add new module indices. self._module_to_indices[module_id].extend( [ @@ -878,7 +873,7 @@ def _add_new_module_indices( # Keep the MAE index for sampling episode_idx, agent_id, - existing_eps_len + i, + existing_sa_eps_len + i, ) for i in range(len(module_eps)) ] From 8d14ed8b844ab3fa914d754d721982efcaad95e4 Mon Sep 17 00:00:00 2001 From: Alan Guo Date: Tue, 28 May 2024 18:52:30 -0700 Subject: [PATCH 35/65] fix node detail page grid being hard to select (#45608) There are no visual changes, but this makes the "size" of the box match the size of the text. Previously, the box was larger than the text so each box would disrupt trying to select text from another box. --- dashboard/client/src/pages/node/NodeDetail.tsx | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/dashboard/client/src/pages/node/NodeDetail.tsx b/dashboard/client/src/pages/node/NodeDetail.tsx index b27931ae9eb7..b6de03a7ae19 100644 --- a/dashboard/client/src/pages/node/NodeDetail.tsx +++ b/dashboard/client/src/pages/node/NodeDetail.tsx @@ -92,7 +92,7 @@ const NodeDetailPage = () => { {nodeDetail && selectedTab === "info" && (
- +
Hostname
{" "} {nodeDetail.hostname} @@ -101,7 +101,7 @@ const NodeDetailPage = () => {
IP
{nodeDetail.ip}
- + {nodeDetail.cpus && ( @@ -118,7 +118,7 @@ const NodeDetailPage = () => { .join("/")} - +
Load per CPU (1/5/15min)
{" "} {nodeDetail?.loadAvg[1] && @@ -131,7 +131,7 @@ const NodeDetailPage = () => { {formatDateFromTimeMs(nodeDetail.bootTime * 1000)}
- +
Sent Tps
{" "} {memoryConverter(nodeDetail?.networkSpeed[0])}/s @@ -141,7 +141,7 @@ const NodeDetailPage = () => { {memoryConverter(nodeDetail?.networkSpeed[1])}/s
- +
Memory
{" "} {nodeDetail?.mem && ( @@ -161,7 +161,7 @@ const NodeDetailPage = () => {
- + {nodeDetail?.disk && Object.entries(nodeDetail?.disk).map(([path, obj]) => ( @@ -175,7 +175,7 @@ const NodeDetailPage = () => { ))} - +
Logs
{" "} { {raylet && Object.keys(raylet).length > 0 && selectedTab === "raylet" && (
- +
Command

@@ -201,7 +201,7 @@ const NodeDetailPage = () => {
- +
Pid
{raylet?.pid}
From 5d056f35a58956ca5b3669d61880e3cc530372be Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Tue, 28 May 2024 19:07:55 -0700 Subject: [PATCH 36/65] [ci] upgrade rayci version (#45607) Upgrade rayci version; this version enable build notification for microcheck build failures Test: - CI Signed-off-by: can --- .rayciversion | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.rayciversion b/.rayciversion index a918a2aa18d5..faef31a4357c 100644 --- a/.rayciversion +++ b/.rayciversion @@ -1 +1 @@ -0.6.0 +0.7.0 From 040b736e5fc4f731a3ffd971048e9bd66984cfe2 Mon Sep 17 00:00:00 2001 From: harborn Date: Wed, 29 May 2024 11:20:12 +0800 Subject: [PATCH 37/65] [Train] Add example of fine-tuning Llama-2 on Intel Gaudi (#44667) Adds an example for fine-tuning Llama-2-7b/70b on multiple HPUs. --------- Signed-off-by: Wu, Gangsheng --- doc/source/train/examples.yml | 10 + .../train/examples/intel_gaudi/llama.ipynb | 592 ++++++++++++++++++ 2 files changed, 602 insertions(+) create mode 100644 doc/source/train/examples/intel_gaudi/llama.ipynb diff --git a/doc/source/train/examples.yml b/doc/source/train/examples.yml index 3130c590c911..1a828ddc1714 100644 --- a/doc/source/train/examples.yml +++ b/doc/source/train/examples.yml @@ -87,6 +87,16 @@ examples: use_cases: - natural language processing link: examples/transformers/huggingface_text_classification + - title: "Fine-tune Llama-2-7b and Llama-2-70b with Intel Gaudi" + frameworks: + - accelerate + - transformers + skill_level: intermediate + use_cases: + - natural language processing + - large language models + contributor: community + link: examples/intel_gaudi/llama - title: Fine-tune a Llama-2 text generation models with DeepSpeed and Hugging Face Accelerate frameworks: diff --git a/doc/source/train/examples/intel_gaudi/llama.ipynb b/doc/source/train/examples/intel_gaudi/llama.ipynb new file mode 100644 index 000000000000..ad763c9dce1f --- /dev/null +++ b/doc/source/train/examples/intel_gaudi/llama.ipynb @@ -0,0 +1,592 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Fine-tuning Llama-2 Model with HPU\n", + "\n", + "In this Jupyter notebook, we will:\n", + "- fine-tuning a [Llama-2-7b](https://huggingface.co/meta-llama/Llama-2-7b-chat-hf) model by using HPU with DDP method\n", + "- fine-tuning a [Llama-2-70b](https://huggingface.co/meta-llama/Llama-2-70b-chat-hf) model by using HPU with DeepSpeed method\n", + "\n", + "We will use PyTorch for model training and Ray for distributed training. We will use dataset [tatsu-lab/alpaca](https://huggingface.co/datasets/tatsu-lab/alpaca).\n", + "\n", + "[Habana Gaudi AI Processors (HPUs)](https://habana.ai) are AI hardware accelerators designed by Habana Labs. For more information, see [Gaudi Architecture](https://docs.habana.ai/en/latest/Gaudi_Overview/index.html) and [Gaudi Developer Docs](https://developer.habana.ai/).\n", + "\n", + "Basic features for this fine-tuning example are:\n", + "- Running on HPUs, support three execution mode: [\"lazy\", \"eager\", \"eager.compile\"](https://docs.habana.ai/en/latest/PyTorch/Reference/PyTorch_Gaudi_Theory_of_Operations.html).\n", + "- LoRA training.\n", + "- DDP or DeepSpeed based method.\n", + "- [`GaudiTrainer`](https://github.com/huggingface/optimum-habana/blob/main/optimum/habana/transformers/trainer.py) based training.\n", + "- Llama-2-7b/Llama-2-70b model.\n", + "- Ray based resource scheduling and management." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Prepare environment\n", + "This example run on single node with 4 HPUs.\n", + "\n", + "We recommend using a prebuilt container to run these examples. To run a container, you need Docker. See [Install Docker Engine](https://docs.docker.com/engine/install/) for installation instructions.\n", + "\n", + "Next, follow [Run Using Containers](https://docs.habana.ai/en/latest/Installation_Guide/Bare_Metal_Fresh_OS.html?highlight=installer#run-using-containers) to install the Habana drivers and container runtime.\n", + "\n", + "### Get docker image\n", + "``` bash\n", + "docker pull vault.habana.ai/gaudi-docker/1.15.1/ubuntu22.04/habanalabs/pytorch-installer-2.2.0:latest\n", + "```\n", + "### Run docker image\n", + "``` bash\n", + "docker run -it --runtime=habana -e HABANA_VISIBLE_DEVICES=all -e OMPI_MCA_btl_vader_single_copy_mechanism=none --cap-add=sys_nice --net=host --ipc=host vault.habana.ai/gaudi-docker/1.15.1/ubuntu22.04/habanalabs/pytorch-installer-2.2.0:latest\n", + "# maybe should mapping your workspace volumns\n", + "```\n", + "### Install dependency\n", + "``` bash\n", + "# \"optimum-habana>1.11.1\" if exection mode \"eager\" or \"eager.compile\" \n", + "# \"ray>=2.20.0\"\n", + "pip install ray[train] notebook transformers datasets evaluate peft accelerate scikit-learn optimum-habana\n", + "\n", + "# install deepspeed\n", + "pip install git+https://github.com/HabanaAI/DeepSpeed.git@1.15.0\n", + "\n", + "# this notebook verfied with packages' version:\n", + "# transformers==4.38.2\n", + "# datasets==2.19.1\n", + "# evaluate==0.4.2\n", + "# peft==0.4.0\n", + "# accelerate==0.27.2\n", + "# scikit-learn==1.4.2\n", + "# optimum-habana==1.11.1\n", + "\n", + "# deepspeed==0.12.4+hpu.synapse.v1.15.0\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Import necessary libraries" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "import copy\n", + "from typing import Dict\n", + "\n", + "import torch\n", + "\n", + "import datasets\n", + "import transformers\n", + "from transformers import DataCollatorForLanguageModeling\n", + "\n", + "from tqdm import tqdm\n", + "\n", + "import peft\n", + "\n", + "from optimum.habana import GaudiTrainer, GaudiConfig, GaudiTrainingArguments\n", + "from optimum.habana.transformers.modeling_utils import adapt_transformers_to_gaudi" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Prepare Dataset Function\n", + "\n", + "Preprocessing the raw dataset's each line with specified format." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "\n", + "def preprocess_dataset(raw_datasets):\n", + "\n", + " PROMPT_DICT = {\n", + " \"prompt_with_input\": (\n", + " \"Below is an instruction that describes a task, paired with an input that provides further context. \"\n", + " \"Write a response that appropriately completes the request.\\n\\n\"\n", + " \"### Instruction:\\n{instruction}\\n\\n### Input:\\n{input}\\n\\n### Response:\"\n", + " ),\n", + " \"prompt_without_input\": (\n", + " \"Below is an instruction that describes a task. \"\n", + " \"Write a response that appropriately completes the request.\\n\\n\"\n", + " \"### Instruction:\\n{instruction}\\n\\n### Response:\"\n", + " ),\n", + " }\n", + "\n", + " def create_prompts(examples):\n", + " prompts = {}\n", + " prompts[\"source\"] = []\n", + " prompts[\"target\"] = []\n", + " for example in examples:\n", + " prompt_template = (\n", + " PROMPT_DICT[\"prompt_with_input\"] if example[\"input\"] != \"\" else PROMPT_DICT[\"prompt_without_input\"]\n", + " )\n", + " source = prompt_template.format_map(example)\n", + " prompts[\"source\"].append(source)\n", + " prompts[\"target\"].append(example[\"output\"])\n", + " return prompts\n", + "\n", + " # Preprocessing the datasets.\n", + " for key in raw_datasets:\n", + " prompts = create_prompts(raw_datasets[key])\n", + " columns_to_be_removed = list(raw_datasets[key].features.keys())\n", + " raw_datasets[key] = raw_datasets[key].add_column(\"prompt_sources\", prompts[\"source\"])\n", + " raw_datasets[key] = raw_datasets[key].add_column(\"prompt_targets\", prompts[\"target\"])\n", + " raw_datasets[key] = raw_datasets[key].remove_columns(columns_to_be_removed)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Dataset to Tokenizer Function\n", + "\n", + "Tokenize each line in dataset by model tokenizer.\n", + "\n", + "In example codes, we concatenate the dataset's line content to accelerate training speed.\n", + "\n", + "All datasets are processed as \"train\" datasets, no evaluation datasets are sampled from raw_datasets." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "\n", + "def preprocess_dataset_to_tokenizer(raw_datasets, tokenizer):\n", + " max_seq_length = 512\n", + " tokenizer.pad_token_id = 0\n", + " tokenizer.eos_token_id = 1\n", + " tokenizer.bos_token_id = 2\n", + "\n", + " def tokenize(prompt, add_eos_token=True):\n", + " results = tokenizer(\n", + " prompt,\n", + " truncation=True,\n", + " max_length=max_seq_length,\n", + " padding=False,\n", + " return_tensors=None,\n", + " )\n", + " for i in range(len(results[\"input_ids\"])):\n", + " if (\n", + " results[\"input_ids\"][i][-1] != tokenizer.eos_token_id\n", + " and len(results[\"input_ids\"][i]) < max_seq_length\n", + " and add_eos_token\n", + " ):\n", + " results[\"input_ids\"][i].append(tokenizer.eos_token_id)\n", + " results[\"attention_mask\"][i].append(1)\n", + "\n", + " results[\"labels\"] = copy.deepcopy(results[\"input_ids\"])\n", + " results[\"input_id_len\"] = [len(result) for result in results[\"input_ids\"]]\n", + " return results\n", + "\n", + " def preprocess_function(examples):\n", + " keys = list(examples.data.keys())\n", + " if len(keys) != 2:\n", + " raise ValueError(\"Unsupported dataset format\")\n", + "\n", + " st = [s + t for s, t in zip(examples[keys[0]], examples[keys[1]])]\n", + "\n", + " examples_tokenized = tokenize(st)\n", + " input_ids = examples_tokenized[\"input_ids\"]\n", + " labels = examples_tokenized[\"labels\"]\n", + " return {\n", + " \"input_ids\": input_ids,\n", + " \"labels\": labels,\n", + " \"attention_mask\": examples_tokenized[\"attention_mask\"],\n", + " }\n", + "\n", + " tokenized_datasets = raw_datasets.map(\n", + " preprocess_function,\n", + " batched=True,\n", + " load_from_cache_file=True,\n", + " )\n", + "\n", + " def concatenate_data(dataset, max_seq_length):\n", + " concatenated_dataset = {}\n", + " for column in dataset.features:\n", + " concatenated_data = [item for sample in dataset[column] for item in sample]\n", + " reshaped_data = [\n", + " concatenated_data[i * max_seq_length : (i + 1) * max_seq_length]\n", + " for i in range(len(concatenated_data) // max_seq_length)\n", + " ]\n", + " concatenated_dataset[column] = reshaped_data\n", + " return datasets.Dataset.from_dict(concatenated_dataset)\n", + "\n", + " tokenized_datasets_ = tokenized_datasets[\"train\"].remove_columns([\"prompt_sources\", \"prompt_targets\"])\n", + " tokenized_datasets[\"train\"] = concatenate_data(tokenized_datasets_, max_seq_length)\n", + "\n", + " return tokenized_datasets" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Prepare training arguments\n", + "\n", + "here some arguments are hard coded, you can pass arguments from `config`" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "\n", + "def prepare_training_args(config: Dict):\n", + " # prepare execution mode config\n", + " execution_mode = config[\"execution_mode\"]\n", + " use_lazy_mode = True if execution_mode == \"lazy\" else False\n", + " torch_compile_backend = \"hpu_backend\" if execution_mode == \"eager.compile\" else None\n", + "\n", + " deepspeed = config[\"deepspeed\"] if \"deepspeed\" in config else None\n", + "\n", + " return GaudiTrainingArguments(deepspeed=deepspeed,\n", + " output_dir=config[\"output\"],\n", + " do_train=True,\n", + " do_eval=False,\n", + " per_device_train_batch_size=config[\"batch_size_per_worker\"],\n", + " bf16=True,\n", + " learning_rate=config[\"lr\"],\n", + " save_strategy=\"no\",\n", + " torch_compile_backend=torch_compile_backend,\n", + " evaluation_strategy=\"no\",\n", + " lr_scheduler_type=\"cosine\",\n", + " num_train_epochs=config[\"epochs\"],\n", + " use_lazy_mode=use_lazy_mode,\n", + " use_habana=True,\n", + " pipelining_fwd_bwd=True,\n", + " save_only_model=True,\n", + " gradient_checkpointing=True,\n", + " warmup_ratio=0.03,\n", + " throughput_warmup_steps=3,\n", + " logging_steps=5)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Prepare model\n", + "\n", + "1. download model from huggingface or read model from local directory.\n", + "2. convert model to lora model.\n", + "3. move model to HPU device.\n", + "\n", + "If you doesn't want to fine-tune with LoRA, just remove LoRA conversion step." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "\n", + "def prepare_model(config: Dict, device):\n", + " # prepare from pretrained model\n", + " deepspeed = config[\"deepspeed\"] if \"deepspeed\" in config else None\n", + " if deepspeed is not None:\n", + " auto_config = transformers.AutoConfig.from_pretrained(config[\"model\"], use_cache=False, revision=\"main\", use_auth_token=None, trust_remote_code=None)\n", + " model = transformers.AutoModelForCausalLM.from_pretrained(config[\"model\"], config=auto_config, **config[\"model_config\"])\n", + " model.generation_config.attn_softmax_bf16 = True\n", + " model.generation_config.use_flash_attention = True\n", + " else:\n", + " model = transformers.AutoModelForCausalLM.from_pretrained(config[\"model\"], **config[\"model_config\"])\n", + " model.enable_input_require_grads()\n", + "\n", + " # convert to peft model for lora training\n", + " peft_config = peft.LoraConfig(**config[\"lora_config\"])\n", + " model = peft.get_peft_model(model, peft_config)\n", + "\n", + " model.to(dtype=config[\"model_config\"][\"torch_dtype\"], device=device)\n", + "\n", + " return model\n" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Training Function\n", + "\n", + "This function will be executed by each worker during training, with following steps:\n", + "\n", + "- preparing training args, an instance of `GaudiTrainingArguments`.\n", + "- loading datasets and preprocess datasets, just load the first 4096 item as training datasets.\n", + "- loading pretrained model as tokenizer, and process datasets to tokenizer.\n", + "- loading pretrained model.\n", + "- preparing data collator and gaidu_config.\n", + "- preparing instance of `GaudiTrainer`.\n", + "- calling `train()` to train model.\n", + "- saving model results.\n", + "\n", + "Compared to a training function for GPU, no changes are needed to port to HPU. Internally, Ray Train does these things:\n", + "\n", + "- Detect HPU and set the device.\n", + "- Initialize the habana PyTorch backend.\n", + "- Initialize the habana distributed backend." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "\n", + "def train_func_per_worker(config: Dict):\n", + " # adapt transformers to gaudi\n", + " adapt_transformers_to_gaudi()\n", + "\n", + " # prepare training arguments\n", + " training_args = prepare_training_args(config)\n", + "\n", + " # prepare datasets\n", + " # here we use dataset \"tatsu-lab/alpaca\" from huggingface\n", + " raw_datasets = datasets.DatasetDict({\"train\": datasets.load_dataset(\"tatsu-lab/alpaca\", split='train[0:4096]')})\n", + " preprocess_dataset(raw_datasets)\n", + "\n", + " # prepare tokenizer\n", + " tokenizer = transformers.AutoTokenizer.from_pretrained(config[\"model\"])\n", + " tokenized_datasets = preprocess_dataset_to_tokenizer(raw_datasets, tokenizer)\n", + "\n", + " # prepare model\n", + " model = prepare_model(config, training_args.device)\n", + "\n", + " # prepare data collator\n", + " data_collator = DataCollatorForLanguageModeling(tokenizer, pad_to_multiple_of=8, return_tensors=\"pt\", mlm=False)\n", + "\n", + " # prepare gaudi config\n", + " gaudi_config = GaudiConfig()\n", + " gaudi_config.use_fused_adam = True\n", + " gaudi_config.use_fused_clip_norm = True\n", + "\n", + " # instance GaudiTrainer\n", + " trainer = GaudiTrainer(\n", + " model=model,\n", + " gaudi_config=gaudi_config,\n", + " args=training_args,\n", + " train_dataset=tokenized_datasets[\"train\"],\n", + " eval_dataset=None,\n", + " tokenizer=tokenizer,\n", + " data_collator=data_collator,\n", + " compute_metrics=None,\n", + " preprocess_logits_for_metrics=None,\n", + " )\n", + "\n", + " train_result = trainer.train()\n", + " print(f\"train_result = {train_result}\")\n", + " trainer.save_model()\n", + "\n", + " return train_result" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Main Training Function\n", + "The `train_llama` function sets up the distributed training environment using Ray and starts the training process. To enable training using HPU, we only need to make the following changes:\n", + "- Set the exectuion mode for training, supported execution mode are:\n", + "\n", + " - \"lazy\": Deferred execution of graphs, comprising of ops delivered from script op by op similar to Eager mode. It gives the Eager mode experience with performance on Gaudi. Unlike Eager Mode with torch.compile, graph is analyzed in each iteration leading to a higher CPU usage.\n", + " - \"eager\": Op-by-op execution as defined in standard PyTorch Eager mode scripts.\n", + " - \"eager.compile\": Eager mode extended with `torch.compile` - Similar to Eager mode but extended with wrapping complete or part of model (such as a function) into a graph. Parts that are not wrapped are executed eagerly.\n", + "\n", + " More detail theory can be found [here](https://docs.habana.ai/en/latest/PyTorch/Reference/PyTorch_Gaudi_Theory_of_Operations.html), and detail performance results can be found [here](https://developer.habana.ai/get-started/habana-models-performance/)\n", + "- Set training method, supported method are:\n", + " - \"ddp\"\n", + " - \"deepspeed\"\n", + "- Require an HPU for each worker in ScalingConfig\n", + "- Set backend to `hccl` in TorchConfig" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "\n", + "def train_llama(num_workers, execution_mode, training_method):\n", + " import ray\n", + " from ray.train import ScalingConfig\n", + " from ray.train.torch import TorchTrainer, TorchConfig\n", + "\n", + " # deepspeed config, can also place it to config file\n", + " deepspeed_config = {\n", + " \"steps_per_print\": 64,\n", + " \"train_batch_size\": \"auto\",\n", + " \"train_micro_batch_size_per_gpu\": \"auto\",\n", + " \"gradient_accumulation_steps\": \"auto\",\n", + " \"bf16\": {\n", + " \"enabled\": True\n", + " },\n", + " \"gradient_clipping\": 1.0,\n", + " \"zero_optimization\": {\n", + " \"stage\": 3,\n", + " \"overlap_comm\": False,\n", + " \"contiguous_gradients\": False,\n", + " \"stage3_gather_16bit_weights_on_model_save\": True\n", + " }\n", + " }\n", + "\n", + " # Preparing train configurations\n", + " train_config = {\n", + " \"execution_mode\": execution_mode,\n", + " \"model\": \"/root/models/models--meta-llama--Llama-2-70b-chat-hf/snapshots/e9149a12809580e8602995856f8098ce973d1080/\",\n", + " \"model_config\": {\"torch_dtype\": torch.bfloat16, \"trust_remote_code\": False, \"use_auth_token\": None},\n", + " \"lora_config\": {\"task_type\": \"CAUSAL_LM\", \"r\": 8, \"lora_alpha\": 32, \"lora_dropout\": 0.1, \"target_modules\": [\"q_proj\", \"v_proj\"]},\n", + " \"lr\": 1e-4,\n", + " \"epochs\": 2,\n", + " \"batch_size_per_worker\": 8,\n", + " \"output\": \"/tmp/ray/\",\n", + " \"deepspeed\": deepspeed_config if training_method == \"deepspeed\" else None,\n", + " }\n", + "\n", + " # Configure computation resources\n", + " # In ScalingConfig, require an HPU for each worker\n", + " scaling_config = ScalingConfig(num_workers=num_workers, resources_per_worker={\"CPU\": 1, \"HPU\": 1})\n", + " # Set backend to hccl in TorchConfig\n", + " torch_config = TorchConfig(backend = \"hccl\")\n", + "\n", + " # start your ray cluster\n", + " ray.init()\n", + "\n", + " # Initialize a Ray TorchTrainer\n", + " trainer = TorchTrainer(\n", + " train_loop_per_worker=train_func_per_worker,\n", + " train_loop_config=train_config,\n", + " torch_config=torch_config,\n", + " scaling_config=scaling_config,\n", + " )\n", + "\n", + " result = trainer.fit()\n", + " print(f\"Training result: {result}\")\n" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Start Training\n", + "\n", + "Finally, we call the `train_llama` function to start the training process. You can adjust the number of workers to use, and the execution mode for HPU." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# set some environment variables\n", + "os.environ[\"RAY_EXPERIMENTAL_NOSET_HABANA_VISIBLE_MODULES\"] = \"0\"\n", + "# if using RAY_EXPERIMENTAL_NOSET_HABANA_VISIBLE_MODULES env var\n", + "# you must set HABANA_VISIBLE_DEVICES, such as\n", + "# os.environ[\"HABANA_VISIBLE_DEVICES\"] = \"0,1,2,3\"\n", + "\n", + "# execution_mode are [\"lazy\", \"eager\", \"eager.compile\"]\n", + "execution_mode = \"lazy\"\n", + "os.environ[\"PT_HPU_LAZY_MODE\"] = \"1\" if execution_mode == \"lazy\" else \"0\"\n", + "\n", + "# training_method are [\"ddp\", \"deepspeed\"]\n", + "training_method = \"deepspeed\"\n", + "if training_method == \"deepspeed\":\n", + " os.environ[\"PT_HPU_MAX_COMPOUND_OP_SIZE\"] = \"10\"\n", + " os.environ[\"DEEPSPEED_HPU_ZERO3_SYNC_MARK_STEP_REQUIRED\"] = \"1\"\n", + "\n", + "# here use 4 HPUs\n", + "train_llama(num_workers=4, execution_mode=execution_mode, training_method=training_method)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Final output\n", + "\n", + "### For DDP on HPUs\n", + "- Llama-2-70b-chat-hf\n", + "- 4 HPU\n", + "- LoRA\n", + "\n", + "``` bash\n", + "(RayTrainWorker pid=123181) {'loss': 1.8051, 'grad_norm': 0.6015625, 'learning_rate': 9.938441702975689e-05, 'epoch': 0.16, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.6754, 'grad_norm': 0.408203125, 'learning_rate': 9.567727288213005e-05, 'epoch': 0.32, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.568, 'grad_norm': 0.4453125, 'learning_rate': 8.885729807284856e-05, 'epoch': 0.48, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.4934, 'grad_norm': 0.4609375, 'learning_rate': 7.938926261462366e-05, 'epoch': 0.65, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.3965, 'grad_norm': 0.3515625, 'learning_rate': 6.7918397477265e-05, 'epoch': 0.81, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.3461, 'grad_norm': 0.34765625, 'learning_rate': 5.522642316338268e-05, 'epoch': 0.97, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.2924, 'grad_norm': 0.32421875, 'learning_rate': 4.2178276747988446e-05, 'epoch': 1.13, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.2643, 'grad_norm': 0.33203125, 'learning_rate': 2.9663167846209998e-05, 'epoch': 1.29, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.263, 'grad_norm': 0.318359375, 'learning_rate': 1.8533980447508137e-05, 'epoch': 1.45, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.2502, 'grad_norm': 0.275390625, 'learning_rate': 9.549150281252633e-06, 'epoch': 1.61, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.2161, 'grad_norm': 0.2734375, 'learning_rate': 3.3209786751399187e-06, 'epoch': 1.77, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=123181) {'loss': 1.2517, 'grad_norm': 0.294921875, 'learning_rate': 2.7390523158633554e-07, 'epoch': 1.94, 'memory_allocated (GB)': 13.64, 'max_memory_allocated (GB)': 48.92, 'total_memory_available (GB)': 94.62}\n", + "```\n", + "\n", + "### For DeepSpeed on HPUs\n", + "- Llama-2-70b-chat-hf\n", + "- 4 HPU\n", + "- LoRA\n", + "\n", + "``` bash\n", + "(RayTrainWorker pid=110856) {'loss': 1.6627, 'grad_norm': 0.35921376943588257, 'learning_rate': 9.938441702975689e-05, 'epoch': 0.16, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.6085, 'grad_norm': 0.35271379351615906, 'learning_rate': 9.567727288213005e-05, 'epoch': 0.32, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.5051, 'grad_norm': 0.4277978837490082, 'learning_rate': 8.885729807284856e-05, 'epoch': 0.48, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.4157, 'grad_norm': 0.5138524770736694, 'learning_rate': 7.938926261462366e-05, 'epoch': 0.65, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.3233, 'grad_norm': 0.3451262414455414, 'learning_rate': 6.7918397477265e-05, 'epoch': 0.81, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.2728, 'grad_norm': 0.38564223051071167, 'learning_rate': 5.522642316338268e-05, 'epoch': 0.97, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.1989, 'grad_norm': 0.36078131198883057, 'learning_rate': 4.2178276747988446e-05, 'epoch': 1.13, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.1552, 'grad_norm': 0.47946077585220337, 'learning_rate': 2.9663167846209998e-05, 'epoch': 1.29, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.1413, 'grad_norm': 0.3357600271701813, 'learning_rate': 1.8533980447508137e-05, 'epoch': 1.45, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.129, 'grad_norm': 0.2777070701122284, 'learning_rate': 9.549150281252633e-06, 'epoch': 1.61, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.0876, 'grad_norm': 0.25669950246810913, 'learning_rate': 3.3209786751399187e-06, 'epoch': 1.77, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "(RayTrainWorker pid=110856) {'loss': 1.1238, 'grad_norm': 0.2423330545425415, 'learning_rate': 2.7390523158633554e-07, 'epoch': 1.94, 'memory_allocated (GB)': 32.88, 'max_memory_allocated (GB)': 43.56, 'total_memory_available (GB)': 94.62}\n", + "```" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "orphan": true, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + } + }, + "nbformat": 4, + "nbformat_minor": 2 +} From 0be0639f37864e2671c4069ee11becb4110b6a85 Mon Sep 17 00:00:00 2001 From: Shubham Gandhi Date: Wed, 29 May 2024 12:38:56 +0530 Subject: [PATCH 38/65] Documentation/improve experimental ray serve advanced guide (#45046) ## Why are these changes needed? Update the experimental feature guide on multi-container deployment approach for Ray Serve. ## Related issue number Closes: #45026 ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: dudeperf3ct --- .../advanced-guides/multi-app-container.md | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/doc/source/serve/advanced-guides/multi-app-container.md b/doc/source/serve/advanced-guides/multi-app-container.md index a7658c906981..8e1bb208409a 100644 --- a/doc/source/serve/advanced-guides/multi-app-container.md +++ b/doc/source/serve/advanced-guides/multi-app-container.md @@ -44,6 +44,9 @@ RUN sudo apt-get update && sudo apt-get install curl -y # Download the source code for the Whisper application into `whisper_example.py`. RUN curl -O https://raw.githubusercontent.com/ray-project/ray/master/doc/source/serve/doc_code/whisper_example.py + +# Add /home/ray path to PYTHONPATH avoid import module error +ENV PYTHONPATH "${PYTHONPATH}:/home/ray" ``` ::: :::{tab-item} resnet.Dockerfile @@ -57,39 +60,42 @@ RUN sudo apt-get update && sudo apt-get install curl -y # Download the source code for the ResNet application into `resnet50_example.py`. RUN curl -O https://raw.githubusercontent.com/ray-project/ray/master/doc/source/serve/doc_code/resnet50_example.py + +# Add /home/ray path to PYTHONPATH avoid import module error +ENV PYTHONPATH "${PYTHONPATH}:/home/ray" ``` ::: :::: -Then, build the corresponding Docker images and push it to your choice of Docker registry. This tutorial uses `alice/whisper_image:latest` and `alice/resnet_image:latest` as placeholder names for the images, but make sure to swap out `alice` for a repo name of your choice. +Then, build the corresponding container images and push it to your choice of container registry. This tutorial uses `alice/whisper_image:latest` and `alice/resnet_image:latest` as placeholder names for the images, but make sure to swap out `alice` for a repo name of your choice. ::::{tab-set} :::{tab-item} Whisper ```bash -# Build the Docker image from the Dockerfile +# Build the container image from the Dockerfile using podman export IMG1=alice/whisper_image:latest -docker build -t $IMG1 -f whisper.Dockerfile . -# Push to a Docker registry. This step is unnecessary if you are deploying Serve locally. -docker push $IMG1 +podman build -t $IMG1 -f whisper.Dockerfile . +# Push to a registry. This step is unnecessary if you are deploying Serve locally. +podman push $IMG1 ``` ::: :::{tab-item} Resnet ```bash -# Build the Docker image from the Dockerfile +# Build the container image from the Dockerfile using podman export IMG2=alice/resnet_image:latest -docker build -t $IMG2 -f resnet.Dockerfile . -# Push to a Docker registry. This step is unnecessary if you are deploying Serve locally. -docker push $IMG2 +podman build -t $IMG2 -f resnet.Dockerfile . +# Push to a registry. This step is unnecessary if you are deploying Serve locally. +podman push $IMG2 ``` ::: :::: -Finally, you can specify the Docker image within which you want to run each application in the `container` field of an application's runtime environment specification. The `container` field has three fields: +Finally, you can specify the container image within which you want to run each application in the `container` field of an application's runtime environment specification. The `container` field has three fields: - `image`: (Required) The image to run your application in. - `worker_path`: The absolute path to `default_worker.py` inside the container. - `run_options`: Additional options to pass to the `podman run` command used to start a Serve deployment replica in a container. See [podman run documentation](https://docs.podman.io/en/latest/markdown/podman-run.1.html) for a list of all options. If you are familiar with `docker run`, most options work the same way. -The following Serve config runs the `whisper` app with the image `IMG1`, and the `resnet` app with the image `IMG2`. Concretely, all deployment replicas in the applications start and run in containers with the respective images. +The following Serve config runs the `whisper` app with the image `IMG1`, and the `resnet` app with the image `IMG2`. `podman images` command can be used to list the names of the images. Concretely, all deployment replicas in the applications start and run in containers with the respective images. ```yaml applications: From 1719a8f137e8c80920eb73879fa91d93bd1ea69f Mon Sep 17 00:00:00 2001 From: chuanzhisongshu <49055103+982945902@users.noreply.github.com> Date: Wed, 29 May 2024 23:58:23 +0800 Subject: [PATCH 39/65] [Core] Remove duplicate included header (#45406) Signed-off-by: lishuo121 --- src/ray/util/logging.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/src/ray/util/logging.cc b/src/ray/util/logging.cc index 8e9f45d4b865..fdcf5d22b56c 100644 --- a/src/ray/util/logging.cc +++ b/src/ray/util/logging.cc @@ -16,7 +16,6 @@ #include -#include #ifdef _WIN32 #include #else From a84a1b2284c15c5ec17fd55afa479f0014abe508 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Wed, 29 May 2024 09:38:47 -0700 Subject: [PATCH 40/65] [runtime_env] unify `container` and (new) `image_uri` under plugin mechanism (#45156) Signed-off-by: Cindy Zhang --- .../runtime_env/agent/runtime_env_agent.py | 13 +- python/ray/_private/runtime_env/container.py | 72 ---------- python/ray/_private/runtime_env/context.py | 16 +-- .../ray/_private/runtime_env/default_impl.py | 5 + python/ray/_private/runtime_env/image_uri.py | 124 ++++++++++++++++++ python/ray/_private/runtime_env/plugin.py | 4 +- python/ray/_private/runtime_env/py_modules.py | 2 +- python/ray/runtime_env/runtime_env.py | 15 +++ python/ray/tests/conftest_docker.py | 2 +- .../test_log_file_exists.py | 13 +- .../runtime_env_container/test_put_get.py | 13 +- .../test_ray_env_vars.py | 15 ++- .../runtime_env_container/test_serve_basic.py | 16 ++- .../test_serve_telemetry.py | 17 ++- .../test_shared_memory.py | 12 +- ...xit_intended_system_exit_and_user_error.py | 20 ++- .../ray/tests/test_runtime_env_container.py | 37 +++++- 17 files changed, 280 insertions(+), 116 deletions(-) delete mode 100644 python/ray/_private/runtime_env/container.py create mode 100644 python/ray/_private/runtime_env/default_impl.py create mode 100644 python/ray/_private/runtime_env/image_uri.py diff --git a/python/ray/_private/runtime_env/agent/runtime_env_agent.py b/python/ray/_private/runtime_env/agent/runtime_env_agent.py index 830e0356aef6..59959f19aad3 100644 --- a/python/ray/_private/runtime_env/agent/runtime_env_agent.py +++ b/python/ray/_private/runtime_env/agent/runtime_env_agent.py @@ -14,9 +14,10 @@ import ray._private.runtime_env.agent.runtime_env_consts as runtime_env_consts from ray._private.ray_logging import setup_component_logger from ray._private.runtime_env.conda import CondaPlugin -from ray._private.runtime_env.container import ContainerManager from ray._private.runtime_env.context import RuntimeEnvContext +from ray._private.runtime_env.default_impl import get_image_uri_plugin from ray._private.runtime_env.java_jars import JavaJarsPlugin +from ray._private.runtime_env.image_uri import ContainerPlugin from ray._private.runtime_env.pip import PipPlugin from ray._private.gcs_utils import GcsAioClient from ray._private.runtime_env.plugin import ( @@ -200,11 +201,12 @@ def __init__( self._working_dir_plugin = WorkingDirPlugin( self._runtime_env_dir, self._gcs_aio_client ) + self._container_plugin = ContainerPlugin(temp_dir) # TODO(jonathan-anyscale): change the plugin to ProfilerPlugin # and unify with nsight and other profilers. self._nsight_plugin = NsightPlugin(self._runtime_env_dir) - self._container_manager = ContainerManager(temp_dir) self._mpi_plugin = MPIPlugin() + self._image_uri_plugin = get_image_uri_plugin(temp_dir) # TODO(architkulkarni): "base plugins" and third-party plugins should all go # through the same code path. We should never need to refer to @@ -215,8 +217,10 @@ def __init__( self._conda_plugin, self._py_modules_plugin, self._java_jars_plugin, + self._container_plugin, self._nsight_plugin, self._mpi_plugin, + self._image_uri_plugin, ] self._plugin_manager = RuntimeEnvPluginManager() for plugin in self._base_plugins: @@ -244,7 +248,7 @@ def __init__( "Listening to address %s, port %d", address, runtime_env_agent_port ) - def uris_parser(self, runtime_env): + def uris_parser(self, runtime_env: RuntimeEnv): result = list() for name, plugin_setup_context in self._plugin_manager.plugins.items(): plugin = plugin_setup_context.class_instance @@ -309,9 +313,6 @@ async def _setup_runtime_env( # avoid lint error. That will be moved to cgroup plugin. per_job_logger.debug(f"Worker has resource :" f"{allocated_resource}") context = RuntimeEnvContext(env_vars=runtime_env.env_vars()) - await self._container_manager.setup( - runtime_env, context, logger=per_job_logger - ) # Warn about unrecognized fields in the runtime env. for name, _ in runtime_env.plugins(): diff --git a/python/ray/_private/runtime_env/container.py b/python/ray/_private/runtime_env/container.py deleted file mode 100644 index afae8455549e..000000000000 --- a/python/ray/_private/runtime_env/container.py +++ /dev/null @@ -1,72 +0,0 @@ -import os -import logging - -from typing import Optional - -from ray._private.runtime_env.context import RuntimeEnvContext - -default_logger = logging.getLogger(__name__) - - -class ContainerManager: - def __init__(self, tmp_dir: str): - # _ray_tmp_dir will be mounted into container, so the worker process - # can connect to raylet. - self._ray_tmp_dir = tmp_dir - - async def setup( - self, - runtime_env: "RuntimeEnv", # noqa: F821 - context: RuntimeEnvContext, - logger: Optional[logging.Logger] = default_logger, - ): - if not runtime_env.has_py_container() or not runtime_env.py_container_image(): - return - - container_driver = "podman" - context.container = runtime_env["container"] - container_command = [ - container_driver, - "run", - "-v", - self._ray_tmp_dir + ":" + self._ray_tmp_dir, - "--cgroup-manager=cgroupfs", - "--network=host", - "--pid=host", - "--ipc=host", - # NOTE(zcin): Mounted volumes in rootless containers are - # owned by the user `root`. The user on host (which will - # usually be `ray` if this is being run in a ray docker - # image) who started the container is mapped using user - # namespaces to the user `root` in a rootless container. In - # order for the Ray Python worker to access the mounted ray - # tmp dir, we need to use keep-id mode which maps the user - # as itself (instead of as `root`) into the container. - # https://www.redhat.com/sysadmin/rootless-podman-user-namespace-modes - "--userns=keep-id", - ] - - # The RAY_RAYLET_PID and RAY_JOB_ID environment variables are - # needed for the default worker. - container_command.append("--env") - container_command.append("RAY_RAYLET_PID=" + os.getenv("RAY_RAYLET_PID")) - container_command.append("--env") - container_command.append("RAY_JOB_ID=$RAY_JOB_ID") - for env_var_name, env_var_value in os.environ.items(): - if env_var_name.startswith("RAY_") and env_var_name not in [ - "RAY_RAYLET_PID", - "RAY_JOB_ID", - ]: - container_command.append("--env") - container_command.append(f"{env_var_name}='{env_var_value}'") - - if runtime_env.py_container_run_options(): - container_command.extend(runtime_env.py_container_run_options()) - # TODO(chenk008): add resource limit - container_command.append("--entrypoint") - container_command.append("python") - container_command.append(runtime_env.py_container_image()) - context.py_executable = " ".join(container_command) - logger.info( - "start worker in container with prefix: {}".format(context.py_executable) - ) diff --git a/python/ray/_private/runtime_env/context.py b/python/ray/_private/runtime_env/context.py index 5eea9690f896..46a01af267be 100644 --- a/python/ray/_private/runtime_env/context.py +++ b/python/ray/_private/runtime_env/context.py @@ -4,7 +4,7 @@ import subprocess import shlex import sys -from typing import Any, Dict, List, Optional +from typing import Dict, List, Optional from ray.util.annotations import DeveloperAPI from ray.core.generated.common_pb2 import Language @@ -24,7 +24,7 @@ def __init__( env_vars: Dict[str, str] = None, py_executable: Optional[str] = None, resources_dir: Optional[str] = None, - container: Dict[str, Any] = None, + override_worker_entrypoint: Optional[str] = None, java_jars: List[str] = None, ): self.command_prefix = command_prefix or [] @@ -35,7 +35,7 @@ def __init__( # the legacy Ray client codepath to pass the resources dir to the shim # process. We should remove it once Ray client uses the agent. self.resources_dir: str = resources_dir - self.container = container or {} + self.override_worker_entrypoint: Optional[str] = override_worker_entrypoint self.java_jars = java_jars or [] def serialize(self) -> str: @@ -72,13 +72,13 @@ def exec_worker(self, passthrough_args: List[str], language: Language): # However, the path to default_worker.py inside the container # can be different. We need the user to specify the path to # default_worker.py inside the container. - default_worker_path = self.container.get("worker_path") - if self.container and default_worker_path: + # default_worker_path = self.container.get("worker_path") + if self.override_worker_entrypoint: logger.debug( - f"Changing the default worker path from {passthrough_args[0]} to " - f"{default_worker_path}." + f"Changing the worker entrypoint from {passthrough_args[0]} to " + f"{self.override_worker_entrypoint}." ) - passthrough_args[0] = default_worker_path + passthrough_args[0] = self.override_worker_entrypoint if sys.platform == "win32": diff --git a/python/ray/_private/runtime_env/default_impl.py b/python/ray/_private/runtime_env/default_impl.py new file mode 100644 index 000000000000..40fd5485be61 --- /dev/null +++ b/python/ray/_private/runtime_env/default_impl.py @@ -0,0 +1,5 @@ +from ray._private.runtime_env.image_uri import ImageURIPlugin + + +def get_image_uri_plugin(ray_tmp_dir: str): + return ImageURIPlugin(ray_tmp_dir) diff --git a/python/ray/_private/runtime_env/image_uri.py b/python/ray/_private/runtime_env/image_uri.py new file mode 100644 index 000000000000..ab8b2f779d8b --- /dev/null +++ b/python/ray/_private/runtime_env/image_uri.py @@ -0,0 +1,124 @@ +import logging +import os +from typing import List, Optional + +from ray._private.runtime_env.context import RuntimeEnvContext +from ray._private.runtime_env.plugin import RuntimeEnvPlugin + +default_logger = logging.getLogger(__name__) + + +def _modify_context_impl( + image_uri: str, + run_options: Optional[List[str]], + worker_path: Optional[str], + context: RuntimeEnvContext, + logger: logging.Logger, + ray_tmp_dir: str, +): + if worker_path: + context.override_worker_entrypoint = worker_path + + container_driver = "podman" + container_command = [ + container_driver, + "run", + "-v", + ray_tmp_dir + ":" + ray_tmp_dir, + "--cgroup-manager=cgroupfs", + "--network=host", + "--pid=host", + "--ipc=host", + # NOTE(zcin): Mounted volumes in rootless containers are + # owned by the user `root`. The user on host (which will + # usually be `ray` if this is being run in a ray docker + # image) who started the container is mapped using user + # namespaces to the user `root` in a rootless container. In + # order for the Ray Python worker to access the mounted ray + # tmp dir, we need to use keep-id mode which maps the user + # as itself (instead of as `root`) into the container. + # https://www.redhat.com/sysadmin/rootless-podman-user-namespace-modes + "--userns=keep-id", + ] + + # The RAY_RAYLET_PID and RAY_JOB_ID environment variables are + # needed for the default worker. + container_command.append("--env") + container_command.append("RAY_RAYLET_PID=" + os.getenv("RAY_RAYLET_PID")) + container_command.append("--env") + container_command.append("RAY_JOB_ID=$RAY_JOB_ID") + for env_var_name, env_var_value in os.environ.items(): + if env_var_name.startswith("RAY_") and env_var_name not in [ + "RAY_RAYLET_PID", + "RAY_JOB_ID", + ]: + container_command.append("--env") + container_command.append(f"{env_var_name}='{env_var_value}'") + + if run_options: + container_command.extend(run_options) + # TODO(chenk008): add resource limit + container_command.append("--entrypoint") + container_command.append("python") + container_command.append(image_uri) + + # Example: + # podman run -v /tmp/ray:/tmp/ray + # --cgroup-manager=cgroupfs --network=host --pid=host --ipc=host + # --userns=keep-id --env RAY_RAYLET_PID=23478 --env RAY_JOB_ID=$RAY_JOB_ID + # --entrypoint python rayproject/ray:nightly-py39 + container_command_str = " ".join(container_command) + logger.info(f"Starting worker in container with prefix {container_command_str}") + + context.py_executable = container_command_str + + +class ImageURIPlugin(RuntimeEnvPlugin): + """Starts worker in a container of a custom image.""" + + name = "image_uri" + + def __init__(self, ray_tmp_dir: str): + self._ray_tmp_dir = ray_tmp_dir + + def modify_context( + self, + uris: List[str], + runtime_env: "RuntimeEnv", # noqa: F821 + context: RuntimeEnvContext, + logger: Optional[logging.Logger] = default_logger, + ): + if not runtime_env.image_uri(): + return + + _modify_context_impl( + runtime_env.image_uri(), [], None, context, logger, self._ray_tmp_dir + ) + + +class ContainerPlugin(RuntimeEnvPlugin): + """Starts worker in container.""" + + name = "container" + + def __init__(self, ray_tmp_dir: str): + self._ray_tmp_dir = ray_tmp_dir + + def modify_context( + self, + uris: List[str], + runtime_env: "RuntimeEnv", # noqa: F821 + context: RuntimeEnvContext, + logger: Optional[logging.Logger] = default_logger, + ): + if not runtime_env.has_py_container() or not runtime_env.py_container_image(): + return + + _modify_context_impl( + runtime_env.py_container_image(), + runtime_env.py_container_run_options(), + runtime_env.py_container_worker_path(), + context, + logger, + self._ray_tmp_dir, + ) diff --git a/python/ray/_private/runtime_env/plugin.py b/python/ray/_private/runtime_env/plugin.py index 98d610b138f8..a1e03a507b59 100644 --- a/python/ray/_private/runtime_env/plugin.py +++ b/python/ray/_private/runtime_env/plugin.py @@ -47,7 +47,7 @@ def get_uris(self, runtime_env: "RuntimeEnv") -> List[str]: # noqa: F821 async def create( self, uri: Optional[str], - runtime_env: "RuntimeEnv", # noqa: F821 + runtime_env, context: RuntimeEnvContext, logger: logging.Logger, ) -> float: @@ -227,7 +227,7 @@ def sorted_plugin_setup_contexts(self) -> List[PluginSetupContext]: async def create_for_plugin_if_needed( - runtime_env, + runtime_env: "RuntimeEnv", # noqa: F821 plugin: RuntimeEnvPlugin, uri_cache: URICache, context: RuntimeEnvContext, diff --git a/python/ray/_private/runtime_env/py_modules.py b/python/ray/_private/runtime_env/py_modules.py index c78f10da5f20..551e22b3650a 100644 --- a/python/ray/_private/runtime_env/py_modules.py +++ b/python/ray/_private/runtime_env/py_modules.py @@ -178,7 +178,7 @@ def delete_uri( return local_dir_size - def get_uris(self, runtime_env: dict) -> List[str]: + def get_uris(self, runtime_env) -> List[str]: return runtime_env.py_modules() async def create( diff --git a/python/ray/runtime_env/runtime_env.py b/python/ray/runtime_env/runtime_env.py index 5cd49f5b32ee..6999e463e761 100644 --- a/python/ray/runtime_env/runtime_env.py +++ b/python/ray/runtime_env/runtime_env.py @@ -288,6 +288,7 @@ class MyClass: "worker_process_setup_hook", "_nsight", "mpi", + "image_uri", } extensions_fields: Set[str] = { @@ -310,6 +311,7 @@ def __init__( config: Optional[Union[Dict, RuntimeEnvConfig]] = None, _validate: bool = True, mpi: Optional[Dict] = None, + image_uri: Optional[str] = None, **kwargs, ): super().__init__() @@ -335,6 +337,8 @@ def __init__( runtime_env["worker_process_setup_hook"] = worker_process_setup_hook if mpi is not None: runtime_env["mpi"] = mpi + if image_uri is not None: + runtime_env["image_uri"] = image_uri if runtime_env.get("java_jars"): runtime_env["java_jars"] = runtime_env.get("java_jars") @@ -368,6 +372,14 @@ def __init__( f"Specified fields: {runtime_env.keys()}" ) + if self.get("image_uri"): + if len(runtime_env) > 1: + raise ValueError( + "The 'image_uri' field currently cannot be used " + "together with other fields of runtime_env. " + f"Specified fields: {runtime_env.keys()}" + ) + for option, validate_fn in OPTION_TO_VALIDATION_FN.items(): option_val = self.get(option) if option_val is not None: @@ -545,6 +557,9 @@ def py_container_run_options(self) -> List: return None return self["container"].get("run_options", []) + def image_uri(self) -> Optional[str]: + return self.get("image_uri") + def plugins(self) -> List[Tuple[str, Any]]: result = list() for key, value in self.items(): diff --git a/python/ray/tests/conftest_docker.py b/python/ray/tests/conftest_docker.py index 52eebb9d6a19..363cee6ccb5a 100644 --- a/python/ray/tests/conftest_docker.py +++ b/python/ray/tests/conftest_docker.py @@ -175,7 +175,7 @@ def run_in_container(cmds: List[List[str]], container_id: str): NESTED_IMAGE_NAME = "rayproject/ray:runtime_env_container_nested" -@pytest.fixture +@pytest.fixture(scope="session") def podman_docker_cluster(): start_container_command = [ "docker", diff --git a/python/ray/tests/runtime_env_container/test_log_file_exists.py b/python/ray/tests/runtime_env_container/test_log_file_exists.py index 5f6cc2e87c07..4288f1fba4be 100644 --- a/python/ray/tests/runtime_env_container/test_log_file_exists.py +++ b/python/ray/tests/runtime_env_container/test_log_file_exists.py @@ -7,6 +7,11 @@ parser = argparse.ArgumentParser() parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) args = parser.parse_args() worker_pth = get_ray_default_worker_file_path() @@ -27,8 +32,14 @@ def task_finished(): return True +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": worker_pth}} + + # Run a basic workload. -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +@ray.remote(runtime_env=runtime_env) def f(): for i in range(10): print(f"test {i}") diff --git a/python/ray/tests/runtime_env_container/test_put_get.py b/python/ray/tests/runtime_env_container/test_put_get.py index a42fc330f4db..d3d58727068d 100644 --- a/python/ray/tests/runtime_env_container/test_put_get.py +++ b/python/ray/tests/runtime_env_container/test_put_get.py @@ -5,12 +5,23 @@ parser = argparse.ArgumentParser() parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) args = parser.parse_args() worker_pth = get_ray_default_worker_file_path() -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": worker_pth}} + + +@ray.remote(runtime_env=runtime_env) def create_ref(): with open("file.txt") as f: assert f.read().strip() == "helloworldalice" diff --git a/python/ray/tests/runtime_env_container/test_ray_env_vars.py b/python/ray/tests/runtime_env_container/test_ray_env_vars.py index b7288caaa29f..d97b714f895b 100644 --- a/python/ray/tests/runtime_env_container/test_ray_env_vars.py +++ b/python/ray/tests/runtime_env_container/test_ray_env_vars.py @@ -6,17 +6,28 @@ parser = argparse.ArgumentParser() parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) args = parser.parse_args() worker_pth = get_ray_default_worker_file_path() -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": worker_pth}} + + +@ray.remote(runtime_env=runtime_env) def f(): return os.environ.get("RAY_TEST_ABC") -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +@ray.remote(runtime_env=runtime_env) def g(): return os.environ.get("TEST_ABC") diff --git a/python/ray/tests/runtime_env_container/test_serve_basic.py b/python/ray/tests/runtime_env_container/test_serve_basic.py index b9fa3356088d..f74412201834 100644 --- a/python/ray/tests/runtime_env_container/test_serve_basic.py +++ b/python/ray/tests/runtime_env_container/test_serve_basic.py @@ -5,16 +5,22 @@ parser = argparse.ArgumentParser() parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) args = parser.parse_args() WORKER_PATH = get_ray_default_worker_file_path() +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": WORKER_PATH}} -@serve.deployment( - ray_actor_options={ - "runtime_env": {"container": {"image": args.image, "worker_path": WORKER_PATH}} - } -) + +@serve.deployment(ray_actor_options={"runtime_env": runtime_env}) class Model: def __call__(self): with open("file.txt") as f: diff --git a/python/ray/tests/runtime_env_container/test_serve_telemetry.py b/python/ray/tests/runtime_env_container/test_serve_telemetry.py index 8f361cbc93e1..abcc44ab6053 100644 --- a/python/ray/tests/runtime_env_container/test_serve_telemetry.py +++ b/python/ray/tests/runtime_env_container/test_serve_telemetry.py @@ -16,6 +16,11 @@ parser = argparse.ArgumentParser( description="Example Python script taking command line arguments." ) +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") args = parser.parse_args() worker_pth = get_ray_default_worker_file_path() @@ -25,6 +30,12 @@ os.environ["RAY_USAGE_STATS_REPORT_INTERVAL_S"] = "1" +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": worker_pth}} + + def check_app(app_name: str, expected: str): app_handle = serve.get_app_handle(app_name) ref = app_handle.remote() @@ -88,7 +99,7 @@ def check_telemetry_deployment(): { "name": "app1", "import_path": "serve_application:app", - "runtime_env": {"container": {"image": args.image, "worker_path": worker_pth}}, + "runtime_env": runtime_env, }, ) client.deploy_apps(ServeDeploySchema.parse_obj(config)) @@ -104,9 +115,7 @@ def check_telemetry_deployment(): { "name": "Model", "ray_actor_options": { - "runtime_env": { - "container": {"image": args.image, "worker_path": worker_pth} - }, + "runtime_env": runtime_env, }, } ], diff --git a/python/ray/tests/runtime_env_container/test_shared_memory.py b/python/ray/tests/runtime_env_container/test_shared_memory.py index da5260957bf8..bc697eca0f16 100644 --- a/python/ray/tests/runtime_env_container/test_shared_memory.py +++ b/python/ray/tests/runtime_env_container/test_shared_memory.py @@ -7,12 +7,22 @@ parser = argparse.ArgumentParser() parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) args = parser.parse_args() worker_pth = get_ray_default_worker_file_path() +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": worker_pth}} -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) + +@ray.remote(runtime_env=runtime_env) def f(): array = np.random.rand(5000, 5000) return ray.put(array) diff --git a/python/ray/tests/runtime_env_container/test_worker_exit_intended_system_exit_and_user_error.py b/python/ray/tests/runtime_env_container/test_worker_exit_intended_system_exit_and_user_error.py index d7dc1964f771..b3ebde5853bf 100644 --- a/python/ray/tests/runtime_env_container/test_worker_exit_intended_system_exit_and_user_error.py +++ b/python/ray/tests/runtime_env_container/test_worker_exit_intended_system_exit_and_user_error.py @@ -10,9 +10,19 @@ parser = argparse.ArgumentParser() parser.add_argument("--image", type=str, help="The docker image to use for Ray worker") +parser.add_argument( + "--use-image-uri-api", + action="store_true", + help="Whether to use the new `image_uri` API instead of the old `container` API.", +) args = parser.parse_args() + worker_pth = get_ray_default_worker_file_path() +if args.use_image_uri_api: + runtime_env = {"image_uri": args.image} +else: + runtime_env = {"container": {"image": args.image, "worker_path": worker_pth}} ray.init(num_cpus=1) @@ -24,12 +34,12 @@ def get_worker_by_pid(pid, detail=True): assert False -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +@ray.remote(runtime_env=runtime_env) def f(): return ray.get(g.remote()) -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +@ray.remote(runtime_env=runtime_env) def g(): return os.getpid() @@ -54,7 +64,7 @@ def verify_exit_by_idle_timeout(): @ray.remote( num_cpus=1, - runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}, + runtime_env=runtime_env, ) class A: def __init__(self): @@ -94,7 +104,7 @@ def verify_exit_by_pg_removed(): wait_for_condition(verify_exit_by_pg_removed) -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +@ray.remote(runtime_env=runtime_env) class PidDB: def __init__(self): self.pid = None @@ -109,7 +119,7 @@ def get_pid(self): p = PidDB.remote() -@ray.remote(runtime_env={"container": {"image": args.image, "worker_path": worker_pth}}) +@ray.remote(runtime_env=runtime_env) class FaultyActor: def __init__(self): p.record_pid.remote(os.getpid()) diff --git a/python/ray/tests/test_runtime_env_container.py b/python/ray/tests/test_runtime_env_container.py index 721bfa68c97b..963f510479e4 100644 --- a/python/ray/tests/test_runtime_env_container.py +++ b/python/ray/tests/test_runtime_env_container.py @@ -16,43 +16,58 @@ @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") -def test_put_get(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_put_get(podman_docker_cluster, use_image_uri_api): """Test ray.put and ray.get.""" container_id = podman_docker_cluster cmd = ["python", "tests/test_put_get.py", "--image", NESTED_IMAGE_NAME] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") -def test_shared_memory(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_shared_memory(podman_docker_cluster, use_image_uri_api): """Test shared memory.""" container_id = podman_docker_cluster cmd = ["python", "tests/test_shared_memory.py", "--image", NESTED_IMAGE_NAME] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") -def test_log_file_exists(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_log_file_exists(podman_docker_cluster, use_image_uri_api): """Verify worker log file exists""" container_id = podman_docker_cluster cmd = ["python", "tests/test_log_file_exists.py", "--image", NESTED_IMAGE_NAME] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") -def test_ray_env_vars(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_ray_env_vars(podman_docker_cluster, use_image_uri_api): """Test ray.put and ray.get.""" container_id = podman_docker_cluster cmd = ["python", "tests/test_ray_env_vars.py", "--image", NESTED_IMAGE_NAME] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") -def test_worker_exit_intended_system_exit_and_user_error(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_worker_exit_intended_system_exit_and_user_error( + podman_docker_cluster, use_image_uri_api +): """ INTENDED_SYSTEM_EXIT - (not tested, hard to test) Unused resource removed @@ -69,25 +84,33 @@ def test_worker_exit_intended_system_exit_and_user_error(podman_docker_cluster): "--image", NESTED_IMAGE_NAME, ] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") -def test_serve_basic(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_serve_basic(podman_docker_cluster, use_image_uri_api): """Test Serve deployment.""" container_id = podman_docker_cluster cmd = ["python", "tests/test_serve_basic.py", "--image", NESTED_IMAGE_NAME] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) @pytest.mark.skipif(sys.platform != "linux", reason="Only works on Linux.") @pytest.mark.skip -def test_serve_telemetry(podman_docker_cluster): +@pytest.mark.parametrize("use_image_uri_api", [True, False]) +def test_serve_telemetry(podman_docker_cluster, use_image_uri_api): """Test Serve deployment telemetry.""" container_id = podman_docker_cluster cmd = ["python", "tests/test_serve_telemetry.py", "--image", NESTED_IMAGE_NAME] + if use_image_uri_api: + cmd.append("--use-image-uri-api") run_in_container([cmd], container_id) From 3f29274fada445fdbc4fdeb6dc6c0d6d58f24ca7 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Wed, 29 May 2024 19:00:59 +0200 Subject: [PATCH 41/65] [RLlib] Fix wrong `env` being passed into `on_episode_end` callback on MultiAgentEnvRunner when sampling whole episodes. (#45617) --- .../tests/test_callbacks_on_env_runner.py | 56 +++++++++++++++---- rllib/env/multi_agent_env_runner.py | 2 +- 2 files changed, 47 insertions(+), 11 deletions(-) diff --git a/rllib/algorithms/tests/test_callbacks_on_env_runner.py b/rllib/algorithms/tests/test_callbacks_on_env_runner.py index 34329c20bf41..ce7d97444cd3 100644 --- a/rllib/algorithms/tests/test_callbacks_on_env_runner.py +++ b/rllib/algorithms/tests/test_callbacks_on_env_runner.py @@ -1,10 +1,15 @@ from collections import Counter import unittest +import gymnasium as gym + import ray +from ray import tune from ray.rllib.algorithms.callbacks import DefaultCallbacks from ray.rllib.algorithms.ppo import PPOConfig -from ray.rllib.utils.test_utils import framework_iterator +from ray.rllib.env.env_runner import EnvRunner +from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole +from ray.rllib.utils.metrics.metrics_logger import MetricsLogger class EpisodeAndSampleCallbacks(DefaultCallbacks): @@ -12,24 +17,38 @@ def __init__(self): super().__init__() self.counts = Counter() - def on_environment_created(self, *args, **kwargs): + def on_environment_created(self, *args, env_runner, metrics_logger, env, **kwargs): + self.counts.update({"env_created": 1}) - def on_episode_start(self, *args, **kwargs): + def on_episode_start(self, *args, env_runner, metrics_logger, env, **kwargs): + assert isinstance(env_runner, EnvRunner) + assert isinstance(metrics_logger, MetricsLogger) + assert isinstance(env, gym.Env) self.counts.update({"start": 1}) - def on_episode_step(self, *args, **kwargs): + def on_episode_step(self, *args, env_runner, metrics_logger, env, **kwargs): + assert isinstance(env_runner, EnvRunner) + assert isinstance(metrics_logger, MetricsLogger) + assert isinstance(env, gym.Env) self.counts.update({"step": 1}) - def on_episode_end(self, *args, **kwargs): + def on_episode_end(self, *args, env_runner, metrics_logger, env, **kwargs): + assert isinstance(env_runner, EnvRunner) + assert isinstance(metrics_logger, MetricsLogger) + assert isinstance(env, gym.Env) self.counts.update({"end": 1}) - def on_sample_end(self, *args, **kwargs): + def on_sample_end(self, *args, env_runner, metrics_logger, **kwargs): + assert isinstance(env_runner, EnvRunner) + assert isinstance(metrics_logger, MetricsLogger) self.counts.update({"sample": 1}) class OnEnvironmentCreatedCallback(DefaultCallbacks): def on_environment_created(self, *, env_runner, env, env_context, **kwargs): + assert isinstance(env_runner, EnvRunner) + assert isinstance(env, gym.Env) # Create a vector-index-sum property per remote worker. if not hasattr(env_runner, "sum_sub_env_vector_indices"): env_runner.sum_sub_env_vector_indices = 0 @@ -63,6 +82,7 @@ def on_episode_created( class TestCallbacks(unittest.TestCase): @classmethod def setUpClass(cls): + tune.register_env("ma_cart", lambda _: MultiAgentCartPole({"num_agents": 2})) ray.init() @classmethod @@ -88,7 +108,14 @@ def test_episode_and_sample_callbacks_batch_mode_truncate_episodes(self): num_sgd_iter=1, ) ) - for _ in framework_iterator(config, frameworks=("torch", "tf2")): + + for multi_agent in [False, True]: + if multi_agent: + config.multi_agent( + policies={"p0", "p1"}, + policy_mapping_fn=lambda aid, *a, **kw: f"p{aid}", + ) + config.environment("ma_cart") algo = config.build() callback_obj = algo.workers.local_worker()._callbacks @@ -133,7 +160,15 @@ def test_episode_and_sample_callbacks_batch_mode_complete_episodes(self): num_sgd_iter=1, ) ) - for _ in framework_iterator(config, frameworks=("torch", "tf2")): + + for multi_agent in [False, True]: + if multi_agent: + config.multi_agent( + policies={"p0", "p1"}, + policy_mapping_fn=lambda aid, *a, **kw: f"p{aid}", + ) + config.environment("ma_cart") + algo = config.build() callback_obj = algo.workers.local_worker()._callbacks @@ -142,8 +177,9 @@ def test_episode_and_sample_callbacks_batch_mode_complete_episodes(self): # Train one iteration. algo.train() - # We must have has exactly one `sample()` call on our EnvRunner. - self.assertEqual(callback_obj.counts["sample"], 1) + # We must have had exactly one `sample()` call on our EnvRunner. + if not multi_agent: + self.assertEqual(callback_obj.counts["sample"], 1) # We should have had at least one episode start. self.assertGreater(callback_obj.counts["start"], 0) # Episode starts must be exact same as episode ends (b/c we always complete diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index 077adbdad589..e179d047cfc9 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -551,7 +551,7 @@ def _sample_episodes( # but after(!) the last env-to-module connector call has been made. # -> All obs (even the terminal one) should have been processed now (by # the connector, if applicable). - self._make_on_episode_callback("on_episode_end") + self._make_on_episode_callback("on_episode_end", _episode) # Finish the episode. done_episodes_to_return.append( From 14bf327349aa5ab71756501725e200718fb10eb0 Mon Sep 17 00:00:00 2001 From: Lonnie Liu <95255098+aslonnie@users.noreply.github.com> Date: Wed, 29 May 2024 10:45:52 -0700 Subject: [PATCH 42/65] [ci] add security requirements constraint (#45616) for bumping package versions up in the container and dodging cve's also upgrade `idna` and add missing `cupy-cuda11x` package in constraints.. Signed-off-by: Lonnie Liu --- ci/ci.sh | 3 ++- python/requirements/security-requirements.txt | 7 +++++++ python/requirements_compiled.txt | 8 +++++++- 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 python/requirements/security-requirements.txt diff --git a/ci/ci.sh b/ci/ci.sh index d29ee58c3fe0..3a09f7f155b8 100755 --- a/ci/ci.sh +++ b/ci/ci.sh @@ -106,7 +106,8 @@ compile_pip_dependencies() { "${WORKSPACE_DIR}/python/requirements/ml/train-requirements.txt" \ "${WORKSPACE_DIR}/python/requirements/ml/train-test-requirements.txt" \ "${WORKSPACE_DIR}/python/requirements/ml/tune-requirements.txt" \ - "${WORKSPACE_DIR}/python/requirements/ml/tune-test-requirements.txt" + "${WORKSPACE_DIR}/python/requirements/ml/tune-test-requirements.txt" \ + "${WORKSPACE_DIR}/python/requirements/security-requirements.txt" # Remove some pins from upstream dependencies: # ray, xgboost-ray, lightgbm-ray, tune-sklearn diff --git a/python/requirements/security-requirements.txt b/python/requirements/security-requirements.txt new file mode 100644 index 000000000000..c6c562790098 --- /dev/null +++ b/python/requirements/security-requirements.txt @@ -0,0 +1,7 @@ +# Requirement constraints for security. +# +# Only for constraining version ranges. Packages listed might not be installed, +# might not be ray requirements. They are only used in compiling the +# constraint file + +idna>=3.7 diff --git a/python/requirements_compiled.txt b/python/requirements_compiled.txt index d95e738ae605..4eedb59f751f 100644 --- a/python/requirements_compiled.txt +++ b/python/requirements_compiled.txt @@ -372,6 +372,8 @@ cryptography==38.0.1 # python-jose # sshpubkeys # trustme +cupy-cuda11x==13.1.0 ; sys_platform != "darwin" + # via -r /ray/ci/../python/requirements/ml/dl-cpu-requirements.txt cycler==0.12.1 # via matplotlib cython==0.29.37 @@ -496,6 +498,8 @@ fasteners==0.19 # gsutil fastjsonschema==2.19.0 # via nbformat +fastrlock==0.8.2 + # via cupy-cuda11x feather-format==0.4.1 # via -r /ray/ci/../python/requirements/test-requirements.txt ffmpy==0.3.1 @@ -765,8 +769,9 @@ humanfriendly==10.0 # coloredlogs hyperopt==0.2.7 # via -r /ray/ci/../python/requirements/ml/tune-requirements.txt -idna==3.6 +idna==3.7 # via + # -r /ray/ci/../python/requirements/security-requirements.txt # anyio # httpx # jsonschema @@ -1239,6 +1244,7 @@ numpy==1.24.4 # cmdstanpy # configspace # contourpy + # cupy-cuda11x # dask # datasets # deepspeed From 75161ab72f8d76c9365756b5d86f111582bbb42a Mon Sep 17 00:00:00 2001 From: Lonnie Liu <95255098+aslonnie@users.noreply.github.com> Date: Wed, 29 May 2024 12:01:36 -0700 Subject: [PATCH 43/65] [deps] remove duplicated lines in requirements.txt (#45615) some packages are declared more than once. Signed-off-by: Lonnie Liu --- python/requirements.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/python/requirements.txt b/python/requirements.txt index efab173a8344..e937c7352584 100644 --- a/python/requirements.txt +++ b/python/requirements.txt @@ -34,7 +34,6 @@ aiorwlock opentelemetry-exporter-otlp scipy colorful -pyyaml rich opentelemetry-sdk fastapi @@ -47,7 +46,6 @@ dm_tree uvicorn scikit-image==0.21.0 prometheus_client>=0.7.1 -requests pandas tensorboardX aiohttp>=3.7,<3.9 @@ -57,5 +55,4 @@ fsspec pandas>=1.3 pydantic!=2.0.*,!=2.1.*,!=2.2.*,!=2.3.*,!=2.4.*,<3 # Serve users can use pydantic<2 py-spy>=0.2.0 -watchfiles memray; sys_platform != "win32" # memray is not supported on Windows From 8752932b2505e4663619499aab44845a72f28ef8 Mon Sep 17 00:00:00 2001 From: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> Date: Wed, 29 May 2024 13:38:01 -0700 Subject: [PATCH 44/65] [ADAG] Support multi-args and kwargs in Accelerate DAG (#45545) This PR adds multi-arg and kwarg support by serializing all positional args and kwargs and passing it through the channel. When the channel is read at runtime, the individual args are extracted first before passing to the consuming tasks. Closes #42793 --------- Signed-off-by: Rui Qiao Signed-off-by: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> --- python/ray/dag/compiled_dag_node.py | 247 ++++++++++++------ .../experimental/test_accelerated_dag.py | 173 ++++++++++-- 2 files changed, 314 insertions(+), 106 deletions(-) diff --git a/python/ray/dag/compiled_dag_node.py b/python/ray/dag/compiled_dag_node.py index e0cb598d9444..d53472f4a1d6 100644 --- a/python/ray/dag/compiled_dag_node.py +++ b/python/ray/dag/compiled_dag_node.py @@ -106,15 +106,6 @@ def _prep_task(self, task: "ExecutableTask") -> None: """ Prepare the task for execution. """ - # Add placeholders for input channels. - for idx, inp in enumerate(task.resolved_args): - if isinstance(inp, ChannelInterface): - task.input_channels.append(inp) - task.input_channel_idxs.append(idx) - task.resolved_inputs.append(None) - else: - task.resolved_inputs.append(inp) - for typ_hint in task.input_type_hints: typ_hint.register_custom_serializer() task.output_type_hint.register_custom_serializer() @@ -158,11 +149,12 @@ def _exec_task(self, task: "ExecutableTask", idx: int) -> bool: input_reader.end_read() return False - for idx, output in zip(task.input_channel_idxs, res): - task.resolved_inputs[idx] = output + resolved_inputs = [] + for task_input in task.task_inputs: + resolved_inputs.append(task_input.resolve(res)) try: - output_val = method(*task.resolved_inputs) + output_val = method(*resolved_inputs) output_writer.write(output_val) except Exception as exc: output_writer.write(_wrap_exception(exc)) @@ -243,6 +235,84 @@ def __str__(self) -> str: """ +@DeveloperAPI +class DAGInputAdapter: + """Adapter to extract individual positional arguments and kwargs + from objects read from DAG input channel.""" + + def __init__( + self, + input_attr_node: Optional["ray.dag.InputAttributeNode"], + dag_input_channel: "ray.experimental.channel.ChannelInterface", + ): + """ + Args: + input_attr_node: The input attribute node that this adapter is + created for. None should be used when creating an adapter for + the DAG input node itself; in this case, the adapter will + extract the 0th positional argument. + dag_input_channel: The DAG input channel. + """ + self._dag_input_channel = dag_input_channel + + def extractor(key: Union[int, str]): + def extract_arg(args_tuple): + positional_args, kwargs = args_tuple + if isinstance(key, int): + return positional_args[key] + else: + return kwargs[key] + + return extract_arg + + if input_attr_node: + key = input_attr_node.get_other_args_to_resolve()["key"] + else: + key = 0 + self._adapt_method = extractor(key) + + def adapt(self, input): + return self._adapt_method(input) + + def get_dag_input_channel(self): + return self._dag_input_channel + + +class _ExecutableTaskInput: + """Represents an input to an ExecutableTask. + + Args: + input_variant: either an unresolved input (when type is ChannelInterface + or DAGInputAdapter), or a resolved input value (when type is Any) + channel_idx: if input_variant is an unresolved input, this is the index + into the input channels list. + """ + + def __init__( + self, + input_variant: Union[ChannelInterface, DAGInputAdapter, Any], + channel_idx: Optional[int], + ): + self.input_variant = input_variant + self.channel_idx = channel_idx + + def resolve(self, channel_results: Any): + """ + Resolve the input value from the channel results. + + Args: + channel_results: The results from reading the input channels. + """ + if isinstance(self.input_variant, ChannelInterface): + value = channel_results[self.channel_idx] + elif isinstance(self.input_variant, DAGInputAdapter): + adapter = self.input_variant + value = adapter.adapt(channel_results[self.channel_idx]) + else: + value = self.input_variant + return value + + @DeveloperAPI class ExecutableTask: """A task that can be executed in a compiled DAG, and it @@ -265,13 +335,37 @@ def __init__( self.method_name = task.dag_node.get_method_name() self.bind_index = task.dag_node._get_bind_index() self.output_channel = task.output_channel - self.resolved_args = resolved_args self.input_type_hints: List["ChannelOutputType"] = task.arg_type_hints self.output_type_hint: "ChannelOutputType" = task.dag_node.type_hint - self.resolved_inputs: List[Union[Any, ChannelInterface]] = [] self.input_channels: List[ChannelInterface] = [] - self.input_channel_idxs: List[int] = [] + self.task_inputs: List[_ExecutableTaskInput] = [] + + # Reverse map for input_channels: maps an input channel to + # its index in input_channels. + input_channel_to_idx: dict[ChannelInterface, int] = {} + + for arg in resolved_args: + if isinstance(arg, ChannelInterface) or isinstance(arg, DAGInputAdapter): + if isinstance(arg, ChannelInterface): + channel = arg + else: + adapter = arg + channel = adapter.get_dag_input_channel() + + if channel in input_channel_to_idx: + # The same channel was added before, so reuse the index. + channel_idx = input_channel_to_idx[channel] + else: + # Add a new channel to the list of input channels. + self.input_channels.append(channel) + channel_idx = len(self.input_channels) - 1 + input_channel_to_idx[channel] = channel_idx + + task_input = _ExecutableTaskInput(arg, channel_idx) + else: + task_input = _ExecutableTaskInput(arg, None) + self.task_inputs.append(task_input) @DeveloperAPI @@ -398,22 +492,28 @@ def _preprocess(self) -> None: nccl_actors: Set["ray.actor.ActorHandle"] = set() + # Find the input node to the DAG. + for idx, task in self.idx_to_task.items(): + if isinstance(task.dag_node, InputNode): + assert self.input_task_idx is None, "more than one InputNode found" + self.input_task_idx = idx + # TODO: Support no-input DAGs (use an empty object to signal). + if self.input_task_idx is None: + raise NotImplementedError( + "Compiled DAGs currently require exactly one InputNode" + ) + # For each task node, set its upstream and downstream task nodes. # Also collect the set of tasks that produce torch.tensors. for node_idx, task in self.idx_to_task.items(): dag_node = task.dag_node if not ( isinstance(dag_node, InputNode) + or isinstance(dag_node, InputAttributeNode) or isinstance(dag_node, MultiOutputNode) or isinstance(dag_node, ClassMethodNode) ): - if isinstance(dag_node, InputAttributeNode): - # TODO(swang): Support multi args. - raise NotImplementedError( - "Compiled DAGs currently do not support kwargs or " - "multiple args for InputNode" - ) - elif isinstance(dag_node, FunctionNode): + if isinstance(dag_node, FunctionNode): # TODO(swang): Support non-actor tasks. raise NotImplementedError( "Compiled DAGs currently only support actor method nodes" @@ -456,6 +556,12 @@ def _preprocess(self) -> None: downstream_actor_handle = None if isinstance(task.dag_node, ClassMethodNode): downstream_actor_handle = task.dag_node._get_actor_handle() + + # If the upstream node is an InputAttributeNode, treat the + # DAG's input node as the actual upstream node + if isinstance(upstream_node.dag_node, InputAttributeNode): + upstream_node = self.idx_to_task[self.input_task_idx] + upstream_node.downstream_node_idxs[node_idx] = downstream_actor_handle task.arg_type_hints.append(upstream_node.dag_node.type_hint) @@ -466,19 +572,12 @@ def _preprocess(self) -> None: if dag_node.type_hint is not None: self._type_hints.append(dag_node.type_hint) - # Find the input node to the DAG. - for idx, task in self.idx_to_task.items(): - if isinstance(task.dag_node, InputNode): - assert self.input_task_idx is None, "more than one InputNode found" - self.input_task_idx = idx - # TODO: Support no-input DAGs (use an empty object to signal). - if self.input_task_idx is None: - raise NotImplementedError( - "Compiled DAGs currently require exactly one InputNode" - ) - # Find the (multi-)output node to the DAG. for idx, task in self.idx_to_task.items(): + if idx == self.input_task_idx or isinstance( + task.dag_node, InputAttributeNode + ): + continue if len(task.downstream_node_idxs) == 0: assert self.output_task_idx is None, "More than one output node found" self.output_task_idx = idx @@ -516,7 +615,13 @@ def _get_or_compile( _dag_output_fetcher will be set and can be used to invoke and fetch outputs for the DAG. """ - from ray.dag import DAGNode, InputNode, MultiOutputNode, ClassMethodNode + from ray.dag import ( + DAGNode, + InputNode, + InputAttributeNode, + MultiOutputNode, + ClassMethodNode, + ) if self.input_task_idx is None: self._preprocess() @@ -582,35 +687,32 @@ def _get_node_id(self): self.actor_refs.add(actor_handle) self.actor_to_tasks[actor_handle].append(task) elif isinstance(task.dag_node, InputNode): - readers = [self.idx_to_task[idx] for idx in task.downstream_node_idxs] - reader_handles = [] reader_handles_set = set() - for reader in readers: - reader_handle = reader.dag_node._get_actor_handle() - if reader_handle in reader_handles_set: - raise NotImplementedError( - "Compiled DAGs currently do not support binding the " - "same input on the same actor multiple times. " - f"Violating actor: {reader_handle}" - ) + for idx in task.downstream_node_idxs: + reader_task = self.idx_to_task[idx] + assert isinstance(reader_task.dag_node, ClassMethodNode) + reader_handle = reader_task.dag_node._get_actor_handle() reader_handles_set.add(reader_handle) - reader_handles.append(reader_handle) task.output_channel = do_allocate_channel( self, - reader_handles, + list(reader_handles_set), typ=type_hint, ) else: - assert isinstance(task.dag_node, MultiOutputNode) + assert isinstance(task.dag_node, InputAttributeNode) or isinstance( + task.dag_node, MultiOutputNode + ) for idx in task.downstream_node_idxs: frontier.append(idx) # Validate input channels for tasks that have not been visited for node_idx, task in self.idx_to_task.items(): - if node_idx == self.input_task_idx: - continue - if node_idx == self.output_task_idx: + if ( + node_idx == self.input_task_idx + or node_idx == self.output_task_idx + or isinstance(task.dag_node, InputAttributeNode) + ): continue if node_idx not in visited: has_at_least_one_channel_input = False @@ -623,6 +725,11 @@ def _get_node_id(self): "or at least one other DAGNode as an input" ) + input_task = self.idx_to_task[self.input_task_idx] + # Register custom serializers for inputs provided to dag.execute(). + input_task.dag_node.type_hint.register_custom_serializer() + self.dag_input_channel = input_task.output_channel + # Create executable tasks for each actor for actor_handle, tasks in self.actor_to_tasks.items(): executable_tasks = [] @@ -631,7 +738,15 @@ def _get_node_id(self): resolved_args = [] has_at_least_one_channel_input = False for arg in task.args: - if isinstance(arg, DAGNode): + if isinstance(arg, InputNode): + input_adapter = DAGInputAdapter(None, self.dag_input_channel) + resolved_args.append(input_adapter) + has_at_least_one_channel_input = True + elif isinstance(arg, InputAttributeNode): + input_adapter = DAGInputAdapter(arg, self.dag_input_channel) + resolved_args.append(input_adapter) + has_at_least_one_channel_input = True + elif isinstance(arg, DAGNode): # Other DAGNodes arg_idx = self.dag_node_to_idx[arg] arg_channel = self.idx_to_task[arg_idx].output_channel assert arg_channel is not None @@ -666,12 +781,6 @@ def _get_node_id(self): executable_tasks, ) - input_task = self.idx_to_task[self.input_task_idx] - # Register custom serializers for inputs provided to dag.execute(). - input_task.dag_node.type_hint.register_custom_serializer() - - self.dag_input_channel = input_task.output_channel - self.dag_output_channels = [] for output in self.idx_to_task[self.output_task_idx].args: assert isinstance(output, DAGNode) @@ -708,7 +817,6 @@ def _get_node_id(self): self._dag_submitter.start() self._dag_output_fetcher.start() - return def _monitor_failures(self): outer = self @@ -808,24 +916,17 @@ def execute( Args: args: Args to the InputNode. - kwargs: Kwargs to the InputNode. Not supported yet. + kwargs: Kwargs to the InputNode Returns: A list of Channels that can be used to read the DAG result. """ - # These errors should already be caught during compilation, but just in - # case. - if len(args) != 1: - raise NotImplementedError("Compiled DAGs support exactly one InputNode arg") - if len(kwargs) != 0: - raise NotImplementedError("Compiled DAGs do not support kwargs") - if self._enable_asyncio: raise ValueError("Use execute_async if enable_asyncio=True") self._get_or_compile() - inp = args[0] + inp = (args, kwargs) self._dag_submitter.write(inp) return self._dag_output_fetcher @@ -841,25 +942,17 @@ async def execute_async( Args: args: Args to the InputNode. - kwargs: Kwargs to the InputNode. Not supported yet. + kwargs: Kwargs to the InputNode. Returns: A list of Channels that can be used to read the DAG result. """ - # These errors should already be caught during compilation, but just in - # case. - if len(args) != 1: - raise NotImplementedError("Compiled DAGs support exactly one InputNode arg") - if len(kwargs) != 0: - raise NotImplementedError("Compiled DAGs do not support kwargs") - if not self._enable_asyncio: raise ValueError("Use execute if enable_asyncio=False") self._get_or_compile() async with self._dag_submission_lock: - inp = args[0] - + inp = (args, kwargs) await self._dag_submitter.write(inp) # Allocate a future that the caller can use to get the result. fut = asyncio.Future() diff --git a/python/ray/dag/tests/experimental/test_accelerated_dag.py b/python/ray/dag/tests/experimental/test_accelerated_dag.py index 2f6f9b4a2d46..1923deaafb56 100644 --- a/python/ray/dag/tests/experimental/test_accelerated_dag.py +++ b/python/ray/dag/tests/experimental/test_accelerated_dag.py @@ -71,6 +71,27 @@ def sleep(self, x): return x +@ray.remote +class Collector: + def __init__(self): + self.results = [] + + def collect(self, x): + self.results.append(x) + return self.results + + def collect_two(self, x, y): + self.results.append(x) + self.results.append(y) + return self.results + + def collect_three(self, x, y, z): + self.results.append(x) + self.results.append(y) + self.results.append(z) + return self.results + + def test_basic(ray_start_regular): a = Actor.remote(0) with InputNode() as i: @@ -175,6 +196,129 @@ def test_regular_args(ray_start_regular): compiled_dag.teardown() +def test_multi_args_basic(ray_start_regular): + a1 = Actor.remote(0) + a2 = Actor.remote(0) + c = Collector.remote() + with InputNode() as i: + branch1 = a1.inc.bind(i[0]) + branch2 = a2.inc.bind(i[1]) + dag = c.collect_two.bind(branch2, branch1) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute(2, 3) + result = output_channel.begin_read() + assert result == [3, 2] + output_channel.end_read() + + compiled_dag.teardown() + + +def test_multi_args_single_actor(ray_start_regular): + c = Collector.remote() + with InputNode() as i: + dag = c.collect_two.bind(i[1], i[0]) + + compiled_dag = dag.experimental_compile() + + for i in range(3): + output_channel = compiled_dag.execute(2, 3) + result = output_channel.begin_read() + assert result == [3, 2] * (i + 1) + output_channel.end_read() + + compiled_dag.teardown() + + +def test_multi_args_branch(ray_start_regular): + a = Actor.remote(0) + c = Collector.remote() + with InputNode() as i: + branch = a.inc.bind(i[0]) + dag = c.collect_two.bind(branch, i[1]) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute(2, 3) + result = output_channel.begin_read() + assert result == [2, 3] + output_channel.end_read() + + compiled_dag.teardown() + + +def test_kwargs_basic(ray_start_regular): + a1 = Actor.remote(0) + a2 = Actor.remote(0) + c = Collector.remote() + with InputNode() as i: + branch1 = a1.inc.bind(i.x) + branch2 = a2.inc.bind(i.y) + dag = c.collect_two.bind(branch2, branch1) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute(x=2, y=3) + result = output_channel.begin_read() + assert result == [3, 2] + output_channel.end_read() + + compiled_dag.teardown() + + +def test_kwargs_single_actor(ray_start_regular): + c = Collector.remote() + with InputNode() as i: + dag = c.collect_two.bind(i.y, i.x) + + compiled_dag = dag.experimental_compile() + + for i in range(3): + output_channel = compiled_dag.execute(x=2, y=3) + result = output_channel.begin_read() + assert result == [3, 2] * (i + 1) + output_channel.end_read() + + compiled_dag.teardown() + + +def test_kwargs_branch(ray_start_regular): + a = Actor.remote(0) + c = Collector.remote() + with InputNode() as i: + branch = a.inc.bind(i.x) + dag = c.collect_two.bind(i.y, branch) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute(x=2, y=3) + result = output_channel.begin_read() + assert result == [3, 2] + output_channel.end_read() + + compiled_dag.teardown() + + +def test_multi_args_and_kwargs(ray_start_regular): + a1 = Actor.remote(0) + a2 = Actor.remote(0) + c = Collector.remote() + with InputNode() as i: + branch1 = a1.inc.bind(i[0]) + branch2 = a2.inc.bind(i.y) + dag = c.collect_three.bind(branch2, i.z, branch1) + + compiled_dag = dag.experimental_compile() + + output_channel = compiled_dag.execute(2, y=3, z=4) + result = output_channel.begin_read() + assert result == [3, 4, 2] + output_channel.end_read() + + compiled_dag.teardown() + + @pytest.mark.parametrize("num_actors", [1, 4]) def test_scatter_gather_dag(ray_start_regular, num_actors): actors = [Actor.remote(0) for _ in range(num_actors)] @@ -252,17 +396,6 @@ def test_dag_errors(ray_start_regular): ): dag.experimental_compile() - with InputNode() as inp: - dag = a.inc.bind(inp) - dag2 = a.inc.bind(inp) - dag3 = a.inc_two.bind(dag, dag2) - with pytest.raises( - NotImplementedError, - match=r"Compiled DAGs currently do not support binding the same input " - "on the same actor multiple times.*", - ): - dag3.experimental_compile() - @ray.remote def f(x): return x @@ -275,24 +408,6 @@ def f(x): ): dag.experimental_compile() - with InputNode() as inp: - dag = a.inc_two.bind(inp[0], inp[1]) - with pytest.raises( - NotImplementedError, - match="Compiled DAGs currently do not support kwargs or multiple args " - "for InputNode", - ): - dag.experimental_compile() - - with InputNode() as inp: - dag = a.inc_two.bind(inp.x, inp.y) - with pytest.raises( - NotImplementedError, - match="Compiled DAGs currently do not support kwargs or multiple args " - "for InputNode", - ): - dag.experimental_compile() - def test_dag_fault_tolerance_chain(ray_start_regular_shared): actors = [ From d76518fe89a6877ed4e858b1d55746fd2a1e9e30 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Wed, 29 May 2024 14:01:54 -0700 Subject: [PATCH 45/65] [Core] Improve doc for --object-store-memory to describe how the default value is set (#45301) Currently it's unclear how the default value is set Signed-off-by: Jiajun Yao --- python/ray/_private/worker.py | 8 ++++++-- python/ray/scripts/scripts.py | 5 ++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index cb6a8871d4b5..d4bba87fc4e8 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -1302,8 +1302,12 @@ def init( quantities for them available. labels: [Experimental] The key-value labels of the node. object_store_memory: The amount of memory (in bytes) to start the - object store with. By default, this is automatically set based on - available system memory. + object store with. + By default, this is 30% + (ray_constants.DEFAULT_OBJECT_STORE_MEMORY_PROPORTION) + of available system memory capped by + the shm size and 200G (ray_constants.DEFAULT_OBJECT_STORE_MAX_MEMORY_BYTES) + but can be set higher. local_mode: Deprecated: consider using the Ray Debugger instead. ignore_reinit_error: If true, Ray suppresses errors from calling ray.init() a second time. Ray won't be restarted. diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 65158ea3f6c9..6ed5fabea435 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -367,7 +367,10 @@ def debug(address): required=False, type=int, help="The amount of memory (in bytes) to start the object store with. " - "By default, this is capped at 20GB but can be set higher.", + "By default, this is 30% (ray_constants.DEFAULT_OBJECT_STORE_MEMORY_PROPORTION) " + "of available system memory capped by " + "the shm size and 200G (ray_constants.DEFAULT_OBJECT_STORE_MAX_MEMORY_BYTES) " + "but can be set higher.", ) @click.option( "--redis-max-memory", From 3518518828de760f2ffb79f494ca6b26fb389bca Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Wed, 29 May 2024 14:25:58 -0700 Subject: [PATCH 46/65] [ci] do not clean up release test data (#45627) This code path deletes the release test working directory upon the job completion. We found repetitive cases where users want the data to be available for debugging purpose. Let's rely on s3 policy to clean up the data after a few days. Test: - CI Signed-off-by: can --- .../ray_release/command_runner/anyscale_job_runner.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/release/ray_release/command_runner/anyscale_job_runner.py b/release/ray_release/command_runner/anyscale_job_runner.py index 223ca7f093ea..893aa9df5081 100644 --- a/release/ray_release/command_runner/anyscale_job_runner.py +++ b/release/ray_release/command_runner/anyscale_job_runner.py @@ -349,9 +349,7 @@ def fetch_output(self) -> Dict[str, Any]: ) def cleanup(self): - try: - self.file_manager.delete(self.path_in_bucket, recursive=True) - except Exception: - # No big deal if we don't clean up, the bucket - # is set to automatically expire objects anyway - pass + # We piggy back on s3 retention policy for clean up instead of doing this + # ourselves. We find many cases where users want the data to be available + # for a short-while for debugging purpose. + pass From 0bb2600cbad5b5e4c7754a32f16f82133c83d943 Mon Sep 17 00:00:00 2001 From: Alan Guo Date: Wed, 29 May 2024 15:09:04 -0700 Subject: [PATCH 47/65] Fail gracefully when cluster status is not available yet. (#45620) --- .../src/components/AutoscalerStatusCards.tsx | 53 +++++++++++-------- dashboard/client/src/pages/job/JobDetail.tsx | 6 +-- .../src/pages/job/hook/useClusterStatus.ts | 4 +- .../src/pages/overview/OverviewPage.tsx | 6 +-- 4 files changed, 40 insertions(+), 29 deletions(-) diff --git a/dashboard/client/src/components/AutoscalerStatusCards.tsx b/dashboard/client/src/components/AutoscalerStatusCards.tsx index 80eeb9b73200..544e44467a79 100644 --- a/dashboard/client/src/components/AutoscalerStatusCards.tsx +++ b/dashboard/client/src/components/AutoscalerStatusCards.tsx @@ -2,38 +2,53 @@ import { Box, Typography } from "@mui/material"; import React from "react"; import { RayStatusResp } from "../service/status"; -const formatNodeStatus = (cluster_status: string) => { +const formatNodeStatus = (clusterStatus?: string) => { // ==== auto scaling status // Node status // .... // Resources // .... - const sections = cluster_status.split("Resources"); - return formatClusterStatus( - "Node Status", - sections[0].split("Node status")[1], - ); + if (!clusterStatus) { + return "No cluster status."; + } + try { + // Try to parse the node status. + const sections = clusterStatus.split("Resources"); + return formatClusterStatus( + "Node Status", + sections[0].split("Node status")[1], + ); + } catch (e) { + return "No cluster status."; + } }; -const formatResourcesStatus = (cluster_status: string) => { +const formatResourcesStatus = (clusterStatus?: string) => { // ==== auto scaling status // Node status // .... // Resources // .... - const sections = cluster_status.split("Resources"); - return formatClusterStatus("Resource Status", sections[1]); + if (!clusterStatus) { + return "No cluster status."; + } + try { + const sections = clusterStatus.split("Resources"); + return formatClusterStatus("Resource Status", sections[1]); + } catch (e) { + return "No cluster status."; + } }; -const formatClusterStatus = (title: string, cluster_status: string) => { - const cluster_status_rows = cluster_status.split("\n"); +const formatClusterStatus = (title: string, clusterStatus: string) => { + const clusterStatusRows = clusterStatus.split("\n"); return (
{title} - {cluster_status_rows.map((i, key) => { + {clusterStatusRows.map((i, key) => { // Format the output. // See format_info_string in util.py if (i.startsWith("-----") || i.startsWith("=====") || i === "") { @@ -54,10 +69,10 @@ const formatClusterStatus = (title: string, cluster_status: string) => { }; type StatusCardProps = { - cluster_status: RayStatusResp | undefined; + clusterStatus: RayStatusResp | undefined; }; -export const NodeStatusCard = ({ cluster_status }: StatusCardProps) => { +export const NodeStatusCard = ({ clusterStatus }: StatusCardProps) => { return ( { overflowY: "scroll", }} > - {cluster_status?.data && cluster_status?.data?.clusterStatus - ? formatNodeStatus(cluster_status?.data.clusterStatus) - : "No cluster status."} + {formatNodeStatus(clusterStatus?.data.clusterStatus)} ); }; -export const ResourceStatusCard = ({ cluster_status }: StatusCardProps) => { +export const ResourceStatusCard = ({ clusterStatus }: StatusCardProps) => { return ( { overflowY: "scroll", }} > - {cluster_status?.data && cluster_status?.data?.clusterStatus - ? formatResourcesStatus(cluster_status?.data.clusterStatus) - : "No cluster status."} + {formatResourcesStatus(clusterStatus?.data.clusterStatus)} ); }; diff --git a/dashboard/client/src/pages/job/JobDetail.tsx b/dashboard/client/src/pages/job/JobDetail.tsx index 1d7231e26544..8853bdf69071 100644 --- a/dashboard/client/src/pages/job/JobDetail.tsx +++ b/dashboard/client/src/pages/job/JobDetail.tsx @@ -55,7 +55,7 @@ export const JobDetailChartsPage = () => { const [actorListFilter, setActorListFilter] = useState(); const [actorTableExpanded, setActorTableExpanded] = useState(false); const actorTableRef = useRef(null); - const { cluster_status } = useRayStatus(); + const { clusterStatus } = useRayStatus(); const { data } = useSWR( job?.job_id ? ["useDataDatasets", job.job_id] : null, @@ -182,10 +182,10 @@ export const JobDetailChartsPage = () => { >
- +
- +
diff --git a/dashboard/client/src/pages/job/hook/useClusterStatus.ts b/dashboard/client/src/pages/job/hook/useClusterStatus.ts index 61ce605f14b0..ec34f1ca072b 100644 --- a/dashboard/client/src/pages/job/hook/useClusterStatus.ts +++ b/dashboard/client/src/pages/job/hook/useClusterStatus.ts @@ -3,7 +3,7 @@ import { API_REFRESH_INTERVAL_MS } from "../../../common/constants"; import { getRayStatus } from "../../../service/status"; export const useRayStatus = () => { - const { data: cluster_status } = useSWR( + const { data: clusterStatus } = useSWR( "useClusterStatus", async () => { try { @@ -19,6 +19,6 @@ export const useRayStatus = () => { ); return { - cluster_status, + clusterStatus, }; }; diff --git a/dashboard/client/src/pages/overview/OverviewPage.tsx b/dashboard/client/src/pages/overview/OverviewPage.tsx index 3065be699ec0..4a5b4455e367 100644 --- a/dashboard/client/src/pages/overview/OverviewPage.tsx +++ b/dashboard/client/src/pages/overview/OverviewPage.tsx @@ -52,7 +52,7 @@ const useStyles = makeStyles((theme) => export const OverviewPage = () => { const classes = useStyles(); - const { cluster_status } = useRayStatus(); + const { clusterStatus } = useRayStatus(); return (
@@ -80,7 +80,7 @@ export const OverviewPage = () => { classes.autoscalerCard, )} > - + { classes.autoscalerCard, )} > - +
} From dd7cbee11767a936a24617fea727f205b33680fb Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Wed, 29 May 2024 17:29:15 -0700 Subject: [PATCH 48/65] [ci] remove 3.8 from ray supported python list (#45622) Notice that we haven't removed this support completely once I work on upgrading python 3.12. Need to change some runtime environment to `oss-ci-base_build` since `forge` is using python 3.8. Test: - CI Signed-off-by: can --- .buildkite/lint.rayci.yml | 3 ++- python/setup.py | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.buildkite/lint.rayci.yml b/.buildkite/lint.rayci.yml index 1bb5266883d9..338584e4737f 100644 --- a/.buildkite/lint.rayci.yml +++ b/.buildkite/lint.rayci.yml @@ -24,9 +24,10 @@ steps: key: lint-medium instance_type: medium depends_on: - - forge + - oss-ci-base_build commands: - ./ci/lint/lint.sh {{matrix}} + job_env: oss-ci-base_build matrix: - api_annotations - api_discrepancy diff --git a/python/setup.py b/python/setup.py index 8c1d573f5109..22d2edaaa909 100644 --- a/python/setup.py +++ b/python/setup.py @@ -27,7 +27,7 @@ logger = logging.getLogger(__name__) -SUPPORTED_PYTHONS = [(3, 8), (3, 9), (3, 10), (3, 11)] +SUPPORTED_PYTHONS = [(3, 9), (3, 10), (3, 11)] # When the bazel version is updated, make sure to update it # in WORKSPACE file as well. From 5faf4761e9505aae7a83319630877eb99bba467d Mon Sep 17 00:00:00 2001 From: Hao Chen Date: Wed, 29 May 2024 20:04:46 -0700 Subject: [PATCH 49/65] [data] Refactor resource manager (#45623) Refactor ResourceManager and avoid it directly depending on concrete operators. --------- Signed-off-by: Hao Chen --- .../execution/interfaces/physical_operator.py | 15 ++++ .../execution/operators/input_data_buffer.py | 3 + .../execution/operators/limit_operator.py | 3 + .../execution/operators/map_operator.py | 3 + .../execution/operators/output_splitter.py | 3 + .../_internal/execution/resource_manager.py | 87 +++++++++---------- 6 files changed, 66 insertions(+), 48 deletions(-) diff --git a/python/ray/data/_internal/execution/interfaces/physical_operator.py b/python/ray/data/_internal/execution/interfaces/physical_operator.py index 739262ef2b80..06a61e7b0b3d 100644 --- a/python/ray/data/_internal/execution/interfaces/physical_operator.py +++ b/python/ray/data/_internal/execution/interfaces/physical_operator.py @@ -426,3 +426,18 @@ def notify_in_task_submission_backpressure(self, in_backpressure: bool) -> None: def get_autoscaling_actor_pools(self) -> List[AutoscalingActorPool]: """Return a list of `AutoscalingActorPool`s managed by this operator.""" return [] + + def implements_accurate_memory_accounting(self) -> bool: + """Return whether this operator implements accurate memory accounting. + + An operator that implements accurate memory accounting should should properly + report its memory usage via the following APIs: + - `self._metrics.on_input_queued`. + - `self._metrics.on_input_dequeued`. + - `self._metrics.on_output_queued`. + - `self._metrics.on_output_dequeued`. + """ + # TODO(hchen): Currently we only enable `ReservationOpResourceAllocator` when + # all operators in the dataset have implemented accurate memory accounting. + # Eventually all operators should implement accurate memory accounting. + return False diff --git a/python/ray/data/_internal/execution/operators/input_data_buffer.py b/python/ray/data/_internal/execution/operators/input_data_buffer.py index cc0447d8d26e..1236093d4130 100644 --- a/python/ray/data/_internal/execution/operators/input_data_buffer.py +++ b/python/ray/data/_internal/execution/operators/input_data_buffer.py @@ -84,3 +84,6 @@ def _initialize_metadata(self): self._stats = { "input": block_metadata, } + + def implements_accurate_memory_accounting(self) -> bool: + return True diff --git a/python/ray/data/_internal/execution/operators/limit_operator.py b/python/ray/data/_internal/execution/operators/limit_operator.py index d021b3db112c..7db0590960da 100644 --- a/python/ray/data/_internal/execution/operators/limit_operator.py +++ b/python/ray/data/_internal/execution/operators/limit_operator.py @@ -120,3 +120,6 @@ def num_outputs_total(self) -> int: def throttling_disabled(self) -> bool: return True + + def implements_accurate_memory_accounting(self) -> bool: + return True diff --git a/python/ray/data/_internal/execution/operators/map_operator.py b/python/ray/data/_internal/execution/operators/map_operator.py index 0c8023ef5397..eb96b3622146 100644 --- a/python/ray/data/_internal/execution/operators/map_operator.py +++ b/python/ray/data/_internal/execution/operators/map_operator.py @@ -402,6 +402,9 @@ def base_resource_usage(self) -> ExecutionResources: def incremental_resource_usage(self) -> ExecutionResources: raise NotImplementedError + def implements_accurate_memory_accounting(self) -> bool: + return True + def _map_task( map_transformer: MapTransformer, diff --git a/python/ray/data/_internal/execution/operators/output_splitter.py b/python/ray/data/_internal/execution/operators/output_splitter.py index b99a9e0b6847..af7a3429d04e 100644 --- a/python/ray/data/_internal/execution/operators/output_splitter.py +++ b/python/ray/data/_internal/execution/operators/output_splitter.py @@ -241,6 +241,9 @@ def _get_location(self, bundle: RefBundle) -> Optional[NodeIdStr]: """ return bundle.get_cached_location() + def implements_accurate_memory_accounting(self) -> bool: + return True + def _split(bundle: RefBundle, left_size: int) -> Tuple[RefBundle, RefBundle]: left_blocks, left_meta = [], [] diff --git a/python/ray/data/_internal/execution/resource_manager.py b/python/ray/data/_internal/execution/resource_manager.py index 26a6cf3681ac..6b63f0bf0968 100644 --- a/python/ray/data/_internal/execution/resource_manager.py +++ b/python/ray/data/_internal/execution/resource_manager.py @@ -12,9 +12,6 @@ ) from ray.data._internal.execution.interfaces.physical_operator import PhysicalOperator from ray.data._internal.execution.operators.input_data_buffer import InputDataBuffer -from ray.data._internal.execution.operators.limit_operator import LimitOperator -from ray.data._internal.execution.operators.map_operator import MapOperator -from ray.data._internal.execution.operators.output_splitter import OutputSplitter from ray.data._internal.execution.util import memory_string from ray.data.context import DataContext @@ -42,15 +39,6 @@ class ResourceManager: # when `ReservationOpResourceAllocator` is not enabled. DEFAULT_OBJECT_STORE_MEMORY_LIMIT_FRACTION_WO_RESOURCE_RESERVATION = 0.25 - # Memory accounting is accurate only for these operators. - # We'll enable memory reservation if a dataset only contains these operators. - _ACCURRATE_MEMORY_ACCOUNTING_OPS = ( - InputDataBuffer, - MapOperator, - LimitOperator, - OutputSplitter, - ) - def __init__(self, topology: "Topology", options: ExecutionOptions): self._topology = topology self._options = options @@ -75,11 +63,11 @@ def __init__(self, topology: "Topology", options: ExecutionOptions): ctx = DataContext.get_current() if ctx.op_resource_reservation_enabled: - should_enable = True - for op in topology: - if not isinstance(op, ResourceManager._ACCURRATE_MEMORY_ACCOUNTING_OPS): - should_enable = False - break + # We'll enable memory reservation if all operators have + # implemented accurate memory accounting. + should_enable = all( + op.implements_accurate_memory_accounting() for op in topology + ) if should_enable: self._op_resource_allocator = ReservationOpResourceAllocator( self, ctx.op_resource_reservation_ratio @@ -213,7 +201,7 @@ def get_downstream_fraction(self, op: PhysicalOperator) -> float: """Return the downstream fraction of the given operator.""" return self._downstream_fraction[op] - def get_downstream_object_store_memory(self, op: PhysicalOperator) -> int: + def get_downstream_object_store_memory(self, op: PhysicalOperator) -> float: """Return the downstream object store memory usage of the given operator.""" return self._downstream_object_store_memory[op] @@ -240,7 +228,7 @@ def __init__(self, resource_manager: ResourceManager): self._resource_manager = resource_manager @abstractmethod - def update_usages(self) -> ExecutionResources: + def update_usages(self): """Callback to update resource usages.""" ... @@ -259,20 +247,22 @@ def max_task_output_bytes_to_read(self, op: PhysicalOperator) -> Optional[int]: class ReservationOpResourceAllocator(OpResourceAllocator): """An OpResourceAllocator implementation that reserves resources for each operator. - This class reserves memory and CPU resources for map operators, and consider runtime - resource usages to limit the resources that each operator can use. + This class reserves memory and CPU resources for eligible operators, and considers + runtime resource usages to limit the resources that each operator can use. It works in the following way: - 1. Currently we only limit map operators. Non-map operators are not throttled, but - their usage will be accounted for their upstream map operators. E.g., for such - a dataset "map1->limit->map2->streaming_split", we'll treat "map1->limit" as + 1. An operator is eligible for resource reservation, if it has enabled throttling + and hasn't completed. Ineligible operators are not throttled, but + their usage will be accounted for their upstream eligible operators. E.g., for + such a dataset "map1->limit->map2->streaming_split", we'll treat "map1->limit" as a group and "map2->streaming_split" as another group. - 2. For each map operator, we reserve `reservation_ratio * global_resources / - num_map_ops` resources, half of which is reserved only for the operator outputs, - excluding pending task outputs. + 2. For each eligible operator, we reserve `reservation_ratio * global_resources / + num_eligible_ops` resources, half of which is reserved only for the operator + outputs, excluding pending task outputs. 3. Non-reserved resources are shared among all operators. - 4. In each scheduling iteration, each map operator will get "remaining of their own - reserved resources" + "remaining of shared resources / num_map_ops" resources. + 4. In each scheduling iteration, each eligible operator will get "remaining of their + own reserved resources" + "remaining of shared resources / num_eligible_ops" + resources. The `reservation_ratio` is set to 50% by default. Users can tune this value to adjust how aggressive or conservative the resource allocation is. A higher value @@ -357,7 +347,7 @@ def __init__(self, resource_manager: ResourceManager, reservation_ratio: float): # the pending task outputs, and/or op's internal output buffers (the latter can # happen when `preserve_order=True`). # Then we'll have no budget to pull blocks from the op. - self._reserved_for_op_outputs: Dict[PhysicalOperator, int] = {} + self._reserved_for_op_outputs: Dict[PhysicalOperator, float] = {} # Total shared resources. self._total_shared = ExecutionResources.zero() # Resource budgets for each operator, excluding `_reserved_for_op_outputs`. @@ -374,12 +364,13 @@ def __init__(self, resource_manager: ResourceManager, reservation_ratio: float): self._idle_detector = self.IdleDetector() + def _is_op_eligible(self, op: PhysicalOperator) -> bool: + """Whether the op is eligible for memory reservation.""" + return not op.throttling_disabled() and not op.completed() + def _get_eligible_ops(self) -> List[PhysicalOperator]: - # Only consider map operators that are not completed. return [ - op - for op in self._resource_manager._topology - if isinstance(op, MapOperator) and not op.completed() + op for op in self._resource_manager._topology if self._is_op_eligible(op) ] def _update_reservation(self): @@ -472,7 +463,7 @@ def _should_unblock_streaming_output_backpressure( # launch tasks. Then we should temporarily unblock the streaming output # backpressure by allowing reading at least 1 block. So the current operator # can finish at least one task and yield resources to the downstream operators. - for next_op in self._get_downstream_map_ops(op): + for next_op in self._get_downstream_eligible_ops(op): if not self._reserved_min_resources[next_op]: # Case 1: the downstream operator hasn't reserved the minimum resources # to run at least one task. @@ -485,16 +476,16 @@ def _should_unblock_streaming_output_backpressure( return True return False - def _get_op_outputs_usage_with_downstream(self, op: PhysicalOperator) -> int: + def _get_op_outputs_usage_with_downstream(self, op: PhysicalOperator) -> float: """Get the outputs memory usage of the given operator, including the downstream - non-Map operators. + ineligible operators. """ # Outputs usage of the current operator. op_outputs_usage = self._resource_manager._mem_op_outputs[op] - # Also account the downstream non-Map operators' memory usage. + # Also account the downstream ineligible operators' memory usage. op_outputs_usage += sum( self._resource_manager.get_op_usage(next_op).object_store_memory - for next_op in self._get_downstream_non_map_ops(op) + for next_op in self._get_downstream_ineligible_ops(op) ) return op_outputs_usage @@ -511,35 +502,35 @@ def max_task_output_bytes_to_read(self, op: PhysicalOperator) -> Optional[int]: res = 1 return res - def _get_downstream_non_map_ops( + def _get_downstream_ineligible_ops( self, op: PhysicalOperator ) -> Iterable[PhysicalOperator]: - """Get the downstream non-Map operators of the given operator. + """Get the downstream ineligible operators of the given operator. E.g., - "cur_map->downstream_map" will return an empty list. - "cur_map->limit1->limit2->downstream_map" will return [limit1, limit2]. """ for next_op in op.output_dependencies: - if not isinstance(next_op, MapOperator): + if not self._is_op_eligible(next_op): yield next_op - yield from self._get_downstream_non_map_ops(next_op) + yield from self._get_downstream_ineligible_ops(next_op) - def _get_downstream_map_ops( + def _get_downstream_eligible_ops( self, op: PhysicalOperator ) -> Iterable[PhysicalOperator]: - """Get the downstream Map operators of the given operator, ignoring intermediate - non-Map operators. + """Get the downstream eligible operators of the given operator, ignoring intermediate + ineligible operators. E.g., - "cur_map->downstream_map" will return [downstream_map]. - "cur_map->limit1->limit2->downstream_map" will return [downstream_map]. """ for next_op in op.output_dependencies: - if isinstance(next_op, MapOperator): + if self._is_op_eligible(next_op): yield next_op else: - yield from self._get_downstream_map_ops(next_op) + yield from self._get_downstream_eligible_ops(next_op) def update_usages(self): self._update_reservation() From f2dfc37a374b8838019869dbd63928d9f73e52dd Mon Sep 17 00:00:00 2001 From: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> Date: Wed, 29 May 2024 20:51:50 -0700 Subject: [PATCH 50/65] [Core] Fix race condition in setting node death info (#45619) Signed-off-by: Rui Qiao --- src/ray/gcs/gcs_server/gcs_node_manager.cc | 51 ++++++++----------- src/ray/gcs/gcs_server/gcs_node_manager.h | 15 ++---- .../test/gcs_actor_scheduler_test.cc | 12 +++-- .../test/gcs_autoscaler_state_manager_test.cc | 32 ++++++++++++ .../gcs_server/test/gcs_node_manager_test.cc | 6 ++- .../gcs_placement_group_scheduler_test.cc | 3 +- .../gcs_server/test/gcs_server_test_util.h | 16 +++++- src/ray/raylet/main.cc | 2 +- 8 files changed, 87 insertions(+), 50 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 1aa65a539ee3..6c04c4e1aede 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -113,8 +113,7 @@ void GcsNodeManager::HandleUnregisterNode(rpc::UnregisterNodeRequest request, rpc::SendReplyCallback send_reply_callback) { NodeID node_id = NodeID::FromBinary(request.node_id()); RAY_LOG(DEBUG) << "HandleUnregisterNode() for node id = " << node_id; - SetDeathInfo(node_id, request.node_death_info()); - auto node = RemoveNode(node_id, /* is_intended = */ true); + auto node = RemoveNode(node_id, request.node_death_info()); if (!node) { RAY_LOG(INFO) << "Node " << node_id << " is already removed"; return; @@ -224,26 +223,9 @@ absl::optional> GcsNodeManager::GetAliveNode( return iter->second; } -void GcsNodeManager::SetDeathInfo(const NodeID &node_id, - const rpc::NodeDeathInfo &death_info) { - auto maybe_node = GetAliveNode(node_id); - if (!maybe_node.has_value()) { - return; - } - - auto node = std::move(maybe_node.value()); - auto node_death_info = node->mutable_death_info(); - node_death_info->CopyFrom(death_info); -} - rpc::NodeDeathInfo GcsNodeManager::InferDeathInfo(const NodeID &node_id) { - auto maybe_node = GetAliveNode(node_id); - RAY_CHECK(maybe_node.has_value()) - << "InferDeathInfo() should be called before node is removed"; - auto node = maybe_node.value(); auto iter = draining_nodes_.find(node_id); rpc::NodeDeathInfo death_info; - bool expect_force_termination; if (iter == draining_nodes_.end()) { expect_force_termination = false; @@ -288,7 +270,11 @@ void GcsNodeManager::SetNodeDraining( const NodeID &node_id, std::shared_ptr drain_request) { auto maybe_node = GetAliveNode(node_id); - RAY_CHECK(maybe_node.has_value()); + if (!maybe_node.has_value()) { + RAY_LOG(INFO) << "Skip setting node " << node_id << " to be draining, " + << "which is already removed"; + return; + } auto iter = draining_nodes_.find(node_id); if (iter == draining_nodes_.end()) { draining_nodes_.emplace(node_id, drain_request); @@ -303,16 +289,20 @@ void GcsNodeManager::SetNodeDraining( } std::shared_ptr GcsNodeManager::RemoveNode( - const ray::NodeID &node_id, bool is_intended /*= false*/) { + const ray::NodeID &node_id, const rpc::NodeDeathInfo &node_death_info) { std::shared_ptr removed_node; auto iter = alive_nodes_.find(node_id); if (iter != alive_nodes_.end()) { removed_node = std::move(iter->second); - auto death_info = removed_node->death_info(); + + // Set node death info. + auto death_info = removed_node->mutable_death_info(); + death_info->CopyFrom(node_death_info); + RAY_LOG(INFO) << "Removing node, node id = " << node_id << ", node name = " << removed_node->node_name() << ", death reason = " - << rpc::NodeDeathInfo_Reason_Name(death_info.reason()) - << ", death message = " << death_info.reason_message(); + << rpc::NodeDeathInfo_Reason_Name(death_info->reason()) + << ", death message = " << death_info->reason_message(); // Record stats that there's a new removed node. stats::NodeFailureTotal.Record(1); // Remove from alive nodes. @@ -320,7 +310,7 @@ std::shared_ptr GcsNodeManager::RemoveNode( node_map_.left.erase(node_id); // Remove from draining nodes if present. draining_nodes_.erase(node_id); - if (!is_intended) { + if (death_info->reason() == rpc::NodeDeathInfo::UNEXPECTED_TERMINATION) { // Broadcast a warning to all of the drivers indicating that the node // has been marked as dead. // TODO(rkn): Define this constant somewhere else. @@ -332,7 +322,7 @@ std::shared_ptr GcsNodeManager::RemoveNode( << " and node name: " << removed_node->node_name() << " has been marked dead because the detector" << " has missed too many heartbeats from it. This can happen when a " - "\t(1) raylet crashes unexpectedly (OOM, preempted node, etc.) \n" + "\t(1) raylet crashes unexpectedly (OOM, etc.) \n" << "\t(2) raylet has lagging heartbeats due to slow network or busy workload."; RAY_EVENT(ERROR, EL_RAY_NODE_REMOVED) .WithField("node_id", node_id.Hex()) @@ -354,11 +344,10 @@ std::shared_ptr GcsNodeManager::RemoveNode( void GcsNodeManager::OnNodeFailure(const NodeID &node_id, const StatusCallback &node_table_updated_callback) { - rpc::NodeDeathInfo death_info = InferDeathInfo(node_id); - SetDeathInfo(node_id, death_info); - bool is_expected_termination = - (death_info.reason() == rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); - if (auto node = RemoveNode(node_id, is_expected_termination)) { + auto maybe_node = GetAliveNode(node_id); + if (maybe_node.has_value()) { + rpc::NodeDeathInfo death_info = InferDeathInfo(node_id); + auto node = RemoveNode(node_id, death_info); node->set_state(rpc::GcsNodeInfo::DEAD); node->set_end_time_ms(current_sys_time_ms()); diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.h b/src/ray/gcs/gcs_server/gcs_node_manager.h index 92035f6b9c46..edefa416cda9 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.h +++ b/src/ray/gcs/gcs_server/gcs_node_manager.h @@ -109,13 +109,14 @@ class GcsNodeManager : public rpc::NodeInfoHandler { void SetNodeDraining(const NodeID &node_id, std::shared_ptr request); - /// Remove from alive nodes. + /// Remove a node from alive nodes. The node's death information will also be set. /// /// \param node_id The ID of the node to be removed. - /// \param is_intended False if this is triggered by `node_failure_detector_`, else - /// True. + /// \param node_death_info The node death info to set. + /// \return The removed node, with death info set. If the node is not found, return + /// nullptr. std::shared_ptr RemoveNode(const NodeID &node_id, - bool is_intended = false); + const rpc::NodeDeathInfo &node_death_info); /// Get alive node by ID. /// @@ -176,12 +177,6 @@ class GcsNodeManager : public rpc::NodeInfoHandler { /// \param node The node which is dead. void AddDeadNodeToCache(std::shared_ptr node); - /// Set the death info of the node. - /// - /// \param node_id The ID of the node. - /// \param death_info The death info of the node. - void SetDeathInfo(const NodeID &node_id, const rpc::NodeDeathInfo &death_info); - /// Infer death cause of the node based on existing draining requests. /// /// \param node_id The ID of the node. The node must not be removed diff --git a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc index 3b3e432c130b..557a23c7fddc 100644 --- a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc @@ -334,7 +334,8 @@ TEST_F(GcsActorSchedulerTest, TestNodeFailedWhenLeasing) { // Remove the node and cancel the scheduling on this node, the scheduling should be // interrupted. - gcs_node_manager_->RemoveNode(node_id); + rpc::NodeDeathInfo death_info; + gcs_node_manager_->RemoveNode(node_id, death_info); ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size()); auto actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id); ASSERT_EQ(1, actor_ids.size()); @@ -422,7 +423,8 @@ TEST_F(GcsActorSchedulerTest, TestNodeFailedWhenCreating) { // Remove the node and cancel the scheduling on this node, the scheduling should be // interrupted. - gcs_node_manager_->RemoveNode(node_id); + rpc::NodeDeathInfo death_info; + gcs_node_manager_->RemoveNode(node_id, death_info); ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size()); auto actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id); ASSERT_EQ(1, actor_ids.size()); @@ -934,7 +936,8 @@ TEST_F(GcsActorSchedulerTest, TestNodeFailedWhenLeasingByGcs) { // Remove the node and cancel the scheduling on this node, the scheduling should be // interrupted. - gcs_node_manager_->RemoveNode(node_id); + rpc::NodeDeathInfo death_info; + gcs_node_manager_->RemoveNode(node_id, death_info); ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size()); auto actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id); ASSERT_EQ(1, actor_ids.size()); @@ -1028,7 +1031,8 @@ TEST_F(GcsActorSchedulerTest, TestNodeFailedWhenCreatingByGcs) { // Remove the node and cancel the scheduling on this node, the scheduling should be // interrupted. - gcs_node_manager_->RemoveNode(node_id); + rpc::NodeDeathInfo death_info; + gcs_node_manager_->RemoveNode(node_id, death_info); ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size()); auto actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id); ASSERT_EQ(1, actor_ids.size()); diff --git a/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc index 1f3c5aca5a21..913cce5a300c 100644 --- a/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc @@ -752,6 +752,38 @@ TEST_F(GcsAutoscalerStateManagerTest, TestDrainingStatus) { } } +TEST_F(GcsAutoscalerStateManagerTest, TestDrainNodeRaceCondition) { + auto node = Mocker::GenNodeInfo(); + + // Adding a node. + node->mutable_resources_total()->insert({"CPU", 2}); + node->mutable_resources_total()->insert({"GPU", 1}); + node->set_instance_id("instance_1"); + AddNode(node); + + rpc::autoscaler::DrainNodeRequest request; + request.set_node_id(node->node_id()); + request.set_reason(rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_PREEMPTION); + request.set_reason_message("preemption"); + request.set_deadline_timestamp_ms(std::numeric_limits::max()); + rpc::autoscaler::DrainNodeReply reply; + auto send_reply_callback = + [](ray::Status status, std::function f1, std::function f2) {}; + gcs_autoscaler_state_manager_->HandleDrainNode(request, &reply, send_reply_callback); + + // At this point, the GCS request is not accepted yet since ralyet has not replied. + ASSERT_FALSE(reply.is_accepted()); + + // Inject a race condition on GCS: remove the node before raylet accepts the request. + RemoveNode(node); + + // Simulates raylet accepts the drain request and replies to GCS. + ASSERT_TRUE(raylet_client_->ReplyDrainRaylet()); + + // The GCS request is accepted now. + ASSERT_TRUE(reply.is_accepted()); +} + TEST_F(GcsAutoscalerStateManagerTest, TestIdleTime) { auto node = Mocker::GenNodeInfo(); diff --git a/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc index 0424b209faa2..a8a0157e0d54 100644 --- a/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc @@ -51,7 +51,8 @@ TEST_F(GcsNodeManagerTest, TestManagement) { node_manager.AddNode(node); ASSERT_EQ(node, node_manager.GetAliveNode(node_id).value()); - node_manager.RemoveNode(node_id); + rpc::NodeDeathInfo death_info; + node_manager.RemoveNode(node_id, death_info); ASSERT_TRUE(!node_manager.GetAliveNode(node_id).has_value()); } @@ -84,8 +85,9 @@ TEST_F(GcsNodeManagerTest, TestListener) { [&removed_nodes](std::shared_ptr node) { removed_nodes.emplace_back(std::move(node)); }); + rpc::NodeDeathInfo death_info; for (int i = 0; i < node_count; ++i) { - node_manager.RemoveNode(NodeID::FromBinary(added_nodes[i]->node_id())); + node_manager.RemoveNode(NodeID::FromBinary(added_nodes[i]->node_id()), death_info); } ASSERT_EQ(node_count, removed_nodes.size()); ASSERT_TRUE(node_manager.GetAllAliveNodes().empty()); diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc index e6a53623d750..f94d3ac2a8b2 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc @@ -125,7 +125,8 @@ class GcsPlacementGroupSchedulerTest : public ::testing::Test { } void RemoveNode(const std::shared_ptr &node) { - gcs_node_manager_->RemoveNode(NodeID::FromBinary(node->node_id())); + rpc::NodeDeathInfo death_info; + gcs_node_manager_->RemoveNode(NodeID::FromBinary(node->node_id()), death_info); gcs_resource_manager_->OnNodeDead(NodeID::FromBinary(node->node_id())); } diff --git a/src/ray/gcs/gcs_server/test/gcs_server_test_util.h b/src/ray/gcs/gcs_server/test/gcs_server_test_util.h index 432ed655bf7f..fcd6b6bf6a3a 100644 --- a/src/ray/gcs/gcs_server/test/gcs_server_test_util.h +++ b/src/ray/gcs/gcs_server/test/gcs_server_test_util.h @@ -200,6 +200,19 @@ struct GcsServerMocker { } } + bool ReplyDrainRaylet() { + if (drain_raylet_callbacks.size() == 0) { + return false; + } else { + rpc::DrainRayletReply reply; + reply.set_is_accepted(true); + auto callback = drain_raylet_callbacks.front(); + callback(Status::OK(), reply); + drain_raylet_callbacks.pop_front(); + return true; + } + } + /// ResourceReserveInterface void PrepareBundleResources( const std::vector> &bundle_specs, @@ -304,7 +317,7 @@ struct GcsServerMocker { const rpc::ClientCallback &callback) override { rpc::DrainRayletReply reply; reply.set_is_accepted(true); - callback(Status::OK(), reply); + drain_raylet_callbacks.push_back(callback); }; void NotifyGCSRestart( @@ -319,6 +332,7 @@ struct GcsServerMocker { int num_release_unused_workers = 0; int num_get_task_failure_causes = 0; NodeID node_id = NodeID::FromRandom(); + std::list> drain_raylet_callbacks = {}; std::list> callbacks = {}; std::list> cancel_callbacks = {}; std::list> release_callbacks = {}; diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 2942bd50ea4a..19d90124892a 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -450,7 +450,7 @@ int main(int argc, char *argv[]) { drain_request->reason() == ray::rpc::autoscaler::DrainNodeReason::DRAIN_NODE_REASON_PREEMPTION && drain_request->deadline_timestamp_ms() != 0 && - drain_request->deadline_timestamp_ms() < current_time_ms()) { + drain_request->deadline_timestamp_ms() < current_sys_time_ms()) { node_death_info.set_reason(ray::rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); node_death_info.set_reason_message(drain_request->reason_message()); } else { From 480b572d48d971cb0a379dd6b3bb3b0d7f21e5bc Mon Sep 17 00:00:00 2001 From: Alan Guo Date: Wed, 29 May 2024 22:03:57 -0700 Subject: [PATCH 51/65] add env var to allow dashboard http server to serve static files with symlinks (#45618) New env var is called RAY_DASHBOARD_BUILD_FOLLOW_SYMLINKS. This is an advanced setting that should only be used with special Ray installations where the dashboard build files are symlinked to a different directory. This is not recommended for most users and can pose a security risk. Please reference the aiohttp docs here: https://docs.aiohttp.org/en/stable/web_reference.html#aiohttp.web.UrlDispatcher.add_static --- dashboard/http_server_head.py | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/dashboard/http_server_head.py b/dashboard/http_server_head.py index 50d3e05473da..e1c427b1b288 100644 --- a/dashboard/http_server_head.py +++ b/dashboard/http_server_head.py @@ -28,6 +28,22 @@ logger = logging.getLogger(__name__) routes = dashboard_optional_utils.DashboardHeadRouteTable +# Env var that enables follow_symlinks for serving UI static files. +# This is an advanced setting that should only be used with special Ray installations +# where the dashboard build files are symlinked to a different directory. +# This is not recommended for most users and can pose a security risk. +# Please reference the aiohttp docs here: +# https://docs.aiohttp.org/en/stable/web_reference.html#aiohttp.web.UrlDispatcher.add_static +ENV_VAR_FOLLOW_SYMLINKS = "RAY_DASHBOARD_BUILD_FOLLOW_SYMLINKS" +FOLLOW_SYMLINKS_ENABLED = os.environ.get(ENV_VAR_FOLLOW_SYMLINKS) == "1" +if FOLLOW_SYMLINKS_ENABLED: + logger.warning( + "Enabling RAY_DASHBOARD_BUILD_FOLLOW_SYMLINKS is not recommended as it " + "allows symlinks to directories outside the dashboard build folder. " + "You may accidentally expose files on your system outside of the " + "build directory." + ) + def setup_static_dir(): build_dir = os.path.join( @@ -47,7 +63,7 @@ def setup_static_dir(): ) static_dir = os.path.join(build_dir, "static") - routes.static("/static", static_dir) + routes.static("/static", static_dir, follow_symlinks=FOLLOW_SYMLINKS_ENABLED) return build_dir From c94140a3a4b8ae9c0e6f1965d6285161189f9851 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Thu, 30 May 2024 08:50:54 +0200 Subject: [PATCH 52/65] [RLlib] Complete do-over of RLlib release tests (new API stack). (#45589) --- release/release_tests.yaml | 600 +----------------- .../multi_gpu_learning_tests.yaml | 237 ------- .../_old_multi_gpu_learning_tests/run.py | 35 - ...lti_gpu_with_attention_learning_tests.yaml | 191 ------ .../run.py | 35 - .../multi_gpu_with_lstm_learning_tests.yaml | 136 ---- .../run.py | 35 - .../_old_rllib_multi_gpu_tests.yaml | 92 --- release/rllib_tests/learning_tests/README.md | 21 - .../learning_tests/run_new_api_stack.py | 61 -- .../rllib_tests/learning_tests/tuned_examples | 1 + rllib/BUILD | 36 +- rllib/algorithms/algorithm.py | 2 +- rllib/env/multi_agent_env_runner.py | 6 - rllib/env/single_agent_env_runner.py | 2 +- rllib/tuned_examples/ppo/atari-ppo.yaml | 35 - rllib/tuned_examples/ppo/atari_ppo.py | 93 +++ .../ppo/cartpole-ppo-fake-gpus.yaml | 1 + .../ppo/cartpole-ppo-grid-search-example.yaml | 14 - .../ppo/cartpole-ppo-hyperband.yaml | 16 - rllib/tuned_examples/ppo/pong-ppo.yaml | 33 - .../tuned_examples/ppo/recomm-sys001-ppo.yaml | 49 -- rllib/utils/test_utils.py | 76 ++- 23 files changed, 167 insertions(+), 1640 deletions(-) delete mode 100644 release/rllib_tests/_old_multi_gpu_learning_tests/multi_gpu_learning_tests.yaml delete mode 100644 release/rllib_tests/_old_multi_gpu_learning_tests/run.py delete mode 100644 release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/multi_gpu_with_attention_learning_tests.yaml delete mode 100644 release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/run.py delete mode 100644 release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/multi_gpu_with_lstm_learning_tests.yaml delete mode 100644 release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/run.py delete mode 100644 release/rllib_tests/_old_rllib_multi_gpu_tests.yaml delete mode 100644 release/rllib_tests/learning_tests/README.md delete mode 100644 release/rllib_tests/learning_tests/run_new_api_stack.py create mode 120000 release/rllib_tests/learning_tests/tuned_examples delete mode 100644 rllib/tuned_examples/ppo/atari-ppo.yaml create mode 100644 rllib/tuned_examples/ppo/atari_ppo.py delete mode 100644 rllib/tuned_examples/ppo/cartpole-ppo-grid-search-example.yaml delete mode 100644 rllib/tuned_examples/ppo/cartpole-ppo-hyperband.yaml delete mode 100644 rllib/tuned_examples/ppo/pong-ppo.yaml delete mode 100644 rllib/tuned_examples/ppo/recomm-sys001-ppo.yaml diff --git a/release/release_tests.yaml b/release/release_tests.yaml index b285fb4c11d8..a9c01afbfc93 100644 --- a/release/release_tests.yaml +++ b/release/release_tests.yaml @@ -2740,473 +2740,13 @@ # ---------------------------------------------------------- # -------------------------- -# APPO (hybrid API stack) +# PPO # -------------------------- -- name: rllib_learning_tests_appo_hybrid_api_stack_torch +- name: rllib_learning_tests_pong_ppo_torch group: RLlib tests working_dir: rllib_tests - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 4gpus_64cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=appo/hybrid_stack --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 4gpus_64cpus_gce.yaml - -- name: rllib_learning_tests_appo_hybrid_api_stack_tf2 - group: RLlib tests - working_dir: rllib_tests - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 4gpus_64cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=appo/hybrid_stack --framework=tf2 - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 4gpus_64cpus_gce.yaml - -# -------------------------- -# APPO (old API stack) -# -------------------------- -- name: rllib_learning_tests_appo_old_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 2gpus_32cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=appo/old_stack --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 2gpus_32cpus_gce.yaml - -- name: rllib_learning_tests_appo_old_api_stack_tf - group: RLlib tests - working_dir: rllib_tests - stable: False - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 2gpus_32cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=appo/old_stack --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 2gpus_32cpus_gce.yaml - - -# -------------------------- -# BC (hybrid API stack) -# -------------------------- -- name: rllib_learning_tests_bc_hybrid_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - frequency: nightly - team: rllib - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=bc --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -- name: rllib_learning_tests_bc_hybrid_api_stack_tf2 - group: RLlib tests - working_dir: rllib_tests - stable: false - - frequency: nightly - team: rllib - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=bc --framework=tf2 - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -# -------------------------- -# CQL (old API stack) -# -------------------------- -- name: rllib_learning_tests_cql_old_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=cql --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -- name: rllib_learning_tests_cql_old_api_stack_tf - group: RLlib tests - working_dir: rllib_tests - - frequency: nightly - team: rllib - - # Marking as unstable since it's currently expected to fail. - stable: false - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=cql --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -# -------------------------- -# DQN (old API stack) -# -------------------------- -- name: rllib_learning_tests_dqn_old_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=dqn --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -- name: rllib_learning_tests_dqn_old_api_stack_tf - group: RLlib tests - working_dir: rllib_tests - stable: False - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_32cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=dqn --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -# -------------------------- -# IMPALA (old API stack) -# -------------------------- -- name: rllib_learning_tests_impala_old_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=impala --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -- name: rllib_learning_tests_impala_old_api_stack_tf - group: RLlib tests - working_dir: rllib_tests - stable: False - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=impala --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - - -# -------------------------- -# MARWIL (old API stack) -# -------------------------- -- name: rllib_learning_tests_marwil_old_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=marwil --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -- name: rllib_learning_tests_marwil_old_api_stack_tf - group: RLlib tests - working_dir: rllib_tests - - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=marwil --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - - -# -------------------------- -# PPO (new API stack) -# -------------------------- -- name: rllib_learning_tests_ppo_new_api_stack_torch - group: RLlib tests - working_dir: rllib_tests - - stable: False + stable: true frequency: nightly team: rllib @@ -3220,8 +2760,8 @@ cluster_compute: 8gpus_96cpus.yaml run: - timeout: 18000 - script: python learning_tests/run_new_api_stack.py --config-dir=ppo/new_stack --framework=torch + timeout: 1800 + script: python learning_tests/tuned_examples/ppo/atari_ppo.py --enable-new-api-stack --env ALE/Pong-v5 --num-gpus=4 --num-env-runners=95 --as-test alert: default @@ -3233,136 +2773,6 @@ cluster: cluster_compute: 8gpus_96cpus_gce.yaml - -# -------------------------- -# PPO (old API stack) -# -------------------------- -- name: rllib_learning_tests_ppo_old_stack_torch - group: RLlib tests - working_dir: rllib_tests - - # Marking as unstable since it's currently expected to fail. - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 2gpus_32cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=ppo/old_stack --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 2gpus_32cpus_gce.yaml - -- name: rllib_learning_tests_ppo_old_stack_tf - group: RLlib tests - working_dir: rllib_tests - frequency: nightly - team: rllib - stable: False - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 2gpus_32cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=ppo/old_stack --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 2gpus_32cpus_gce.yaml - -# -------------------------- -# SAC (old API stack) -# -------------------------- -- name: rllib_learning_tests_sac_tf - group: RLlib tests - working_dir: rllib_tests - stable: False - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=sac --framework=tf - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - -- name: rllib_learning_tests_sac_torch - group: RLlib tests - working_dir: rllib_tests - - frequency: nightly - team: rllib - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 1gpu_16cpus.yaml - - run: - timeout: 18000 - script: python learning_tests/run.py --yaml-sub-dir=sac --framework=torch - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 1gpu_16cpus_gce.yaml - - ######################## # Core Nightly Tests ######################## diff --git a/release/rllib_tests/_old_multi_gpu_learning_tests/multi_gpu_learning_tests.yaml b/release/rllib_tests/_old_multi_gpu_learning_tests/multi_gpu_learning_tests.yaml deleted file mode 100644 index 8a312996532a..000000000000 --- a/release/rllib_tests/_old_multi_gpu_learning_tests/multi_gpu_learning_tests.yaml +++ /dev/null @@ -1,237 +0,0 @@ - -a2c-cartpole-v1: - env: CartPole-v1 - run: A2C - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - lr: 0.001 - -appo-cartpole-v1-no-vtrace: - env: CartPole-v1 - run: APPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - vtrace: false - num_gpus: 2 - num_workers: 5 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Double batch size (2 GPUs). - train_batch_size: 1000 - -appo-cartpole-v1-vtrace: - env: CartPole-v1 - run: APPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 5 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Double batch size (2 GPUs). - train_batch_size: 1000 - -ddpg-repeat-after-me-env: - env: ray.rllib.examples.env.repeat_after_me_env.RepeatAfterMeEnv - run: DDPG - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: -50.0 - timesteps_total: 8000 - stop: - time_total_s: 600 - config: - env_config: - config: - continuous: true - repeat_delay: 0 - - num_gpus: 2 - num_workers: 0 - # Double batch size (2 GPUs). - train_batch_size: 512 - -dqn-cartpole-v1: - env: CartPole-v1 - run: DQN - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 50000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 0 - # Double batch size (2 GPUs). - train_batch_size: 64 - # Mimic tuned_example for DQN CartPole. - n_step: 3 - model: - fcnet_hiddens: [64] - fcnet_activation: linear - -impala-cartpole-v1: - env: CartPole-v1 - run: IMPALA - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Double batch size (2 GPUs). - train_batch_size: 1000 - -pg-cartpole-v1: - env: CartPole-v1 - run: PG - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 130.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Double batch size (2 GPUs). - train_batch_size: 400 - model: - fcnet_hiddens: [64] - fcnet_activation: linear - -ppo-cartpole-v1: - env: CartPole-v1 - run: PPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 300000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Double batch size (2 GPUs). - train_batch_size: 8000 - -sac-repeat-after-me-env: - env: ray.rllib.examples.env.repeat_after_me_env.RepeatAfterMeEnv - run: SAC - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 40.0 - timesteps_total: 4500 - stop: - time_total_s: 600 - config: - env_config: - config: - repeat_delay: 0 - num_gpus: 2 - num_workers: 0 - replay_buffer_config: - type: MultiAgentPrioritizedReplayBuffer - initial_alpha: 0.001 - # Double batch size (2 GPUs). - train_batch_size: 512 - num_steps_sampled_before_learning_starts: 0 - -sac-repeat-after-me-env-continuous: - env: ray.rllib.examples.env.repeat_after_me_env.RepeatAfterMeEnv - run: SAC - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: -50.0 - timesteps_total: 4500 - stop: - time_total_s: 600 - config: - env_config: - config: - continuous: true - repeat_delay: 0 - replay_buffer_config: - type: MultiAgentPrioritizedReplayBuffer - num_gpus: 2 - num_workers: 0 - initial_alpha: 0.001 - # Double batch size (2 GPUs). - train_batch_size: 512 - # start learning immediately - num_steps_sampled_before_learning_starts: 0 - - -simpleq-cartpole-v1: - env: CartPole-v1 - run: SimpleQ - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 85000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 0 - -td3-repeat-after-me-env: - env: ray.rllib.examples.env.repeat_after_me_env.RepeatAfterMeEnv - run: TD3 - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: -50.0 - timesteps_total: 25000 - stop: - time_total_s: 600 - config: - env_config: - config: - continuous: true - repeat_delay: 0 - - num_gpus: 2 - num_workers: 0 - # Double batch size (2 GPUs). - train_batch_size: 200 diff --git a/release/rllib_tests/_old_multi_gpu_learning_tests/run.py b/release/rllib_tests/_old_multi_gpu_learning_tests/run.py deleted file mode 100644 index eafe75a1396c..000000000000 --- a/release/rllib_tests/_old_multi_gpu_learning_tests/run.py +++ /dev/null @@ -1,35 +0,0 @@ -"""Multi-GPU learning tests for RLlib (torch and tf). -""" - -import json -import os -from pathlib import Path - -from ray.rllib.utils.test_utils import run_learning_tests_from_yaml - -if __name__ == "__main__": - # Get path of this very script to look for yaml files. - abs_yaml_path = Path(__file__).parent - print("abs_yaml_path={}".format(abs_yaml_path)) - - yaml_files = abs_yaml_path.rglob("*.yaml") - yaml_files = sorted( - map(lambda path: str(path.absolute()), yaml_files), reverse=True - ) - - # Run all tests in the found yaml files. - results = run_learning_tests_from_yaml(yaml_files) - - test_output_json = os.environ.get( - "TEST_OUTPUT_JSON", "/tmp/rllib_multi_gpu_learning_tests.json" - ) - with open(test_output_json, "wt") as f: - json.dump(results, f) - - if len(results["not_passed"]) > 0: - raise ValueError( - "Not all learning tests successfully learned the tasks.\n" - f"Results=\n{results}" - ) - else: - print("Ok.") diff --git a/release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/multi_gpu_with_attention_learning_tests.yaml b/release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/multi_gpu_with_attention_learning_tests.yaml deleted file mode 100644 index d29c1ed63982..000000000000 --- a/release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/multi_gpu_with_attention_learning_tests.yaml +++ /dev/null @@ -1,191 +0,0 @@ - -appo-stateless-cartpole-no-vtrace: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: APPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - vtrace: false - num_gpus: 2 - num_workers: 5 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Test w/ GTrXL net. - use_attention: true - max_seq_len: 10 - attention_num_transformer_units: 1 - attention_dim: 32 - attention_memory_inference: 10 - attention_memory_training: 10 - attention_num_heads: 1 - attention_head_dim: 32 - attention_position_wise_mlp_dim: 32 - # Double batch size (2 GPUs). - train_batch_size: 1000 - -appo-stateless-cartpole-vtrace: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: APPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 5 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Test w/ GTrXL net. - use_attention: true - max_seq_len: 10 - attention_num_transformer_units: 1 - attention_dim: 32 - attention_memory_inference: 10 - attention_memory_training: 10 - attention_num_heads: 1 - attention_head_dim: 32 - attention_position_wise_mlp_dim: 32 - # Double batch size (2 GPUs). - train_batch_size: 1000 - -impala-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: IMPALA - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Double batch size (2 GPUs). - train_batch_size: 1000 - - # Test w/ GTrXL net. - model: - use_attention: true - max_seq_len: 10 - attention_num_transformer_units: 1 - attention_dim: 32 - attention_memory_inference: 10 - attention_memory_training: 10 - attention_num_heads: 1 - attention_head_dim: 32 - attention_position_wise_mlp_dim: 32 - -pg-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: PG - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 130.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Double batch size (2 GPUs). - train_batch_size: 400 - model: - fcnet_hiddens: [64] - fcnet_activation: linear - # Test w/ GTrXL net. - use_attention: true - max_seq_len: 10 - attention_num_transformer_units: 1 - attention_dim: 32 - attention_memory_inference: 10 - attention_memory_training: 10 - attention_num_heads: 1 - attention_head_dim: 32 - attention_position_wise_mlp_dim: 32 - -ppo-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: PPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 200000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Test w/ GTrXL net. - use_attention: true - max_seq_len: 10 - attention_num_transformer_units: 1 - attention_dim: 32 - attention_memory_inference: 10 - attention_memory_training: 10 - attention_num_heads: 1 - attention_head_dim: 32 - attention_position_wise_mlp_dim: 32 - # Double batch size (2 GPUs). - train_batch_size: 8000 - _enable_new_api_stack: false - -# TODO (Kourosh): Activate these tests back when the new modeling stack is merged -# r2d2-stateless-cartpole: -# env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole -# run: R2D2 -# # Minimum reward and total ts (in given time_total_s) to pass this test. -# pass_criteria: -# sampler_results/episode_reward_mean: 150.0 -# timesteps_total: 130000 -# stop: -# time_total_s: 1200 -# config: -# num_gpus: 2 -# num_workers: 0 -# # R2D2 settings. -# burn_in: 20 -# zero_init_states: true -# lr: 0.0005 -# # Give some more time to explore. -# exploration_config: -# epsilon_timesteps: 50000 -# model: -# # Test w/ GTrXL net. -# use_attention: true -# max_seq_len: 20 -# attention_num_transformer_units: 1 -# attention_dim: 32 -# attention_memory_inference: 10 -# attention_memory_training: 10 -# attention_num_heads: 1 -# attention_head_dim: 32 -# attention_position_wise_mlp_dim: 32 -# # Use a very simple base-model. -# fcnet_hiddens: [64] -# fcnet_activation: linear diff --git a/release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/run.py b/release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/run.py deleted file mode 100644 index 4ed7bb58c416..000000000000 --- a/release/rllib_tests/_old_multi_gpu_with_attention_learning_tests/run.py +++ /dev/null @@ -1,35 +0,0 @@ -"""Multi-GPU + GTrXL (attention net) learning tests for RLlib (torch and tf). -""" - -import json -import os -from pathlib import Path - -from ray.rllib.utils.test_utils import run_learning_tests_from_yaml - -if __name__ == "__main__": - # Get path of this very script to look for yaml files. - abs_yaml_path = Path(__file__).parent - print("abs_yaml_path={}".format(abs_yaml_path)) - - yaml_files = abs_yaml_path.rglob("*.yaml") - yaml_files = sorted( - map(lambda path: str(path.absolute()), yaml_files), reverse=True - ) - - # Run all tests in the found yaml files. - results = run_learning_tests_from_yaml(yaml_files) - - test_output_json = os.environ.get( - "TEST_OUTPUT_JSON", "/tmp/rllib_multi_gpu_with_attention_learning_tests.json" - ) - with open(test_output_json, "wt") as f: - json.dump(results, f) - - if len(results["not_passed"]) > 0: - raise ValueError( - "Not all learning tests successfully learned the tasks.\n" - f"Results=\n{results}" - ) - else: - print("Ok.") diff --git a/release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/multi_gpu_with_lstm_learning_tests.yaml b/release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/multi_gpu_with_lstm_learning_tests.yaml deleted file mode 100644 index 4e973d556d63..000000000000 --- a/release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/multi_gpu_with_lstm_learning_tests.yaml +++ /dev/null @@ -1,136 +0,0 @@ - -a2c-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: A2C - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Use a large train batch size to make sure mini batches work - # after split to 2 GPU towers. - train_batch_size: 200 - lr: 0.001 - # Test w/ LSTMs. - model: - use_lstm: true - -appo-stateless-cartpole-no-vtrace: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: APPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - vtrace: false - num_gpus: 2 - num_workers: 5 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Test w/ LSTMs. - use_lstm: true - # Double batch size (2 GPUs). - train_batch_size: 1000 - -appo-stateless-cartpole-vtrace: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: APPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 5 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Test w/ LSTMs. - use_lstm: true - # Double batch size (2 GPUs). - train_batch_size: 1000 - -impala-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: IMPALA - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Double batch size (2 GPUs). - train_batch_size: 1000 - - # Test w/ LSTMs. - model: - use_lstm: true - -pg-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: PG - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 130.0 - timesteps_total: 500000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - # Double batch size (2 GPUs). - train_batch_size: 400 - model: - fcnet_hiddens: [64] - fcnet_activation: linear - # Test w/ LSTMs. - use_lstm: true - -ppo-stateless-cartpole: - env: ray.rllib.examples.env.stateless_cartpole.StatelessCartPole - run: PPO - # Minimum reward and total ts (in given time_total_s) to pass this test. - pass_criteria: - sampler_results/episode_reward_mean: 150.0 - timesteps_total: 200000 - stop: - time_total_s: 600 - config: - num_gpus: 2 - num_workers: 23 - lr: 0.0003 - observation_filter: MeanStdFilter - num_sgd_iter: 6 - vf_loss_coeff: 0.01 - model: - fcnet_hiddens: [32] - fcnet_activation: linear - vf_share_layers: true - # Test w/ LSTMs. - use_lstm: true - # Double batch size (2 GPUs). - train_batch_size: 8000 - _enable_new_api_stack: false diff --git a/release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/run.py b/release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/run.py deleted file mode 100644 index 975a3cdd0bc3..000000000000 --- a/release/rllib_tests/_old_multi_gpu_with_lstm_learning_tests/run.py +++ /dev/null @@ -1,35 +0,0 @@ -"""Multi-GPU + LSTM learning tests for RLlib (torch and tf). -""" - -import json -import os -from pathlib import Path - -from ray.rllib.utils.test_utils import run_learning_tests_from_yaml - -if __name__ == "__main__": - # Get path of this very script to look for yaml files. - abs_yaml_path = Path(__file__).parent - print("abs_yaml_path={}".format(abs_yaml_path)) - - yaml_files = abs_yaml_path.rglob("*.yaml") - yaml_files = sorted( - map(lambda path: str(path.absolute()), yaml_files), reverse=True - ) - - # Run all tests in the found yaml files. - results = run_learning_tests_from_yaml(yaml_files) - - test_output_json = os.environ.get( - "TEST_OUTPUT_JSON", "/tmp/rllib_multi_gpu_with_lstm_learning_tests.json" - ) - with open(test_output_json, "wt") as f: - json.dump(results, f) - - if len(results["not_passed"]) > 0: - raise ValueError( - "Not all learning tests successfully learned the tasks.\n" - f"Results=\n{results}" - ) - else: - print("Ok.") diff --git a/release/rllib_tests/_old_rllib_multi_gpu_tests.yaml b/release/rllib_tests/_old_rllib_multi_gpu_tests.yaml deleted file mode 100644 index 831c4b5a3731..000000000000 --- a/release/rllib_tests/_old_rllib_multi_gpu_tests.yaml +++ /dev/null @@ -1,92 +0,0 @@ - -- name: rllib_multi_gpu_learning_tests - group: RLlib tests - working_dir: rllib_tests - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 8gpus_96cpus.yaml - - run: - timeout: 7200 - script: python multi_gpu_learning_tests/run.py - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 8gpus_96cpus_gce.yaml - -- name: rllib_multi_gpu_with_lstm_learning_tests - group: RLlib tests - working_dir: rllib_tests - stable: false - - frequency: nightly - team: rllib - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 8gpus_96cpus.yaml - - run: - timeout: 7200 - script: python multi_gpu_with_lstm_learning_tests/run.py - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - cluster_compute: 8gpus_96cpus_gce.yaml - -- name: rllib_multi_gpu_with_attention_learning_tests - group: RLlib tests - working_dir: rllib_tests - - frequency: nightly - team: rllib - - cluster: - byod: - type: gpu - post_build_script: byod_rllib_test.sh - runtime_env: - - RLLIB_TEST_NO_JAX_IMPORT=1 - - LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/home/ray/.mujoco/mujoco210/bin - cluster_compute: 8gpus_96cpus.yaml - - run: - timeout: 7200 - script: python multi_gpu_with_attention_learning_tests/run.py - - alert: default - - variations: - - __suffix__: aws - - __suffix__: gce - env: gce - frequency: manual - cluster: - # TODO(https://github.com/ray-project/ray/issues/34591) - # Revert to the comment below once ^ closed. - cluster_compute: 8gpus_96cpus_gce.yaml diff --git a/release/rllib_tests/learning_tests/README.md b/release/rllib_tests/learning_tests/README.md deleted file mode 100644 index 8f3db7f56abc..000000000000 --- a/release/rllib_tests/learning_tests/README.md +++ /dev/null @@ -1,21 +0,0 @@ -# RLlib Hard Learning Test - -Test most important RLlib algorithms with hard enough tasks to prevent performance regression. - -Algorithms in this suite are split into multiple tests, so groups of tests can run in parallel. This is to ensure reasonable total runtime. - -All learning tests have ``stop`` and ``pass_criteria`` configured, where ``stop`` specifies a fixed test duration, and ``pass_criteria`` specified performance goals like ``minimum reward`` and ``minimum throughput``. - -Unlike normal tuned examples, these learning tests always run to the full specified test duration, and would NOT stop early when the ``pass_criteria`` is met. - -This is so they can serve better as performance regression tests: - -* By giving these tests more time, we get better idea of where they actually peak out (instead of simply stopping at a pre-specified reward). So we will have better ideas of minor peak performance regressions when they happen. -* By decoupling peak performance from ``pass_criteria``, we can specify a relatively conservative ``pass_criteria``, to avoid having flaky tests that pass and fail because of random fluctuations. -* These conservative passing thresholds help alert us when some algorithms are badly broken. -* Peak reward and throughput numbers gets save in DB, so we can see, hopefully step function, trends over time when we improve things. - -TODO: we don't see progress right now in the time series chart, if an algorithm learns faster, but to the same peak performance. -For that, we need to plot multiple lines at different percentage time mark. - -If you have any questions about these tests, ping jungong@. \ No newline at end of file diff --git a/release/rllib_tests/learning_tests/run_new_api_stack.py b/release/rllib_tests/learning_tests/run_new_api_stack.py deleted file mode 100644 index b04b28d2d6a0..000000000000 --- a/release/rllib_tests/learning_tests/run_new_api_stack.py +++ /dev/null @@ -1,61 +0,0 @@ -"""Learning regression tests on the new API stack for RLlib.""" - -import json -import os -from pathlib import Path - -from ray.rllib.utils.test_utils import run_learning_tests_from_yaml_or_py - - -if __name__ == "__main__": - import argparse - - parser = argparse.ArgumentParser() - parser.add_argument( - "--smoke-test", - action="store_true", - default=False, - help="Finish quickly for training.", - ) - parser.add_argument( - "--config-dir", - type=str, - default="", - help="Sub directory under yaml_files/ to look for .py config files.", - ) - parser.add_argument( - "--framework", - type=str, - default="tf", - help="The framework (tf|tf2|torch) to use.", - ) - args = parser.parse_args() - - assert args.config_dir, "--config-dir can't be empty." - - # Get path of this very script to look for yaml files. - abs_config_path = os.path.join( - str(Path(__file__).parent), "yaml_files", args.config_dir - ) - print("abs_config_path={}".format(abs_config_path)) - - py_files = Path(abs_config_path).rglob("*.py") - py_files = sorted(map(lambda path: str(path.absolute()), py_files), reverse=True) - - # Run all tests in the found yaml files. - results = run_learning_tests_from_yaml_or_py( - config_files=py_files, - framework=args.framework, - ) - - test_output_json = os.environ.get("TEST_OUTPUT_JSON", "/tmp/learning_test.json") - with open(test_output_json, "wt") as f: - json.dump(results, f) - - if len(results["not_passed"]) > 0: - raise ValueError( - "Not all learning tests successfully learned the tasks.\n" - f"Results=\n{results}" - ) - else: - print("Ok.") diff --git a/release/rllib_tests/learning_tests/tuned_examples b/release/rllib_tests/learning_tests/tuned_examples new file mode 120000 index 000000000000..c0b75244d04b --- /dev/null +++ b/release/rllib_tests/learning_tests/tuned_examples @@ -0,0 +1 @@ +../../../rllib/tuned_examples \ No newline at end of file diff --git a/rllib/BUILD b/rllib/BUILD index e6966fa341b6..72e97d846854 100644 --- a/rllib/BUILD +++ b/rllib/BUILD @@ -226,7 +226,7 @@ py_test( py_test( name = "learning_tests_cartpole_crashing_appo_old_api_stack", main = "tests/run_regression_tests.py", - tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "crashing_cartpole", "no_tf_static_graph"], + tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "crashing_cartpole", "torch_only"], size = "large", srcs = ["tests/run_regression_tests.py"], data = ["tuned_examples/appo/cartpole-crashing-recreate-workers-appo.py"], @@ -248,7 +248,7 @@ py_test( py_test( name = "learning_tests_multi_agent_cartpole_crashing_appo_old_api_stack", main = "tests/run_regression_tests.py", - tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "crashing_cartpole"], + tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "crashing_cartpole", "torch_only"], size = "large", srcs = ["tests/run_regression_tests.py"], data = ["tuned_examples/appo/multi-agent-cartpole-crashing-recreate-workers-appo.py"], @@ -259,7 +259,7 @@ py_test( py_test( name = "learning_tests_multi_agent_cartpole_crashing_and_stalling_appo_old_api_stack", main = "tests/run_regression_tests.py", - tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "crashing_cartpole"], + tags = ["team:rllib", "exclusive", "learning_tests", "learning_tests_cartpole", "learning_tests_discrete", "crashing_cartpole", "torch_only"], size = "large", srcs = ["tests/run_regression_tests.py"], data = ["tuned_examples/appo/multi-agent-cartpole-crashing-and-stalling-recreate-workers-appo.py"], @@ -2400,26 +2400,6 @@ py_test( args = ["--enable-new-api-stack", "--num-agents=2", "--as-test", "--stop-reward=900.0", "--num-cpus=6", "--evaluation-duration=10", "--evaluation-duration-unit=episodes"] ) -# @OldAPIStack -py_test( - name = "examples/evaluation/evaluation_parallel_to_training_13_episodes_tf_old_api_stack", - main = "examples/evaluation/evaluation_parallel_to_training.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/evaluation/evaluation_parallel_to_training.py"], - args = ["--as-test", "--framework=tf", "--stop-reward=50.0", "--num-cpus=6", "--evaluation-duration=13"] -) - -# @OldAPIStack -py_test( - name = "examples/evaluation/evaluation_parallel_to_training_duration_auto_tf_old_api_stack", - main = "examples/evaluation/evaluation_parallel_to_training.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/evaluation/evaluation_parallel_to_training.py"], - args = ["--as-test", "--stop-reward=50.0", "--framework=tf", "--num-cpus=6", "--evaluation-duration=auto"] -) - # @OldAPIStack py_test( name = "examples/evaluation/evaluation_parallel_to_training_duration_auto_torch_old_api_stack", @@ -2430,16 +2410,6 @@ py_test( args = ["--as-test", "--stop-reward=50.0", "--num-cpus=6", "--evaluation-duration=auto"] ) -# @OldAPIStack -py_test( - name = "examples/evaluation/evaluation_parallel_to_training_duration_auto_tf2_old_api_stack", - main = "examples/evaluation/evaluation_parallel_to_training.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/evaluation/evaluation_parallel_to_training.py"], - args = ["--as-test", "--framework=tf2", "--stop-reward=30.0", "--num-cpus=6", "--evaluation-num-env-runners=3", "--evaluation-duration=auto"] -) - # @OldAPIStack py_test( name = "examples/evaluation/evaluation_parallel_to_training_211_ts_torch_old_api_stack", diff --git a/rllib/algorithms/algorithm.py b/rllib/algorithms/algorithm.py index 6b71fd59dd0c..c0a8b3fecd29 100644 --- a/rllib/algorithms/algorithm.py +++ b/rllib/algorithms/algorithm.py @@ -542,7 +542,7 @@ def default_logger_creator(config): # (although their values may be nan), so that Tune doesn't complain # when we use these as stopping criteria. self.evaluation_metrics = { - "evaluation": { + EVALUATION_RESULTS: { ENV_RUNNER_RESULTS: { EPISODE_RETURN_MAX: np.nan, EPISODE_RETURN_MIN: np.nan, diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index e179d047cfc9..f374f45b35e6 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -623,12 +623,6 @@ def get_metrics(self) -> ResultDict: module_episode_returns, ) - # TODO (simon): This results in hundreds of warnings in the logs - # b/c reducing over NaNs is not supported. - # # If no episodes at all, log NaN stats. - # if len(self._done_episodes_for_metrics) == 0: - # self._log_episode_metrics(np.nan, np.nan, np.nan) - # Log num episodes counter for this iteration. self.metrics.log_value( NUM_EPISODES, diff --git a/rllib/env/single_agent_env_runner.py b/rllib/env/single_agent_env_runner.py index e18e32d7010a..5f708fa600a7 100644 --- a/rllib/env/single_agent_env_runner.py +++ b/rllib/env/single_agent_env_runner.py @@ -802,7 +802,7 @@ def _increase_sampled_metrics(self, num_steps): def _log_episode_metrics(self, length, ret, sec): # Log general episode metrics. - # To mimick the old API stack behavior, we'll use `window` here for + # To mimic the old API stack behavior, we'll use `window` here for # these particular stats (instead of the default EMA). win = self.config.metrics_num_episodes_for_smoothing self.metrics.log_value(EPISODE_LEN_MEAN, length, window=win) diff --git a/rllib/tuned_examples/ppo/atari-ppo.yaml b/rllib/tuned_examples/ppo/atari-ppo.yaml deleted file mode 100644 index 44f76cae0e26..000000000000 --- a/rllib/tuned_examples/ppo/atari-ppo.yaml +++ /dev/null @@ -1,35 +0,0 @@ -# Runs on a single g3.16xl node -# See https://github.com/ray-project/rl-experiments for results -atari-ppo: - env: - grid_search: - - ALE/Breakout-v5 - - ALE/BeamRider-v5 - - ALE/Qbert-v5 - - ALE/SpaceInvaders-v5 - run: PPO - config: - # Works for both torch and tf. - framework: torch - # Make analogous to old v4 + NoFrameskip. - env_config: - frameskip: 1 - full_action_space: false - repeat_action_probability: 0.0 - 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 - rollout_fragment_length: 100 - sgd_minibatch_size: 500 - num_sgd_iter: 10 - num_env_runners: 10 - num_envs_per_env_runner: 5 - batch_mode: truncate_episodes - observation_filter: NoFilter - model: - vf_share_layers: true - num_gpus: 1 diff --git a/rllib/tuned_examples/ppo/atari_ppo.py b/rllib/tuned_examples/ppo/atari_ppo.py new file mode 100644 index 000000000000..ee76d8d3f9ce --- /dev/null +++ b/rllib/tuned_examples/ppo/atari_ppo.py @@ -0,0 +1,93 @@ +import gymnasium as gym + +from ray.rllib.algorithms.ppo import PPOConfig +from ray.rllib.connectors.env_to_module.frame_stacking import FrameStackingEnvToModule +from ray.rllib.connectors.learner.frame_stacking import FrameStackingLearner +from ray.rllib.env.wrappers.atari_wrappers import wrap_atari_for_new_api_stack +from ray.rllib.utils.metrics import ( + ENV_RUNNER_RESULTS, + EPISODE_RETURN_MEAN, + NUM_ENV_STEPS_SAMPLED_LIFETIME, +) +from ray.rllib.utils.test_utils import add_rllib_example_script_args +from ray import tune + + +parser = add_rllib_example_script_args() +# Use `parser` to add your own custom command line options to this script +# and (if needed) use their values toset up `config` below. +args = parser.parse_args() + + +def _make_env_to_module_connector(env): + return FrameStackingEnvToModule(num_frames=4) + + +def _make_learner_connector(input_observation_space, input_action_space): + return FrameStackingLearner(num_frames=4) + + +# Create a custom Atari setup (w/o the usual RLlib-hard-coded framestacking in it). +# We would like our frame stacking connector to do this job. +def _env_creator(cfg): + return wrap_atari_for_new_api_stack( + gym.make(args.env, **cfg), + # Perform frame-stacking through ConnectorV2 API. + framestack=None, + ) + + +tune.register_env("env", _env_creator) + + +config = ( + PPOConfig() + .environment( + "env", + env_config={ + # Make analogous to old v4 + NoFrameskip. + "frameskip": 1, + "full_action_space": False, + "repeat_action_probability": 0.0, + }, + clip_rewards=True, + ) + .env_runners( + # num_envs_per_env_runner=5, # 5 on old yaml example + env_to_module_connector=_make_env_to_module_connector, + ) + .training( + learner_connector=_make_learner_connector, + train_batch_size_per_learner=4000, # 5000 on old yaml example + mini_batch_size_per_learner=128, # 500 on old yaml example + lambda_=0.95, + kl_coeff=0.5, + clip_param=0.1, + vf_clip_param=10.0, + entropy_coeff=0.01, + num_sgd_iter=10, + lr=0.00015 * args.num_gpus, + grad_clip=100.0, + grad_clip_by="global_norm", + ) + .rl_module( + model_config_dict={ + "vf_share_layers": True, + "conv_filters": [[16, 4, 2], [32, 4, 2], [64, 4, 2], [128, 4, 2]], + "conv_activation": "relu", + "post_fcnet_hiddens": [256], + "uses_new_env_runners": True, + } + ) +) + +stop = { + f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": 20.0, + NUM_ENV_STEPS_SAMPLED_LIFETIME: 1500000, +} + + +if __name__ == "__main__": + from ray.rllib.utils.test_utils import run_rllib_example_script_experiment + + run_rllib_example_script_experiment(config, args=args, stop=stop) diff --git a/rllib/tuned_examples/ppo/cartpole-ppo-fake-gpus.yaml b/rllib/tuned_examples/ppo/cartpole-ppo-fake-gpus.yaml index af5b338997be..b4815746cf8c 100644 --- a/rllib/tuned_examples/ppo/cartpole-ppo-fake-gpus.yaml +++ b/rllib/tuned_examples/ppo/cartpole-ppo-fake-gpus.yaml @@ -1,3 +1,4 @@ +# @OldAPIStack cartpole-ppo-fake-gpus: env: CartPole-v1 run: PPO diff --git a/rllib/tuned_examples/ppo/cartpole-ppo-grid-search-example.yaml b/rllib/tuned_examples/ppo/cartpole-ppo-grid-search-example.yaml deleted file mode 100644 index 594aa27182ac..000000000000 --- a/rllib/tuned_examples/ppo/cartpole-ppo-grid-search-example.yaml +++ /dev/null @@ -1,14 +0,0 @@ -cartpole-ppo-grid-search-example: - env: CartPole-v1 - run: PPO - stop: - env_runners/episode_return_mean: 200 - time_total_s: 180 - config: - # Works for both torch and tf. - framework: torch - num_env_runners: 2 - num_sgd_iter: - grid_search: [1, 4] - sgd_minibatch_size: - grid_search: [128, 256, 512] diff --git a/rllib/tuned_examples/ppo/cartpole-ppo-hyperband.yaml b/rllib/tuned_examples/ppo/cartpole-ppo-hyperband.yaml deleted file mode 100644 index 2d76a3e98eb6..000000000000 --- a/rllib/tuned_examples/ppo/cartpole-ppo-hyperband.yaml +++ /dev/null @@ -1,16 +0,0 @@ -cartpole-ppo: - env: CartPole-v1 - run: PPO - num_samples: 3 - stop: - env_runners/episode_return_mean: 200 - time_total_s: 180 - config: - # Works for both torch and tf. - framework: torch - num_env_runners: 1 - num_sgd_iter: - grid_search: [1, 4] - sgd_minibatch_size: - grid_search: [128, 256, 512] - observation_filter: MeanStdFilter diff --git a/rllib/tuned_examples/ppo/pong-ppo.yaml b/rllib/tuned_examples/ppo/pong-ppo.yaml deleted file mode 100644 index 8e1419024ae2..000000000000 --- a/rllib/tuned_examples/ppo/pong-ppo.yaml +++ /dev/null @@ -1,33 +0,0 @@ -# On a single GPU, this achieves maximum reward in ~15-20 minutes. -# -# $ python train.py -f tuned_configs/pong-ppo.yaml -# -pong-ppo: - env: ALE/Pong-v5 - run: PPO - config: - # Works for both torch and tf. - framework: torch - # Make analogous to old v4 + NoFrameskip. - env_config: - frameskip: 1 - full_action_space: false - repeat_action_probability: 0.0 - 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 - rollout_fragment_length: auto - sgd_minibatch_size: 500 - num_sgd_iter: 10 - num_env_runners: 32 - num_envs_per_env_runner: 5 - batch_mode: truncate_episodes - observation_filter: NoFilter - num_gpus: 1 - model: - dim: 42 - vf_share_layers: true diff --git a/rllib/tuned_examples/ppo/recomm-sys001-ppo.yaml b/rllib/tuned_examples/ppo/recomm-sys001-ppo.yaml deleted file mode 100644 index 937da8c4b547..000000000000 --- a/rllib/tuned_examples/ppo/recomm-sys001-ppo.yaml +++ /dev/null @@ -1,49 +0,0 @@ -recomm-sys001-ppo: - env: ray.rllib.examples.envs.classes.recommender_system_envs.RecommSys001 - run: PPO - stop: - #evaluation/env_runners/episode_return_mean: 48.0 - timesteps_total: 200000 - config: - framework: torch - - metrics_num_episodes_for_smoothing: 1000 - - # Env c'tor kwargs: - env_config: - # Number of different categories a doc can have and a user can - # have a preference for. - num_categories: 2 - # Number of docs to choose (a slate) from each timestep. - num_docs_to_select_from: 10 - # Slate size. - slate_size: 1 - # Re-sample docs each timesteps. - num_docs_in_db: 100 - # Re-sample user each episode. - num_users_in_db: 100 - # User time budget (determines lengths of episodes). - user_time_budget: 60.0 - - # Larger networks seem to help (large obs/action spaces). - model: - fcnet_hiddens: [256, 256] - - # Larger batch sizes seem to help (more stability, even with higher lr). - #train_batch_size: 32 - - #num_env_runners: 2 - #num_gpus: 0 - - #lr_choice_model: 0.002 - #lr_q_model: 0.002 - - #target_network_update_freq: 500 - #tau: 1.0 - - # Evaluation settings. - evaluation_interval: 1 - evaluation_num_env_runners: 4 - evaluation_duration: 200 - evaluation_duration_unit: episodes - evaluation_parallel_to_training: true diff --git a/rllib/utils/test_utils.py b/rllib/utils/test_utils.py index e6bd99c11cfb..6fe3f8069d43 100644 --- a/rllib/utils/test_utils.py +++ b/rllib/utils/test_utils.py @@ -6,6 +6,7 @@ from gymnasium.spaces import Dict as GymDict from gymnasium.spaces import Tuple as GymTuple import inspect +import json import logging import numpy as np import os @@ -220,6 +221,14 @@ def add_rllib_example_script_args( "be achieved within --stop-timesteps AND --stop-iters, otherwise this " "script will throw an exception at the end.", ) + parser.add_argument( + "--as-release-test", + action="store_true", + help="Whether this script should be run as a release test. If set, " + "all that applies to the --as-test option is true, plus, a short JSON summary " + "will be written into a results file whose location is given by the ENV " + "variable `TEST_OUTPUT_JSON`.", + ) # Learner scaling options. # Old API stack: config.num_gpus. @@ -630,8 +639,8 @@ def check_learning_achieved( Raises: ValueError: If `min_reward` not reached. """ - # Get maximum reward of all trials - # (check if at least one trial achieved some learning) + # Get maximum value of `metrics` over all trials + # (check if at least one trial achieved some learning, not just the final one). recorded_values = [] for _, row in tune_results.get_dataframe().iterrows(): if evaluation or ( @@ -1373,11 +1382,15 @@ def run_rllib_example_script_experiment( parser = add_rllib_example_script_args() args = parser.parse_args() + # If run --as-release-test, --as-test must also be set. + if args.as_release_test: + args.as_test = True + # Initialize Ray. ray.init(num_cpus=args.num_cpus or None, local_mode=args.local_mode) # Define one or more stopping criteria. - if not stop: + if stop is None: stop = { f"{ENV_RUNNER_RESULTS}/episode_return_mean": args.stop_reward, f"{NUM_ENV_STEPS_SAMPLED_LIFETIME}": args.stop_timesteps, @@ -1391,8 +1404,8 @@ def run_rllib_example_script_experiment( # Set the framework. config.framework(args.framework) - # Add an env specifier? - if args.env is not None: + # Add an env specifier (only if not already set in config)? + if args.env is not None and config.env is None: config.environment(args.env) # Enable the new API stack? @@ -1423,6 +1436,7 @@ def run_rllib_example_script_experiment( # Run the experiment w/o Tune (directly operate on the RLlib Algorithm object). if args.no_tune: + assert not args.as_test and not args.as_release_test algo = config.build() for _ in range(stop.get(TRAINING_ITERATION, args.stop_iters)): results = algo.train() @@ -1443,7 +1457,9 @@ def run_rllib_example_script_experiment( break if val is not None and not np.isnan(val) and val >= threshold: print(f"Stop criterium ({key}={threshold}) fulfilled!") + ray.shutdown() return results + ray.shutdown() return results @@ -1490,6 +1506,7 @@ def run_rllib_example_script_experiment( os.environ["RAY_AIR_NEW_OUTPUT"] = "0" # Run the actual experiment (using Tune). + start_time = time.time() results = tune.Tuner( trainable or config.algo_class, param_space=config, @@ -1505,30 +1522,61 @@ def run_rllib_example_script_experiment( ), tune_config=tune.TuneConfig(num_samples=args.num_samples), ).fit() + time_taken = time.time() - start_time + + ray.shutdown() # If run as a test, check whether we reached the specified success criteria. + test_passed = False if args.as_test: # Success metric not provided, try extracting it from `stop`. if success_metric is None: for try_it in [ - f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/episode_return_mean", - f"{ENV_RUNNER_RESULTS}/episode_return_mean", + f"{EVALUATION_RESULTS}/{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}", + f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}", ]: if try_it in stop: success_metric = {try_it: stop[try_it]} break if success_metric is None: success_metric = { - f"{ENV_RUNNER_RESULTS}/episode_return_mean": args.stop_reward, + f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": args.stop_reward, } # TODO (sven): Make this work for more than one metric (AND-logic?). - metric = next(iter(success_metric.keys())) - check_learning_achieved( - tune_results=results, - metric=metric, - min_value=success_metric[metric], + # Get maximum value of `metric` over all trials + # (check if at least one trial achieved some learning, not just the final one). + success_metric_key, success_metric_value = next(iter(success_metric.items())) + best_value = max( + row[success_metric_key] for _, row in results.get_dataframe().iterrows() ) - ray.shutdown() + if best_value >= success_metric_value: + test_passed = True + print(f"`{success_metric_key}` of {success_metric_value} reached! ok") + + if args.as_release_test: + trial = results._experiment_analysis.trials[0] + stats = trial.last_result + stats.pop("config", None) + json_summary = { + "time_taken": float(time_taken), + "trial_states": [trial.status], + "last_update": float(time.time()), + "stats": stats, + "passed": [test_passed], + "not_passed": [not test_passed], + "failures": {str(trial): 1} if not test_passed else {}, + } + with open( + os.environ.get("TEST_OUTPUT_JSON", "/tmp/learning_test.json"), + "wt", + ) as f: + json.dump(json_summary, f) + + if not test_passed: + raise ValueError( + f"`{success_metric_key}` of {success_metric_value} not reached!" + ) + return results From e528cb04f2fae2ed6a5ac35d4a82917358d16d3c Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Thu, 30 May 2024 12:56:54 -0700 Subject: [PATCH 53/65] [core] add EC2InstanceTerminator and refactor killer creation (#45630) Signed-off-by: hongchaodeng --- python/ray/_private/test_utils.py | 62 ++++++++++++------- python/ray/tests/conftest.py | 4 +- python/ray/tests/test_chaos.py | 7 +-- .../chaos_test/test_chaos_basic.py | 2 +- release/nightly_tests/setup_chaos.py | 41 +++++++++--- 5 files changed, 78 insertions(+), 38 deletions(-) diff --git a/python/ray/_private/test_utils.py b/python/ray/_private/test_utils.py index 62e51016af2e..9e56c2b148fa 100644 --- a/python/ray/_private/test_utils.py +++ b/python/ray/_private/test_utils.py @@ -1493,8 +1493,7 @@ async def get_total_killed(self): return self.killed -@ray.remote(num_cpus=0) -class NodeKillerActor(ResourceKillerActor): +class NodeKillerBase(ResourceKillerActor): async def _find_resource_to_kill(self): node_to_kill_ip = None node_to_kill_port = None @@ -1521,6 +1520,16 @@ async def _find_resource_to_kill(self): return node_id, node_to_kill_ip, node_to_kill_port + def _get_alive_nodes(self, nodes): + alive_nodes = 0 + for node in nodes: + if node["Alive"]: + alive_nodes += 1 + return alive_nodes + + +@ray.remote(num_cpus=0) +class RayletKiller(NodeKillerBase): def _kill_resource(self, node_id, node_to_kill_ip, node_to_kill_port): if node_to_kill_port is not None: try: @@ -1533,6 +1542,33 @@ def _kill_resource(self, node_id, node_to_kill_ip, node_to_kill_port): ) self.killed.add(node_id) + def _kill_raylet(self, ip, port, graceful=False): + import grpc + from grpc._channel import _InactiveRpcError + from ray.core.generated import node_manager_pb2_grpc + + raylet_address = f"{ip}:{port}" + channel = grpc.insecure_channel(raylet_address) + stub = node_manager_pb2_grpc.NodeManagerServiceStub(channel) + try: + stub.ShutdownRaylet( + node_manager_pb2.ShutdownRayletRequest(graceful=graceful) + ) + except _InactiveRpcError: + assert not graceful + + +@ray.remote(num_cpus=0) +class EC2InstanceTerminator(NodeKillerBase): + def _kill_resource(self, node_id, node_to_kill_ip, _): + if node_to_kill_ip is not None: + try: + self._terminate_ec2_instance(node_to_kill_ip) + except Exception: + pass + logging.info(f"Terminated instance, {node_id=}, address={node_to_kill_ip}") + self.killed.add(node_id) + def _terminate_ec2_instance(self, ip): # This command uses IMDSv2 to get the host instance id and region. # After that it terminates itself using aws cli. @@ -1552,28 +1588,6 @@ def _terminate_ec2_instance(self, ip): print(f"STDOUT:\n{result.stdout}\n") print(f"STDERR:\n{result.stderr}\n") - def _kill_raylet(self, ip, port, graceful=False): - import grpc - from grpc._channel import _InactiveRpcError - from ray.core.generated import node_manager_pb2_grpc - - raylet_address = f"{ip}:{port}" - channel = grpc.insecure_channel(raylet_address) - stub = node_manager_pb2_grpc.NodeManagerServiceStub(channel) - try: - stub.ShutdownRaylet( - node_manager_pb2.ShutdownRayletRequest(graceful=graceful) - ) - except _InactiveRpcError: - assert not graceful - - def _get_alive_nodes(self, nodes): - alive_nodes = 0 - for node in nodes: - if node["Alive"]: - alive_nodes += 1 - return alive_nodes - @ray.remote(num_cpus=0) class WorkerKillerActor(ResourceKillerActor): diff --git a/python/ray/tests/conftest.py b/python/ray/tests/conftest.py index fcc70b023918..ead0ec9648ad 100644 --- a/python/ray/tests/conftest.py +++ b/python/ray/tests/conftest.py @@ -37,7 +37,7 @@ find_available_port, wait_for_condition, find_free_port, - NodeKillerActor, + RayletKiller, ) from ray.cluster_utils import AutoscalingCluster, Cluster, cluster_not_supported @@ -921,7 +921,7 @@ def _ray_start_chaos_cluster(request): assert len(nodes) == 1 if kill_interval is not None: - node_killer = get_and_run_resource_killer(NodeKillerActor, kill_interval) + node_killer = get_and_run_resource_killer(RayletKiller, kill_interval) yield cluster diff --git a/python/ray/tests/test_chaos.py b/python/ray/tests/test_chaos.py index f59308c4142b..3ec1e0cce317 100644 --- a/python/ray/tests/test_chaos.py +++ b/python/ray/tests/test_chaos.py @@ -12,7 +12,7 @@ from ray.data._internal.progress_bar import ProgressBar from ray.util.placement_group import placement_group from ray._private.test_utils import ( - NodeKillerActor, + RayletKiller, get_log_message, get_and_run_resource_killer, WorkerKillerActor, @@ -337,15 +337,14 @@ def test_node_killer_filter(autoscaler_v2): worker_nodes = [node for node in list_nodes() if not node["is_head_node"]] node_to_kill = random.choice(worker_nodes) node_killer = get_and_run_resource_killer( - NodeKillerActor, + RayletKiller, 1, max_to_kill=1, kill_filter_fn=lambda: lambda node: node["NodeID"] == node_to_kill.node_id, ) def check_killed(): - # Check that killed node is consistent across list_nodes() and - # NodeKillerActor + # Check that killed node is consistent across list_nodes() killed = list(ray.get(node_killer.get_total_killed.remote())) dead = [node.node_id for node in list_nodes() if node.state == "DEAD"] if len(killed) != 1 or len(dead) != 1: diff --git a/release/nightly_tests/chaos_test/test_chaos_basic.py b/release/nightly_tests/chaos_test/test_chaos_basic.py index 50b772caac1f..9cf120b0265b 100644 --- a/release/nightly_tests/chaos_test/test_chaos_basic.py +++ b/release/nightly_tests/chaos_test/test_chaos_basic.py @@ -206,7 +206,7 @@ def main(): # Step 3 print("Running with failures") start = time.time() - node_killer = ray.get_actor("NodeKillerActor", namespace="release_test_namespace") + node_killer = ray.get_actor("RayletKiller", namespace="release_test_namespace") node_killer.run.remote() workload(total_num_cpus, args.smoke) print(f"Runtime when there are many failures: {time.time() - start}") diff --git a/release/nightly_tests/setup_chaos.py b/release/nightly_tests/setup_chaos.py index d1f6b7fbf3d9..b88c81ed3db2 100644 --- a/release/nightly_tests/setup_chaos.py +++ b/release/nightly_tests/setup_chaos.py @@ -5,14 +5,28 @@ from ray._private.test_utils import ( get_and_run_resource_killer, - NodeKillerActor, + RayletKiller, WorkerKillerActor, + EC2InstanceTerminator, ) def parse_script_args(): parser = argparse.ArgumentParser() + + # '--kill-workers' to be deprecated in favor of '--chaos' parser.add_argument("--kill-workers", action="store_true", default=False) + + parser.add_argument( + "--chaos", + type=str, + default="", + help=( + "Chaos to inject into the test environment. " + "Options: KillRaylet, KillWorker, TerminateEC2Instance." + ), + ) + parser.add_argument("--kill-interval", type=int, default=60) parser.add_argument("--max-to-kill", type=int, default=2) parser.add_argument( @@ -77,6 +91,24 @@ def _filter_fn(node): return _task_node_filter +def get_chaos_killer(args): + if args.chaos != "": + chaos_type = args.chaos + elif args.kill_workers: + chaos_type = "KillWorker" + else: + chaos_type = "KillRaylet" # default + + if chaos_type == "KillRaylet": + return RayletKiller, task_node_filter(args.task_names) + elif chaos_type == "KillWorker": + return WorkerKillerActor, task_filter(args.task_names) + elif chaos_type == "TerminateEC2Instance": + return EC2InstanceTerminator, task_node_filter(args.task_names) + else: + raise ValueError(f"Chaos type {chaos_type} not supported.") + + def main(): """Start the chaos testing. @@ -84,12 +116,7 @@ def main(): """ args, _ = parse_script_args() ray.init(address="auto") - if args.kill_workers: - resource_killer_cls = WorkerKillerActor - kill_filter_fn = task_filter(args.task_names) - else: - resource_killer_cls = NodeKillerActor - kill_filter_fn = task_node_filter(args.task_names) + resource_killer_cls, kill_filter_fn = get_chaos_killer(args) get_and_run_resource_killer( resource_killer_cls, From ff3e39398dc2dcce35de793249e736f0e9fce57d Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 30 May 2024 14:09:12 -0700 Subject: [PATCH 54/65] add oss tag to container tests (#45629) add oss tag to container tests Add `oss` tag to container tests. Signed-off-by: Cindy Zhang Signed-off-by: Cindy Zhang --- .buildkite/core.rayci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 0c0bb5921e75..cbc2b9a17259 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -315,6 +315,7 @@ steps: tags: - python - docker + - oss instance_type: medium commands: - bazel run //ci/ray_ci:build_in_docker -- docker --platform cpu From 18eb4337552c5e94be730f0ea8a55283fb48f641 Mon Sep 17 00:00:00 2001 From: Zhi Lin Date: Fri, 31 May 2024 05:09:31 +0800 Subject: [PATCH 55/65] [HPU] [Train] Add a Stable Diffusion fine-tuning and serving example (#45217) This PR adds an example for stable diffusion model fine-tuning and serving using HPU. Moreover, it also covers how to adapt an existing HPU example to run on Ray, so that users can use Ray to run the examples on huggingface/optimum-habana. --------- Signed-off-by: Zhi Lin Signed-off-by: Yunxuan Xiao Signed-off-by: Samuel Chan <116198444+anyscalesam@users.noreply.github.com> Co-authored-by: Yunxuan Xiao Co-authored-by: Yunxuan Xiao Co-authored-by: Samuel Chan <116198444+anyscalesam@users.noreply.github.com> Co-authored-by: Peyton Murray --- doc/source/train/examples.yml | 10 + .../train/examples/intel_gaudi/sd.ipynb | 427 ++++++++++++++++++ 2 files changed, 437 insertions(+) create mode 100644 doc/source/train/examples/intel_gaudi/sd.ipynb diff --git a/doc/source/train/examples.yml b/doc/source/train/examples.yml index 1a828ddc1714..505c757e92ae 100644 --- a/doc/source/train/examples.yml +++ b/doc/source/train/examples.yml @@ -73,6 +73,16 @@ examples: - computer vision - generative ai link: examples/pytorch/dreambooth_finetuning + - title: Finetune Stable Diffusion and generate images with Intel Gaudi + skill_level: intermediate + frameworks: + - accelerate + - transformers + use_cases: + - computer vision + - generative ai + contributor: community + link: examples/intel_gaudi/sd - title: Train a text classifier with PyTorch Lightning and Ray Data frameworks: - lightning diff --git a/doc/source/train/examples/intel_gaudi/sd.ipynb b/doc/source/train/examples/intel_gaudi/sd.ipynb new file mode 100644 index 000000000000..efe2146781ce --- /dev/null +++ b/doc/source/train/examples/intel_gaudi/sd.ipynb @@ -0,0 +1,427 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Finetune Stable Diffusion and generate images with Intel Gaudi\n", + "In this tutorial, we will go through how to finetune a stable diffusion model and generate images with Intel Gaudi(HPU). Moreover, we will show how to adapt an existing HPU example to use Ray. Once you learned how to make the adaption, you can easily access more models and optimizations that has been developed for HPU by \"Ray-ifying\" examples from [optimum-habana/examples](https://github.com/huggingface/optimum-habana/tree/main/examples) and [Model References](https://github.com/HabanaAI/Model-References)!\n", + "\n", + "Now, let's see how we can \"Ray-ify\" this [stable diffusion example](https://github.com/huggingface/optimum-habana/tree/main/examples/stable-diffusion)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Configuration\n", + "A node with Gaudi/Gaudi2 installed is required to run this example. Both Gaudi and Gaudi2 have 8 HPUs.\n", + "\n", + "We recommend using a prebuilt container to run these examples. To run a container, you need Docker. See [Install Docker Engine](https://docs.docker.com/engine/install/) for installation instructions.\n", + "\n", + "Next, follow [Run Using Containers](https://docs.habana.ai/en/latest/Installation_Guide/Bare_Metal_Fresh_OS.html?highlight=installer#run-using-containers) to install the Gaudi drivers and container runtime.\n", + "\n", + "Then, start the Gaudi container:\n", + "```bash\n", + "docker pull vault.habana.ai/gaudi-docker/1.15.1/ubuntu22.04/habanalabs/pytorch-installer-2.2.0:latest\n", + "docker run -it --runtime=habana -e HABANA_VISIBLE_DEVICES=all -e OMPI_MCA_btl_vader_single_copy_mechanism=none --cap-add=sys_nice --net=host --ipc=host vault.habana.ai/gaudi-docker/1.15.1/ubuntu22.04/habanalabs/pytorch-installer-2.2.0:latest\n", + "```\n", + "\n", + "Inside the container, clone [Optimum-Habana](https://github.com/huggingface/optimum-habana/) and install the dependencies:\n", + "```bash\n", + "git clone https://github.com/huggingface/optimum-habana.git\n", + "pip install ray[train,serve] optimum-habana\n", + "cd optimum-habana/\n", + "pip install -r examples/stable-diffusion/requirements.txt\n", + "pip install -r examples/stable-diffusion/training/requirements.txt\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Fine-tuning with Textual Inversion\n", + "First, let's start with fine-tuning. Check `examples/stable-diffusion/training/textual_inversion.py` which fine-tunes a Stable Diffusion model on HPU. You can follow [this document](https://github.com/huggingface/optimum-habana/blob/main/examples/stable-diffusion/training/README.md#textual-inversion) and try it once without using Ray.\n", + "\n", + "In order to run this script on Ray, we need to make some changes. But don't worry, it's actually pretty simple. Basically, we just need to identify the main training loop, and run it in TorchTrainer.\n", + "\n", + "First, check this block at the end of the file:\n", + "```python\n", + "if __name__ == \"__main__\":\n", + " main()\n", + "```\n", + "\n", + "Originally, this script will be started by MPI if multiple workers are used. But with Ray, we should setup TorchTrainer and supply a main function, which is `main()` in this example.\n", + "\n", + "Therefore, it becomes straight-forward to make these changes:\n", + "```python\n", + "if __name__ == \"__main__\":\n", + " import ray\n", + " from ray import train\n", + " from ray.train import ScalingConfig, Checkpoint, CheckpointConfig, RunConfig\n", + " from ray.train.torch import TorchTrainer, TorchConfig\n", + "\n", + " ray.init(address=\"auto\")\n", + "\n", + " # Configure computation resources\n", + " # In ScalingConfig, require an HPU for each worker\n", + " scaling_config = ScalingConfig(num_workers=1, resources_per_worker={\"CPU\": 1, \"HPU\": 1})\n", + " # Set backend to hccl in TorchConfig\n", + " torch_config = TorchConfig(backend = \"hccl\")\n", + " # Initialize a Ray TorchTrainer\n", + " trainer = TorchTrainer(\n", + " train_loop_per_worker=main,\n", + " torch_config=torch_config,\n", + " scaling_config=scaling_config,\n", + " )\n", + "\n", + " result = trainer.fit()\n", + "```\n", + "\n", + "Before we try to run, we need to inspect the `main` function to see if it can work in such way. As we skim through the function, it's clear that it does not take any input parameters, but it calls `parse_args` to get all configurations. Originally, these configurations are set in command line by MPI. But because we switch to Ray to start the workers, command line arguments are no longer accessible. Therefore, `parse_args` should be called in the main program and passed to `main` function.\n", + "\n", + "Apart from this, no other changes are necessary. By inserting the following code, you can now run the script on Ray." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Replace lines below:\n", + "# def main():\n", + "# args = parse_args()\n", + "# with these lines:\n", + "def main(config):\n", + " args = config[\"args\"]" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Replace lines below:\n", + "# if __name__ == \"__main__\":\n", + "# main()\n", + "# with these lines:\n", + "if __name__ == \"__main__\":\n", + " import ray\n", + " from ray import train\n", + " from ray.train import ScalingConfig, Checkpoint, CheckpointConfig, RunConfig\n", + " from ray.train.torch import TorchTrainer, TorchConfig\n", + "\n", + " ray.init(address=\"auto\")\n", + "\n", + " # Configure computation resources\n", + " # In ScalingConfig, require an HPU for each worker\n", + " scaling_config = ScalingConfig(num_workers=1, resources_per_worker={\"CPU\": 1, \"HPU\": 1})\n", + " # Set backend to hccl in TorchConfig\n", + " torch_config = TorchConfig(backend = \"hccl\")\n", + " # Initialize a Ray TorchTrainer\n", + " trainer = TorchTrainer(\n", + " train_loop_per_worker=main,\n", + "\t\ttrain_loop_config={\"args\": parse_args()},\n", + " torch_config=torch_config,\n", + " scaling_config=scaling_config,\n", + " )\n", + "\n", + " result = trainer.fit()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "One last thing: remember to use absolute path in the command line arguments. The reason is similar to why we move `parse_args` out, Ray's workers do not share the current working directory. Now, you can run the fine-tuning of Stable Diffusion on Ray!\n", + "An example command:\n", + "```bash\n", + "python /root/optimum-habana/examples/stable-diffusion/training/textual_inversion.py \\\n", + " --pretrained_model_name_or_path runwayml/stable-diffusion-v1-5 \\\n", + " --train_data_dir \"/root/cat\" \\\n", + " --learnable_property object \\\n", + " --placeholder_token \"\" \\\n", + " --initializer_token toy \\\n", + " --resolution 512 \\\n", + " --train_batch_size 4 \\\n", + " --max_train_steps 3000 \\\n", + " --learning_rate 5.0e-04 \\\n", + " --scale_lr \\\n", + " --lr_scheduler constant \\\n", + " --lr_warmup_steps 0 \\\n", + " --output_dir /tmp/textual_inversion_cat \\\n", + " --save_as_full_pipeline \\\n", + " --gaudi_config_name Habana/stable-diffusion \\\n", + " --throughput_warmup_steps 3\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "The output of the example command:\n", + "```bash\n", + "Training started with configuration:\n", + "╭───────────────────────────────────────────────╮\n", + "│ Training config │\n", + "├───────────────────────────────────────────────┤\n", + "│ train_loop_config/args ...t_warmup_steps=3) │\n", + "╰───────────────────────────────────────────────╯\n", + "(RayTrainWorker pid=15683) Setting up process group for: env:// [rank=0, world_size=1]\n", + "(TorchTrainer pid=15530) Started distributed worker processes: \n", + "(TorchTrainer pid=15530) - (ip=172.17.0.2, pid=15683) world_rank=0, local_rank=0, node_rank=0\n", + "(RayTrainWorker pid=15683) [WARNING|utils.py:185] 2024-05-09 05:21:13,961 >> optimum-habana v1.10.4 has been validated for SynapseAI v1.14.0 but habana-frameworks v1.15.1.15 was found, this could lead to undefined behavior!\n", + "(RayTrainWorker pid=15683) [WARNING|utils.py:198] 2024-05-09 05:21:15,401 >> optimum-habana v1.10.4 has been validated for SynapseAI v1.14.0 but the driver version is v1.15.0, this could lead to undefined behavior!\n", + "(RayTrainWorker pid=15683) /usr/local/lib/python3.10/dist-packages/diffusers/utils/outputs.py:63: UserWarning: torch.utils._pytree._register_pytree_node is deprecated. Please use torch.utils._pytree.register_pytree_node instead.\n", + "(RayTrainWorker pid=15683) torch.utils._pytree._register_pytree_node(\n", + "(RayTrainWorker pid=15683) /usr/local/lib/python3.10/dist-packages/huggingface_hub/file_download.py:1132: FutureWarning: `resume_download` is deprecated and will be removed in version 1.0.0. Downloads always resume when possible. If you want to force a new download, use `force_download=True`.\n", + "(RayTrainWorker pid=15683) warnings.warn(\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:15 - INFO - __main__ - Distributed environment: MULTI_HPU Backend: hccl\n", + "(RayTrainWorker pid=15683) Num processes: 1\n", + "(RayTrainWorker pid=15683) Process index: 0\n", + "(RayTrainWorker pid=15683) Local process index: 0\n", + "(RayTrainWorker pid=15683) Device: hpu\n", + "(RayTrainWorker pid=15683) \n", + "(RayTrainWorker pid=15683) Mixed precision type: bf16\n", + "(RayTrainWorker pid=15683) \n", + "(RayTrainWorker pid=15683) {'timestep_spacing', 'rescale_betas_zero_snr', 'dynamic_thresholding_ratio', 'prediction_type', 'thresholding', 'sample_max_value', 'clip_sample_range'} was not found in config. Values will be initialized to default values.\n", + "(RayTrainWorker pid=15683) ============================= HABANA PT BRIDGE CONFIGURATION =========================== \n", + "(RayTrainWorker pid=15683) PT_HPU_LAZY_MODE = 1\n", + "(RayTrainWorker pid=15683) PT_RECIPE_CACHE_PATH = \n", + "(RayTrainWorker pid=15683) PT_CACHE_FOLDER_DELETE = 0\n", + "(RayTrainWorker pid=15683) PT_HPU_RECIPE_CACHE_CONFIG = \n", + "(RayTrainWorker pid=15683) PT_HPU_MAX_COMPOUND_OP_SIZE = 9223372036854775807\n", + "(RayTrainWorker pid=15683) PT_HPU_LAZY_ACC_PAR_MODE = 0\n", + "(RayTrainWorker pid=15683) PT_HPU_ENABLE_REFINE_DYNAMIC_SHAPES = 0\n", + "(RayTrainWorker pid=15683) ---------------------------: System Configuration :---------------------------\n", + "(RayTrainWorker pid=15683) Num CPU Cores : 152\n", + "(RayTrainWorker pid=15683) CPU RAM : 1056440348 KB\n", + "(RayTrainWorker pid=15683) ------------------------------------------------------------------------------\n", + "(RayTrainWorker pid=15683) {'scaling_factor', 'force_upcast'} was not found in config. Values will be initialized to default values.\n", + "(RayTrainWorker pid=15683) {'addition_embed_type', 'resnet_out_scale_factor', 'transformer_layers_per_block', 'attention_type', 'conv_out_kernel', 'time_embedding_type', 'addition_embed_type_num_heads', 'dropout', 'only_cross_attention', 'projection_class_embeddings_input_dim', 'num_attention_heads', 'upcast_attention', 'reverse_transformer_layers_per_block', 'resnet_time_scale_shift', 'resnet_skip_time_act', 'time_embedding_act_fn', 'class_embeddings_concat', 'time_cond_proj_dim', 'num_class_embeds', 'mid_block_type', 'cross_attention_norm', 'addition_time_embed_dim', 'dual_cross_attention', 'mid_block_only_cross_attention', 'encoder_hid_dim_type', 'time_embedding_dim', 'class_embed_type', 'conv_in_kernel', 'timestep_post_act', 'encoder_hid_dim', 'use_linear_projection'} was not found in config. Values will be initialized to default values.\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - ***** Running training *****\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - Num examples = 600\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - Num Epochs = 20\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - Instantaneous batch size per device = 4\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - Total train batch size (w. parallel, distributed & accumulation) = 4\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - Gradient Accumulation steps = 1\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:21:20 - INFO - __main__ - Total optimization steps = 3000\n", + "Steps: 0%| | 0/3000 [00:00> Running on CPU.\n", + "(RayTrainWorker pid=15683) Configuration saved in /tmp/textual_inversion_cat/vae/config.json\n", + "(RayTrainWorker pid=15683) Model weights saved in /tmp/textual_inversion_cat/vae/diffusion_pytorch_model.safetensors\n", + "(RayTrainWorker pid=15683) Configuration saved in /tmp/textual_inversion_cat/unet/config.json\n", + "(RayTrainWorker pid=15683) Model weights saved in /tmp/textual_inversion_cat/unet/diffusion_pytorch_model.safetensors\n", + "(RayTrainWorker pid=15683) Configuration saved in /tmp/textual_inversion_cat/scheduler/scheduler_config.json\n", + "(RayTrainWorker pid=15683) Configuration saved in /tmp/textual_inversion_cat/model_index.json\n", + "(RayTrainWorker pid=15683) 05/09/2024 05:40:31 - INFO - __main__ - Saving embeddings\n", + "Steps: 100%|██████████| 3000/3000 [19:10<00:00, 2.61it/s, loss=0.0261, lr=0.002]\n", + "\n", + "Training completed after 0 iterations at 2024-05-09 05:40:33. Total running time: 19min 30s\n", + "2024-05-09 05:40:33,116\tINFO tune.py:1007 -- Wrote the latest version of all result files and experiment state to '/root/ray_results/TorchTrainer_2024-05-09_05-21-02' in 0.0022s.\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "One of the advantages of Ray is that it scales easily. In this example, we can easily scale the training to multiple workers by changing `num_workers` in `ScalingConfig`. Torch distributed environment will be automatically initialized in Ray." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Serve the fine-tuned model on Ray\n", + "Now that we have fine-tuned a Stable Diffusion model, we can serve it for image generation. The code below loads the fine-tuned model and generates an image." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import torch\n", + "from optimum.habana.diffusers import GaudiStableDiffusionPipeline\n", + "model_id = \"/tmp/textual_inversion_cat/\"\n", + "pipe = GaudiStableDiffusionPipeline.from_pretrained(\n", + " model_id,\n", + " torch_dtype=torch.bfloat16,\n", + " use_habana=True,\n", + " use_hpu_graphs=True,\n", + " gaudi_config=\"Habana/stable-diffusion\",\n", + ")\n", + "prompt = \"a is dancing on the grass.\"\n", + "image = pipe(prompt, num_inference_steps=50, guidance_scale=7.5).images[0]\n", + "image.save(\"cat-backpack.png\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "We can easily use Ray Serve to deploy it as an HTTP service. The code below is modified from this [example](https://docs.ray.io/en/master/serve/tutorials/stable-diffusion.html). Save it to `gaudi_sd_deploy.py`, and use `serve run gaudi_sd_deploy:entrypoint` to start the Serve application." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import torch\n", + "from optimum.habana.diffusers import GaudiStableDiffusionPipeline\n", + "from io import BytesIO\n", + "from fastapi import FastAPI\n", + "from fastapi.responses import Response\n", + "\n", + "from ray import serve\n", + "from ray.serve.handle import DeploymentHandle\n", + "\n", + "\n", + "app = FastAPI()\n", + "\n", + "\n", + "@serve.deployment(num_replicas=1)\n", + "@serve.ingress(app)\n", + "class APIIngress:\n", + " def __init__(self, diffusion_model_handle: DeploymentHandle) -> None:\n", + " self.handle = diffusion_model_handle\n", + "\n", + " @app.get(\n", + " \"/imagine\",\n", + " responses={200: {\"content\": {\"image/png\": {}}}},\n", + " response_class=Response,\n", + " )\n", + " async def generate(self, prompt: str, img_size: int = 512):\n", + " assert len(prompt), \"prompt parameter cannot be empty\"\n", + "\n", + " image = await self.handle.generate.remote(prompt, img_size=img_size)\n", + " file_stream = BytesIO()\n", + " image.save(file_stream, \"PNG\")\n", + " return Response(content=file_stream.getvalue(), media_type=\"image/png\")\n", + "\n", + "\n", + "@serve.deployment(\n", + " ray_actor_options={\"resources\": {\"HPU\": 1}}\n", + ")\n", + "class GaudiStableDiffusion:\n", + " def __init__(self, model_id):\n", + " self.pipe = GaudiStableDiffusionPipeline.from_pretrained(\n", + " model_id,\n", + " torch_dtype=torch.bfloat16,\n", + " use_habana=True,\n", + " use_hpu_graphs=True,\n", + " gaudi_config=\"Habana/stable-diffusion\",\n", + " )\n", + "\n", + " def generate(self, prompt: str, img_size: int = 512):\n", + " assert len(prompt), \"prompt parameter cannot be empty\"\n", + "\n", + " image = self.pipe(prompt, num_inference_steps=50, guidance_scale=7.5).images[0]\n", + "\t\treturn image\n", + "\n", + "\n", + "entrypoint = APIIngress.bind(GaudiStableDiffusion.bind(\"/tmp/textual_inversion_cat/\"))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "After you successfully deployed this Serve application, run the code below to generate an image." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import requests\n", + "\n", + "prompt = \"a is dancing on the grass.\"\n", + "input = \"%20\".join(prompt.split(\" \"))\n", + "resp = requests.get(f\"http://127.0.0.1:8000/imagine?prompt={input}\")\n", + "with open(\"output.png\", 'wb') as f:\n", + " f.write(resp.content)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Here is an example image:" + ] + }, + { + "cell_type": "code", + "execution_count": 1, + "metadata": { + "metadata": {} + }, + "outputs": [ + { + "data": { + "image/png": "iVBORw0KGgoAAAANSUhEUgAAAgAAAAIACAIAAAB7GkOtAAEAAElEQVR4nFT92ZJkW7IcBqraWts9IjLzzFWnhlu4vGgAbAAEQXZThC9s6Y/oj+uf6ieKdFNIELhTTWfOKSLcfa9l2g9qa0eirtxTpzIj3Pdeg5mampoZ/9//r393nxfNQETkBDTBaCFlUAIktegTCklKkEqAQYaQASUJISJSIgQIRCYaIwVC/qtgZCbDfwmCSESEMsnIOREtSH+CIEkRBP0QEEGKSoEgIUIEBYhMAFCIJEBAFMGcimgSoNkaMycZKQoKChBAiAQESiIACCTkd0lI9f3pl9ojQiTYxOhbv7TPdfoc999ERMxr3q5dT/P6dNf2nKMjcyRmMohMKZFgC2HK/y91MqW1ikyRgiD4+SIgpNIrkUJEyykGoQkghWATKGQLSZIINgGNEhJipoIBEgAI/ydIgILoPyQJCsL6OZJSklo/HykEqToGkAiSgDQlAQTpHZcyyEyRDfSPpgQBwfAWSdP/RYTop1AKJOlH0KzVB4JUKuGlSRIMAUgEIxiaOfv5LtHa1qPHBYrzNlrP7awtuAWD0QIIIqhkY5ARHYwEWwsAYqheXPWmgIAUU6lEpjIlKSWAKUhgvc6xuOHtgxJkeH9BSKjVE+pM+V8TVH2ApBzRAhIigIxovYXvV28E2HsLgmBEtGgRzbcRAiOUAgLwsvl5mJkk0tsaVHqxlSkwarcB37j03gnyTYIYQVDweQAksu4PlEEKIShI32VJ9PZIUCqoRO0woEwQAighqBSJTDECmQKCyOMg1E0QITGU6fcE5DtbH0pKIpl+Nv+5kmT6gNezUfABJ+Glt82QH9vP4cUr0wRl3RgRdQnrW2uNCSDTnwPvKFF7jWMXlIhQ+vEQQeW0GaxPoleHdVbWLx/fXGesXs9Wy9bK/5rBSCXBXJ9GYN1obyVTKYoAGRDFU3/IHYIamRMCArQ9kiAFkVJO71lSIqEyRYlMBINKQTMZQCah6OHvDO9HzmBASQAjk7NFyPZgglQoI5gpNHpzKaUAW3vIJ1szBbVOyRu91mcmI2w/ajtT0bg2yc8gG9ZjLQMA10Kv3fMKADMicqqOGSSKDEQklciOIHW9zPHm7un0xWP/9dbvO/bAh7Oemt6FPiKfQvvIa4+QJiWCUqrMhqKMS0JAJlpAgJIkZiIICUmSQSKlsjPJ9eARtq1IKBiaAyGut1QqCFAMIMU4PAsQrC/y8UiRAJN1WqhUmSGJASUSApDMIKRkPYK/aPp/gmwtlGISzd7CJr7unRKS/JCsY+rrIVt9fzhsRJQEpJCQ5eOTEXXPERNJIInGYMNUtq3NnI0MKMdEi4iZADIZDfVESQApNRuhZASQvlxQAq1wRDl+XzCAgpfisK7S4URJZCYYUbbFq2OzK38hUgB8a3Jm9DbnhB1KZkSzS1UKVPiXlZKv6wRabS6iTDMBCEqxFXwJKtPbqkwESSKQmYzImRIZxnJJIgJz1uMdJySViIjaL5u3sPFU+lDGi3XyP6TEum51zgzUQCCVKvsJ0AcZmct217rU2ZzKqM/0XxBlcm2Lw/e33h1cHmIZX586SlIIbMypQjUo9yAW/POTAYiAIJ99b1HtMkQys2xuFGqy7QAIFgTQcgcUUqmwiYNBknzrCeRMRmQej2rEWYZay4cAIpCi/qsvPW6wyj0vu1X3McWIiHp4Lc+UVPigEpm1CiRCt66cjBbBsv7RmmScw2brU1+Yc0bvhdCIuv1AsCGMxjMlthDIiFpy36wCEcb+AURE2W5MoEewCaKYYzJIMsJL47tmj+a3ZZALKEsVEYTAFpSUM6O1FOjvSD8DonVBwZiZOZOtBYnmCyMkGNRyKuUUSCEzvd+hQBDa5wimeDs9/GW++gVf5/nXAjf085uv7vbLw/nD/fW78/5+PP/UA6kZgq2eGMgsYxP0DQeJqNMXtpgtxCifASBtHcMuWYTmhP+EIhhSZlKF9Amm/XZrYUhrAA0us87MjCCCwfAyLqMPMBgLqBJEoO47kcqFpFr0VKZRVA8lIGgmBLaQ5G/LKaUQ4NpzCJkZLRYWA1KZCsKOfBmaiGAEhdojuxJv+JjJ1tC6gNtMMoQ5obZt13HLto1Mf/jWt+lVIlkRoa+z/8zmS3D8SRLItet+cdYpwgGqRBKMYD1X3fTgMkaQHL0JUEJIf3SLPjFAoiU0Ce9zoBVGmylRLSJIGOvb8zEgtN6Jun+2rS1CWSBVqQgklr3zT+V0AFHmIlA/b0cLRVthVhT0FBBlQAoaGdD5HpFRrj0zpQjfM0oKhyCOezI1xSDBYKyAoRBNzEQwJSRAtAiSk9ScghDBAu7wEYMPxwrA1laxrGFWABUMAq3FnMPxb0gR5Z5QsW5kZkqtlS8EoJzKZAsHKi2QSkwhSEYwvR5pSxKEmCuc42EmCWUil2EsmCst/8GIYELK9NWE9HIAEWjgzPRKwngIhRDCL4xCuirMFCClzEyGb5jtfIXyfqEylJIDZgBkS02CHVQqOegTMqWgEGCKCNXniCyb7bPu+IONh6sUMgifYxNFRocH5E4IUMSBplQHOSRpmjhCIifZV/BqJyEwQUQw5wAiM6M1n+MIKlnuMUCYSbJP1QE6CinYnhFszUih3oVQ+F+Y+YJY0/EDRFNXpHL23oWYiO9v8X9ez7+oPdzFFltOvXp4c38+3efpzel8f/n+VeZDfozrR1CNyjm9DiIoHqiHYaxXl502RsK61obn66CgoLJ3vXA7ERRsylf4ZyuREAIF0CEpCIp5gNy66C82sUKHRQg42or6nyxGJtObQscWfr7M6YdkhNH5QUgIpjWIfHk2/2QqSd/Sl/O1YhuuaNZehDMJCkEg7FeCBCIxSGjOGQS2XYj7++SptfMUX+y4YVZ5AxpdOugniFTYk73QYTwwlx8kSATTz5QKe2cbRBSPaFdecbakWKjOvjcipUxlTiCjb5BhAKWJ8DuZNfXxzcWK0dYGQLSGLJ6BEUTo4PNIkjbyLHSaZTATElqLzEQQRGaSjWSLljiIHymB5aBZL8aDxgHqvQ0vUgjgOHU6Iqd1w/w55XoXG3ZA7PrNQirlMsrJKgmwMZNIGOeH961+sa4wva+oU1YLYWMXxDQJxaJ2jbjs5qhgzJk4KAP62V8guUNKf/vhdQoFA8WqcZ2Rw05+QtiYMnvZoeMH/McBpG9rkmBFScuKS/DtXqzXel/H4lG75u/yD6NAQh0CUcoCYHVDEROAOju0K5GUwd6yM0Ez6TiQfK1+scB2aMbiBQciD8wOSXOKjIic+QKsWS7UxHdxHPUbCZgPncdFclC5Dg2iBUNKKcex1XR4I/8z/e40IyZJieZNd+C7aBaIiDxwbgskSAWVaZaL5k9ypDF1bC0xQ5qN1xn/9NT+03WO2+Uh3j18/s3tum9Dnz9sZ50+5OsvTls+vNL48XX83C4/S8+2hFKyPHsxUH443761f+AKsx3IC+ILvTAZLJsq0usVFTCHOTSqOKVomdNxk32JaSgGhOQMdND5FFJQi8j0qszlrv1byLGjfOP6nBWepHxwM80GjDC3YLPORmXSzytHWwufzgkqTdklisR0tJOmmrgCPUU0X9fMNOuVmWngRuh2a1tP6JKn+frr7HepOKltyJy3vs0AqB20gXW+SajYfxE9meVHfdd8qHIdQl/ZRmb9ezpWjkZWAsuRkKMGyQFZBZZzZiVDbGLCiaGJimcTkAFaSq01FP6DIHbbWK3rmL21Yg4hhtFKEU1szRR24fjAlGL9mJRGYMF2gOeRN0TYMQVWHGY4UqeuiBqSBtRsgUzaWSsd1ZGhmVEJJJsSFGiFpIxoNpu9MWeKRsrLqrCYCoLSVCqWmwGZZRUXcFMZlvKUKvs2TbUbyvSWObgcp7GAVyaRwajMEjIilrerI6Go19f6jx1LHQzDiBUkA5hzOFyCqHS6VKgXlBK1uZl1sMD690I8nzgMVBjtYMXhgyqrkQG2iEQ9RqYiGk3XrwztwlOFs3m45yxaXkiwdUyAajQ0OxjGaVvtp15hhZ/eRAGdFSxfJ5/nJi1+iSSb1zyiUYqINP+kLF8+k+s0MBBBTSL64gWKkrT5wMJiEqEMNFvOomgL6vieSc6HYgFbFc0ABKRliZwyQuVsVCCltiF1nPzGJjFATQU4U0B/ezv/87vT22zRePnl+fv3f3l488V5tsfnD5+9enhWz7sHnN/M/iX4w0Oe+vP30iU4W2F+A5pk0SVAiFm0LmREaaKhiY7TphkkrP306VrHr2ARlf51LMY12HzgKg9rAs7uvJErE+efyJdz0+TAaVYCINiMUDQhKBYjBxHFGjcQzqVC7M3flkGqObBoWDzs2jRC6Wx/hWsqM2fW2V6YQmph6iRFO6poaWYnwRtON97dTt+8P//6kQ9D57tT6/v1M4z7023mx60D2oGM2EIUmUgqDIEN3L3ojbHgsP+8YIrKnq1Tb7eaCDSbN9Zn1Dum0kd0GcQCj8FY/2/OjRozgsE+cnhDQ+H8sMkfZFENWCQhyhpBqnCl3HA0JOSsO5MMyjg94cTTwki+Tm1rSs1ha6iIYGUAhPXJKzBTOQYtZHa4Z7MWZniDgBrbTJ+WjNbBiQTZlRWkYHFmjnJqfSQU0Vdv77hHBJhIRYtKcpSgwJF+edpc1rOgqtGcHS7L9DOaWXI73WiRcx5nPjVBsPAwRRUMqdNa0Wg5exXOLrBQ/pHOaRMhzRWyObDSIudCCQZatMxZIaMvbYTKrhYQbNGMwCPMJUeFJinT2iZOc8WXBIJRFMhaH7AiZrtOq3Kg7JaUaAloojFzHlxeW+oNtoCQOXvraWPUENHmVGvM+bIADq8czkh2TT4mGfVppNeUzWu67KGOwOiFmMaLY2drK3BwsqMZkGkdS6uMDB4caINsXKqM+hj/VzjmR4lSiFK21J2CTMuHptbGg5jKGbGN1n98bm8vfXLotg88J/v756fz3TkTT6+eP7s/9ej9dI/Tacfp9Wyvb9hu39+3pzQH6RjLj21Dt2Jav01EEQcvlFuFho4xRYRixb0rwirIdSBa6xpa809EC2qyMnMVgiqHRFGMTgGYjkeL2CiGx5wAc6pVdt2RtdbK1BmOxbb6VpgcyuWOjP2XbzImsOVUqJm4PABXGQhnGaSIusJsJEIJNqYmoonzlv3j6YvHV9++O3/zvH31PLchjPfXbY5vH/jb7fyqx2182LoaGEpEIuUgK6L7yCwn5Pz0CqkKjyxGs3JP2aKlw5QKgI41LzkZgYCWraWiNCo+mYu/ikox9eC6ljNHa80bYUI9DmoCiEbY6xVFjIgIlnrncD+1jnU6qFSz9K2OBOqxIkhMpaOivihjFL1RRgfFBB68dj2Qyg9WksDfmw60m2Gcz09B+yRorC3NMRms4yFVLqyFioiy3dcKIwr7LQBRWemVpWdad7jigIQXpLKAyVSiRUM9zHFj6to4xJ/TKw8xX/h0AitPrUr4M4qlScLCKqUMvEkggmtnpRRbyBQTmZmt9cyEj1BOAgm18qSLOSKzYkUs7wPAqXNKE+xslGaWSS3dTKCSryxr0dJOiPTfppLO9jIC2WMLZSFGZFHThCwHSpWKwbwFI1Kzwon0tzFzXW8lqGC3wi+wtrY+wVoasYeTn9EKI9gg5ZwFDTIzM+p4tdqkVDRaFsGSkKDsJ4vJIaGcIMHm1BtNB+ZSpKRz0sWTLENUqI5a5Jvq2jteyJzZ1FoL5aRI3tR/vvWPV+kMRlMCvN2er/P6JMUYl8BnPwTiFeL1q3nmHJN3+2vNsY+GSzKLEoGFLqJkyZTTJ2Sr58u1JMGFvXIxXsv0S1Ai7EMKNaCsdIUS5rzruk5HlLV6driSqBm9awwRFacLzYR1xZ6KxYIsQ2CRQbJVkO/oCdGVU87CtTjADDIZ9APQFwwKOvFV2hIH/oRFTpWxZFFBCSUbGpkNc7+14J55xfnp/ouf7/72h+1Xl/jiMiPZH59ujx8eY79dnsbrh6/uNpxikInImdmNcKEkpIlwkGRPSVME6zp6kQkpV86XK0cjihlHdOpjFIXrrXkIwuhb5VOLCeEYGW3FCplB5JzRrBx13ppWedbxFJVIZY8gg5Cz5EZ/kFFeMiKLKdI6JWCReevhZX4qWzRTr2QgZT+4flAH6AWDqEDthQVZ4MAMD4u7TP/ui1kwuYUErROU6Kyb3zpAzsziQAq3LuZsgZVFb/nMJiOKSrDDMsg+znTRNYIl1IVQi/uaOckwKQA5SS6rD3lALWdHAjRZfXjEMnoNB14z/7XspJWLh/kuE6U0jWEiLTkZRJZeiFj8WF1wv7CyZH4AYWELihRQCqlpIkGL30CFQYaNTrbHEr9WytHW3+fZIuAuldictRoiMqPo2iKBaVfNRswULCARMM3+OJmeXPCNBUEUs7i6ChBYvlOlxbIuShXr0xoVCtPbX0yHPQoredOYyaaVrFIKTMK6ACmjkjDlWipDmtOXwhHSwSOXDY3ifCxxOXJIRuqzxIlzkEkhgpepH55xA/t2F7Gxb5xT83nMse+7sP/l8d3zV1/nHNjiq3Mf26vRv9nvUqHz5a8bn0FgDHarUGzcFIwM2VmioltfBQSYoBm5MrOFCJBy3HOkokhGHWirX+0jYH11OqEddfNJu0kYmU4VEi6fnbGuYdgb+hokVry/oteyLyTQEEwAlt/5XPeIpZUuRrSieECrKKG0cVHeOhiaJhac6YKVS0ROkAZWY452enf35YeH3//86nfvr/cfnnXDvN4u737+mOM6ro/jPr94e/vyzWf3WzL32Oa87dL0K8enXK5BbaYPV4uW095SFEp6THIuoSrQSDXraEPAtKKsJL9SMHMCS04AES01AUWwtfDtNR2VGTgKESrJoMPSkGgtvMJkLNWUTdDK/pBp9UiJdE22ikSw2bZGazY6TvskDERgryOhtcZDZxIkmfPFFRa1BJeEYFFnQBxyeYWT5Fa7A5rJxpxopQ0DqDnzYCioUqvafvfWSzdj8QWYc5QnMgoqHctKP67LnkovVgTnNHOrFo0laDU4SX9PwDIolmWqhCYCnDmLHnM+KL2AVpeaX5goebizLGTdv/IKK4wSiGZaSY6FWLqc8lwVIsdhlsvihaCoyMSmyPmAGS1qweF0TsHrsDjBcqxZG5PeOWv9lpTOiitVfhtd9sdBBipbu3ZipRIU/rfMLClOMkIJMY1zSv/mXztKtQBaBEkrK0qblDmV00Q8mNGi8p8mX0pz4rinuGUtqZGlikEmpgr1FIPPVpChQvhF/8neZR3oAE2H2RfhRUt7uOFKUc45vQe5Tm3OHexkfxp8f8up3hk5k53jOsbIUMYYuF2vl/357unHmRrsv/nVQ2y4fxOnFk2facT4vsWVRGdk1TiJ0YhofHkSFiGounKfIApBJuVQ2l4ZoJXmfoG4ih8hIFWqGgLEXDKDzGg9wWSG1ehWPTfJiO+AjZoRLcdBHFnVZUd9VBnVqcoyeAVvXwIOO6igIb4RIQjNOtJHQMYIpHxU0ew81HtL5RyzNyfB+mOcPjz86v2rP3xs3zzr9WXXbep5vz2+fffx7VPbsF/n25w/vL/d+Aotdz3meDRtzwxbHlMZZDPKjBZR4kkxAghp4oU2Y3k1rZsIn+U6Jgc7ksq6fRAClfkjqszKtRdZuvai1O3BA1Ao1Mz0mA8sgawO1cdMW282thXyMVaBycxpZ5tQIMpYkViJ1kKeSXIAkBVCjv69i6zYchFcMactY2WqUeFMcsk6BLmgAYuMKk6+ijpNGzJnZSbhgKC0sAXDtZj8sipIIYOx0pDFi7wkqyqY+8R1o7GFZM2rINrtzTmjojCZAjEScehp/mbkRAnyCmIdzGQhslWTY1bE2JZgiKp8QOHtUjBTmiLYWitOt8gl+7C6E3U7KjKGObny/DyKCYsDbr1ZyhFOgJUqtIKBelgU8md5laW6tCKoudYV3fUmGmZvyJUIdnpUhS8yWulSj4TJoerOgqyfmCchqZLuESBaawKQCckKLy+VA0CXO2gREt7lzIwIETP3T9g/VmjCSp3WekvIyshU9OcNUAV1WmUXQgoMMVE/KSCoaIGpgwOTRXhYrzARRKCltpHtaY+PQ9k62na7jtO58dTxzAjFdhoDkby8f3rOD7ePswu//fWXp9Pd44jMb4kc758elA99zF2NTIV8uxlVj5CZqaMGIysUE/y8S/3MtejFMmdtnRKIA6qVwMc8YE5AqIKGLC/QGqEXdVs4P1J3ikZoc2TvBFyU5wCZXMbOB8xkjVlKRkQc2nDkqpr105WhNF9nlXoAFTa7aFMSoq2HalGhPdR6A4iup53vP/vd+y/+9mN8+6xXz1ddb/v18fbxw09PHz/u1yGEpH3ww/P+PKn+WvNVYg/c4IMIrWsSlfgxnmC4DN4WxQY3wDRNsXgHFSYsS1TxpZopj4O6984xEAjUNyFVF6Ui3WhenVgcCsEpKUdv3QZpqha4imYqaFEW8EE5YkdRMsAKZpqg9netCm1GUJkMZip6tyFvsRKwL5kvAIpoc84ChXJZuEraXdSIAUaRYzap5MHbSKW/KAXcSrQU5CGRgnIynJKlIwRD0PI2i0rEce6F1lo5KjPQQbrYqhHrxLiYHmWyxPL9x65BquIsuv7OHE5BHculDk2Y98jI8dBeGPcWUAKgmRVhv2jNgeOhi6GiMlv0MUYhJB1g9PA/eBG5ZYbF6zPRFp9CHmm2ZeKysglERDTfZ39eRTMyNeByiu4jGgYmPtesq0vw8MZAJCZTjZxpcLr8e9WwFCGshTpXZtvZAkbFm0QQGQU2TH8fRKGFHozMmVIiGQ1KZYItGMIo6ta7Fa22hSuBy0hlpoqYXhtQvM4sIrbSkgFlUkJjkWZoqHUsSGvQ1PoJ2qne+rZr+3jTjkA/Sz1nXp+fbjknEeDT4+38+ev7z865zxRu1+cffvj57m5rrx9e399D+GXedPe7flUfP2+NmZl7olGAosA+DcSc4U0EWCDOmJpTK/50NGtvxpccWG0fCZdxFbBmRFR5p7c/AU2z7Jjpe1sxdRyBFCw6DK0cQLmcYq8qej/8f5UFpVo0hwl1nAkQOYVSB5WPWRGkhQ0r0mFllczdoqEFQ8ypaJHCx1v75e5XP7/5Vx+3Xz/FZ8/76fH2/u2H2y8//JDz4/Vy6dvWo4lx7np++vj8fMGXX2e8wnikIhOtBcFECejp0BNssawDWchUzpRWeYRJoPVKLxTBmNlszxxuQ9Es4JVSjU6QN6e2jyMGKacARXTzChRba2ASVM6MDGHMedo2b+KCqDx8KhbrUK04Zp1eIwL/oWlcVK6VWTGhBS2JqmurDC0OyTl9EtLGvcyo2aWi8w73t6wcX6LOKBtysDW2fovpBlXsSjrSkeOqFEtm5rQEj4CrBGmuRMESKRCUG9jYNJEgGnNO02Ws1cqVJVfxYM41M1IOoHM5PcMsgiz4AhSmLERVGCiOlz9qzWibKz9kQXh8krY0tJaZKwu3uCon6hItYkpFg2A9rCTKCQSU4/GVSZdr2nRXCfehuCuPUMFuGV4AUK/vM2g57v8yI6Q7srDSX8AsxgtarVm0Lr9lOSi6xxlJZ7rRgJk5x4jeSJXQu6xw0MsXZvGRhyQ+Ea2pMu8QUspcFNPCvbVlstCTrVxzhSyBI/1VUXLp8xaOwxH/VYDWivFb+icqrTqIOQdbu6G9u+GK0+QWjAFtiNa0A5NkP9/dn079rNOec789X3/66bt+YuO32/0rbuePtzf9/Lum/YzB2/vWsnWXNpijywMM+JmzOiKUm6vTWKFwEIUrF3UmHK9bKGDdOjBzMrwkkDDc0SFLoVvYDVCR8oI1A0Xxu+aIVYwpYJUKrY106UTAlPjCQ+W0cgXHrUmK1nJmuW37blMun3gVxwiwxYuy1GTLOXecPmyfv3/9h1/6ry66+/jx8jgvv7x9++Hjx+vjR0ae7t5QPUJ5uyja5XJ997Tv7TO2zzl/gbpy+jNXCsD8YbDkwsx1mXV4IS3f6cY1BheJ1cpHrYW10ZQpXqyqabWICObk4oi0AGGl6Wxnp/w5TciZSapbdOUgCaIrt1HomFVk6+iRqOhj2dSserVYlW4Obp0rO/K5xRdYny4U4i7vbxgbWnSHE0OL0Ch17EzBRadu6iXMVewy64vKeTjNAKlF2VaschKybuhyEC6lXn/ng7qQXsUfy5pysRs+pbYoNofDzTYWE3q4Kp+vYCgzrQCEBM2Rh2yBLiFmbf6ngigtgmgtPsxD2C45g2LgZeO+kgVmbt2jrFxpleBVTLDkmOZop6MkmmRba1NYzjjO8KtFLKlpBawr61vRkSWXjZEFXtKf0FHvUuwr6b5XLDsTPCwjl2svjqvsL5Y/Z8VF0eoTFzlzIO4KLkuetfyd7wGjBOWCpnMd9mCqGwkWkxMq7CPrFpYCQxVh+ZnLTLE4oPWMlmeoVEAV7tlpBVbqLpEreBdBVQYlGWS0654frjF1SmxTbBty7goR83Tett409uvtyqRaRmBg/vLulxY4Pbz68ny6v3tzPemR+1n7q8yWj1VOmgu0o7aDTgZWWr9Ath1+nRIZNB3R3woR5KZliVkpEQkrDig9D6YiKJpmSdvxAr9SfUX1GCgEtqR+XhNEhXsHXgBbXWKWIqvgf/EVlcROYnUjWCEAy+dE/QgjnB9NMVpSLuBsPaB2u/Lp/Pnbhz/80n73YZ539XdPjx+vl0vebtdntdhOD2Dst8HrddxuGsg+3j8+X4U3d6/nfrI1VtVDGBb5dVYwwAMruFmTj5nqlJfG55DAA1CL5o0wDEoXfdVltT7HZtWsm5X5sfacUyvxKPfcY2ewCitQMu1U27oTs772jFanNErkZzKrBDiup60gxiyfYQRztbVgCcDcU4B15M0DMBxqg9PbX6CilSKNS1VegTJKyW5JDNOLaRDVcukoy1YCKJKT1eUvhRBKDq8CFgu2eZ0TpbKp6BHF+vrGJMorjDmC6+aUjzuoXRx6Or+u7QerHp8RWKUDMO9kBUDhmhZGaUq0VkVkvjX1FMexrj48WH/juFC9x9JHVSapFDGlAKylDTeFCTmYjhZ2EisqqfRTHdR8Mc2ZJvND0FS24qlqh2o1AB/IOmZwWbiJh2qFJRsX70IRorEiecO6KkTCCkX5EpbWfypH1LcNgNz+olBMw8KxNjhL61fLmAADgeY+Ze7m4QUza7tIWlRHwwNSSVFig2UKl6TQ5Q6Vq6u9X5SbOzuWJ1nsny1hmFafEZxDTdhTl8xEzDlODVSMMTQmqH63cWjst962tp0G9n3epPz49u3lw/vYHvDbb/tnp8u448Ovu/aG5OXW9ltQrW+s/owkoWk+GEAlTXzjUeSMSYM4WACr1ow4Teg5VDWpsl647lK1QGEB7qUctnOXY3BVRTS1TrVEzRlOdy04XHjFpEc7QBuzRC8rQFuXALl+RdUth4rlb9P9n5bgQnWrWUzUDGlvT+c3P7/+w88P/+JnvRl5uqE/PeW7Dx8H58fHp+38Gc9vxL2x7+MqRYpzzMfn6/Pl9tmbe27nmKfM6eaaLwF5yPU/lcErit0oKDKXpUZh+BV1obIGULTIkViQa+1NhZEQrB6p7gsVo3mFhVQyQRNcGQvUwumuVMh68+m+J9F6j4MxgCMRC5tMqsAFc7NsaL1K2mrW3eTKM2oBKKcnzCWi8KNhOapLQAmXUXaxAH6x4RbWLQMrEBEtMxMz2BBV84LKnBxX7jAsDFIRmbZqRXMv14EDqr6YsQWcVrhTXT8ramCk7dwyNKtHG+xPWBVhMl2MKW+Qa68sfin9UIVNolVo8RKpopKmK+WwmttGhVc2sRXSz5ksb+yU0vJxtazUwgggSLqkb+aRslz9i3KdPdu4RWOWcShjYUUdTCQ0xpyzzhsYrWmObqV5Qej1IJBzoBUtYf3xEUghmZVXLMaQRy7MxtyVKkFUwa0y5ypXidKtgVwbVudgeVQIUoObzOGIU4tQq5tZRMThg1Qf6GCuXoTgwkwoyZcJR+v2Fvfmv6hWS1hJDCw5XTjkb5xol4HHgWQotjFy368pWITTEXHS9bLv+07u27adXr0aIHRVx3fff/fmzav7V59B0U8PH25f8voe/d2ZH095SywJQxVrsF7ID5IVupdFScKntiC2+VEtQbdlUU1iapjnjJVEUwGxyBSo1lvp7usvuIpNliw6xaicjRsJsNo+l+TZJsjqcnJRp6Dd1xGg1Po617PCPyf99PLjWWKlrDB2ZbolYQ5e4/Tzw9/85e5v3m1fXeb9vOnd+4/ffff90+3p/vPPuD2wn1PSnDNvc6YYgvahx+t4vo744iHZR6qV6cMnQf7KrfuA52SYfvQTldWQSxa1IqUKQJlKTh05zPjkJpON4OLu0iY4q95JjT051VhVclzaLbcXFYUZ9SRNwMzZHSALOTO4muzReyUIbADC7VJUoRgPpLgAWvESh29e//vIbmoBA4pm37NKTAuk25lFWeECUaVPM/wyBZEpRAks6nwty18ruexdRZCVMOZxILRSwb70EQw0ezTfXVv2ygdUixgJsDinUOnC12V0DwIAZQkqWmmR0wUKWLqi6hRW3IZFX9mEbFxlfcY1qOj5MIazWNUyUH7X1tpqC7xoz+Xh/DFZ0XOZLBz1Unxx+yZ1g8sL1hU1Ai7/BBd0Acnqb/hJniMB9GLicIRdQH3iwvSVDqLzLUcjWNZPRrpvfsXCqzRjxTiCoCnAMwaow9IAJac1v9YzXTiEYJ91dtO6aRVEMeuJSgeU8V+RTW1A+aNYFYZxLKIczdfb+lXTkJ8gu8mAzNCyvVxAxPQ80RXtacfzxGQTNcYeEdRgSprXpw8BRCDaNvY5Z3BwAmNkMJ8f333/y19efb49fP7m8fGS/bN+96s2n6DZOXQbfWtzCDmE0iDbyvCwVguGLQW/zDMXSEMJfltwzoSVHoOCY+3mJIX7r5AQDEgdg6aSbGtD/QAEW7OIDRDFtIfUyrPV/ZOwZCWS0nXjWv7Xkiozd8los9yYwcHqDECBwoRVjav/GQMcc2+hzvZB2+Pd1z+8+t3H8ze3eH1LfHx+/9fvv7termB7fn/l9qqfHjTHzAkmybZFKOeOx6fr9TqlLdpJsYEjc7gqZDl7wzenK1xMr2DsyKprKC3JEe67YYLlK0vVbvueGa0HMOcIi+Fh5pOCcrEbU1LmbC7wcFiWmhlbX6REAa8oui0BRnUlcbe28uwujfaPs6SBxfI7AnTygA7b3DRUSmVEFGgqfyWp5NMB51T9A94psRVYZEGEMqIOGPhiiQ6DXqA9p+sfXMhdZ9JRrNzTsGrB7C2M8XL5GrQICcxV4V9lGUtGjJdSrFj5RWVOZcEIOI/yolUwi1AunHUD6k1YNsaRTC7SqeCQQw2nxhIzZ6w8dZQ5isOcFobS6rM7tUTAVV8Sy/zbXiYU0SAxII8z8cGy3pSLjPnEafuYVNNfX2NL3OFaTFUJsrGkYyBHnFOEurecOIg8VXf5WhcHjTBR4GPomjmtnNCB0KVUmY0qqVoQy3mpVdPo6gd3JUmQssa5RUyAGlODgbFna51WRlpTAUBkWrBUzrj2ZilqA5GrEWWEU+xcyMbem5rTic2DPsyUDxheOMr6KzGU0yfB632d2hVg33p3k61QzHFNjZy31lxphhDmmI0jx9i2OPd2m5cf/vLn+/Ppzf3DZw+fX54/budfb3k9Pd/uxu28hebYWptQCa3kdgXWrS2MAAWQIbmu70BKy6tCITCiGUeU7MutOkk4c6tlwA8SouhPrGkzqpLySjdUkGgMTKEqVJtlAi+tUXz+VnBtBStYhReBaFaeVEoYTSr1Fw9sBaYrvcBoVKpHYOLG89P5Vz/0v31395sLX10Rbz++//4vf3768PZ8/xB35zF5G9xvTzmumoPIJnKKZLR2uVw/Pj7O/LrHaaLnZOuO0aORuZi+sggo1wuoMUY1r6oC8pLFzApYDgNoRXZBZtvTWLiowil76kpfthY3jcYX00Mu/UGx8zOl3vrUpICGDWsCTISgZvrJkXtZcrgxOCuBQUGtdbiVMReoO0BpriSBHzsg1NmqViMF4laEUSnrBcfgDuHT0EzQoaXxL1bmjHRiZVleP1vpfFjgrfl7GZE5wWwtKjgkC8lFtUfwY+R0ajq5jBQIZ/AATLuxqNSoXZ1/xrTOsQzGnVqamXRDJHKuyQpHh76cLgtFq0QIMzNarUiV5ma9qSVSJcpOtBZVywlNHTzNSydRmNgulsKB1oxoR6PiORPIaM22u14ka3DWClCpKTTVmteqLJh8xCFh9Qf7weu45g3uoVHh2fGhx1I5UNQC3aYExKh2KMevFfdEWUBTKK/Eok5HZZ07d6hSAtEa56TmjN5c95gaFmxAR1v6dSjtlgKCtUFBZvqz8khRRHlsHUGJXo6CAGENRgFypqRlCooYAOAKCbbMNhDPOW4K9e7CD80cc2+MHrxentVai86N7Kfz6TzGk1Kn3m+3J2WOy+2HP/7pm6++Pn37+d328O525fZN0/X8dGO+7RphkTUFiQp4yllxc3J+Okuzu+rIy8jIGETTXZwsoa/kEojlCIllyiGFxxOYgClbXf1Vypc7YMqjigJIsXXUljFnWoDgQ1G1rUWcLophnT46GefWx6WVdIYNRAgTBJVYAXcGe9tyzF28bJ/98vDtzw+/v7UvE308fXz/4w+/fP/Ddmqtb4geyo49x8QcwIzcMXeoTQqI6/X24fHpeh1b9Igt2aGZqdaAFlSs0LAuthZfUAev1LCrXd+STaveF6vyiPCoA9m6OnC0iua41RLd4lgtWlEcWZ7CLbpaEDkIRoviRCAwJuBCCwHd6Xgilc481zd+whIsXmiCDLYsxD0zFeSsYoXiU9kIhgXH6TRcNFW5SR6UguAi+vrdOqLlPAsjLLiMYLPCkt21cPFieJVZEj0Y5CwEkAcnHUVUYjEAIriodncsy4pVsbK7Cw3Zdrvz/hK+rb8IYJYVI+xwl3F0b6WIOUqwBP8Zi94JUQWiQMLZAp/nXOVaggJuZGRw2TI1hw82JEVvvoyqEnGuY4SoziglxrL5IijNl+QKipcuEdiR7q4jVD8X5XUrpeMT7mai7hlEqBeA/kQfugKlqryoardPjUIZf5NCS6JXkMEtJOusRwGrg+wrr6fyc+n0GihTrkzE0YLYzejnhLL1XhrHLPiAaixXY48AlNw5V9tr1cMES9+xJGgoGqswkZfPSpninRmBJRHUEWWAiDYSl8TIEPsciBYtCMxxvTUOipgac3Z05dw5xpjjNrht0R/OvfH6+Px0+4d//Ke7V19ub15h3F9mbvj14/6E/flN31MDC3cA8roa5tQBXubA6ZcSYtd4zpX7YJmrldlfB4delqK8fWIYRE6soRcQG7uIeYTfC7ZyZauKGTxYxXRA6Ju3RPRIcw4H6+xgA6zcMHI15SW0upTIUyk0EVyCx0xuVzy8PX/9y/23H+++urW756fxw19/+fkvf8YY7bwhFUMzr/vlNvfRG6hbaMxxmQq2TXs27B+eni/77fW9j/bR1waHnrlCKS2PxeXVXE21Qm+VFTl0sVgGsHoyGzSIwhCPnwUdEVXQL6fBm3IUi1Mr5NWt0mogUtPzK6W0KroRqgjeFdFJraIkliLlkDJAmFVtQ0E5jsev0LmErIw5MxoBTYe81VpBjOZeZgAQNWSq0gAVZ6i4pjp9xQvV99inW4u0BAgVBakGQmROczie8+GSaNfM8WXBk2IEqy7DRAyKrK6iySziWkA0i/k8GrZ8r03Q2lxX0iJNMxTvV7UrLmB01lWaY85obCxVepaepM5HBUkVB6y+EVOxupcXBk1764iqmlx+Cuv/HTMBpBobgDFnD3erDffIWVS+YxO7ZIHVedTlJp9kd1ycuAqMogZyQJ7hYCFXVOdm2apU+rr4uNUIwZ4mME0yd4eeolvfeT+s1a0KaNSpO9CAa0kqQeZFdKe6cl6ZSmXOKlQqYsoEaEXIOArKIVThInPoxTbiBcCBikbHB5mz1LiqqtRljkSH0ihPnMJUVnNVFbmklc8ayttuPqPlzL519m5vhbAhppN+Aei650jNCbH3u12YUko//vmvP/74QwbOn725xunD6fP38avr9s0F23A3Dq2RaWbGfV6hJauJFDUNEBy7IbBq64h017RiJ1ySS8LqIMgXPoRg8f8VjDaySZyZc8612MV7KOHBcSRTE5XfyYr6JgC0CoaPxkyLUfGWeJvT9AXpLpZgY4kxwWCLaMHYlIgeIkfOgfbh/qt3D7/9ePr61u+vmW/f//Tdn/7x+vy0nZoyb5fr/vx4fbrcnp61D+1X7DdpR5digkl0ZTw+Pj0/PwMd7JkQQozSjqyDg7KaBCymTTsniOkOfpV0PVKpLBrdZsekSA3CKfbMP7bWwtJJRp2rFDS12kSUDcmsvkjFBdnDxLpmFu0tsr5s2fqPQ0FWx28uG5GmxIVFhqgmRfvlI9x4TpTQovXWa3gcMOduW50HcVFFPy9C2MwXSlYl/8MLLbl8EW21C6MUc2NyuO7mTJhgtHEAo/nYla2cKR11Bua/4lP4v3S1Rh60FaqzvMI6/+GxgYVpVlbDU8UOFrw+ZF09VcRQwqhYKN6IhY1BRK740Vr0CDcVJsgWjc6fLptf4BIFPgQlF/sPbn3Liaw8HOM4Twe/bvtWLdpxmIsCrgeupdXVddQl5JRUDRic9Pz0CjgMmb68MotsN2oqbjkelFmqEu9YSRQ3qYCjoRWT1G/5hNQIEQaA9IiD0usgMceA8xcMoKG8N4GYq8KlaOfq1R6sgcIVyCU85c1E9FQpKd1epjgN1kMdPh0LQPugZpGkFSOjRcup29BIzmltDCGhacwce7Zti37q51O0mHMw8nx34rmDmuOGnJ4fkpr/9Pf/+/d//f4ZmXd3z3H6cPrmff/m2j6b20NFt4zuQXmoio8y8U732WUKMF0n91SJ8mmAkxeFiRBm/b3cIrOSZrV/4e7KQdANpBLKgKIKC8vzegyLacU5p1t02YpFSQ0QbIw4VC4rWwdQbJAmNDUtCQHAFuvYeVKemwo0RY+midTA6X18/tPdb3+6+83z+bMZp8v1+ssPf/7wy48RPN2fZo79drnddt2uW2BDNk0oM+c+U6RbaF2v4/ly3feRbIiWS9Zj81pGYyn36k5Wt4hANQ+3OW+qK8aDO8RCcwBmrv7sjlDXbWSV20fFO7KPODp6hdJic0D85FNfHP8RBEoKspXfwUvKoWLkYi4q/HUcEbFcgnmuOIIAAHgZhmpioyoKFT4qkmRncXymSjPIEkTaK+AwZ551o7K8DmbrxgVr1SFgzGlT0NoK3FsZDgPsQl7lpRFEBFu0IrwN/t0LoSotym2rqgidQTAvFSbN64pXRXPRAljhgV9z5iy/mZNBfhIFBo+KePMkiz5hfT8XRV9SQvjtGAdXswKRtQO1k2XWl0ay0srL/1UTWB/WKthWGSfNmXMZmAWdyTpvh2kXrSrG8m29Uk6pI9UgwLPdvXHeZJSsrV4RLg5w+l4rPVkvdJxe+7OjV20Fcms7VywJ9xz2N5lyREQj2HpLqaJDn5iZ4KqwcNp9OfvqGQerUBj2aqmqca5VrkeoeLvYtKoPc0hb8GqWXLH2cM0/GeJVLtpRzpx7AklF20779bL186l1idfL89C8O9+dX72K3udtbFuX5px97nvb2oef3v6Xv//PX/76y1f39/vI610+89vHj48dc+sZY3dfFBC21SQ8ta+qRRxCggv5wOBnBXpHdGm3xkO8NbPURZlojQ3Emnbn5BmbseEiBOWR4GWMyFUJ7FaM/sNVCJM5XYDtX61GCJmBWFlfc822MrU/zaWQ1HTyPQpODI2Mfom79/dfvX/43Ud+MfIhoz2+f/vLd38kcjud5jUxmTG4C7ljR7TI20DsrhseiVNrke0y5vV2e75ep85E9wEDqJxY1qTOhu2t6jzTnMCqZLbZKavh19FKlh0X+Tj2ZPEjdVwtgYzpQWYRgagxFEFUvR1ohnPxqACnZndPFEfc8qp3mxinOm3fF89RFWZuQWxKxK0ebKpykTL5UsvjqsOW8xh2q23r0zUZVQQIHd191nYWFi2nYhPxAq5tiHUoIymImYB198rmjtbr7DmmyNTqfuxxdyuUWEq1nFU3b4bj6CZpEuPo6UWgBeeogKQhFESqtVWRwLIDLjThqtgtE2mcHyTRgmOsFzCAPeQT3vAsaxJH31wANQGl0KdX0vmeEtAfHOFSVx5uzCHJrJYbzQJ6+10Rwcg5K8Z3T34P9ViEireFQLE4RdkFCw+ShFKxHLnjkYXiDZYZhFvU+ohgoUgWlo91zteGG/fl8pUrCFFhhOMnTTYd+AW19k6JRCuws4S5fsLIZN2RIkYWy1m11tKaDlzLEWtKbfoy1s5VrCeRS//ncVMiqyIUwThm6/ifgkQlNSuDob7VTjsrdzrdeU4ZW6Ax3WVnjPPdqfcmiNHbdrfdvTrFw3Z++On7v/zz3/9Dku3uPM53l/svnu9+9YzPdpyzNxFz7u4oj/L8sSpP/C/lCVB8WUkVS+dbz5iIBLLW8pAM2RfW4dOa1bja2rRCDDM9ky9egiPfGWMen3VW6JqAU3qGfsGmFRCsRtDLpC4IRWW9GdHIFsHGFh41KCQHt8fT1+9e/f796Yt9e5397t3PP3/3z398/vjh/u5u6ydEdR4dOeZQP7H1iC0y2gSnFNFabGBEi33fr/s+M0oJpenHKDvoyyQsV1VhH5zfWF2qeIjKWKdaVp7MFWqvnVkBJt3Yyl90nNpqsGGXUumQwnIJrLr0+pSlS6xjr4VqtKBNfeiC+OtJKxz0A9U9zKVrfPllYt01+6wWrbUWEZ5xuBBo+SOur9Ph0VFQ1DAaxz2vVy1tjC1hOYdVDQsyM52j89fkLDIYNq9hzU+hYefhj6KWCnW4OKWa6V2Xhv7wqJWcmmbb8sUvapndjPqOZfTLf8RBtTGqY+rwnAesSHExLUUSKctNkj5Qs+o7Xfas+tLDaVW0WeC73mUKiOmakhbRHFRVenih1+KxtfyjW+PhEJod9IyOC1i55hXqxtFh3PxYHK1KtZqi1Poo06h4Yd+ZR4sk/1TFJgI+fbOFute9VwTN1biCxLZfC61XiFGsnxZBvRrarcJmlHJwUd7+0Uy3v5dmIlM5rd8y7UYXnWOtDtf+HKEkLJM2uq7WH2YRl3YWI3WTBPRgtC5h32e01rdTRLPJHWOKLfoJABIaQFCwEgCt3d2uKfXL0/5f/rf//OH5sr1+g237OPl4/vrx7tsnfTbUM+SGYnRmtDI7VY9Qo5vtk47rv4qwSsidiwbly9t52iAREX2do1XGshhSb0NmtrYxnBOqlgmmaP2H7vrrXV8IpwSJ8MRotiztYQeD0Ymo47/yb7EQi8fMOXkzRwrccXfrn717+O3709fXeBhtu92e37/9+Ze//pGIiFNOSwkzIHIqIEuNo4kh9hwKIcfw5UvNfcz9NiELvoolqLN7nOCiOHAA3rJdUEIz5wGs6hC9+NSo3rZaWSaYxOeBi7Naz6xkJUpiVmAoYhkkZNYO1pWmVRgrzcyye3W1fFmy8pJLWXmY9eO7iv/1JlpLmsdNjWALV2w4C5EL59rALH9UrWtxVGmt4NoB4/LzywPlGsVR/na5WBogT2XOmUuvHgUxl4Mh0beNfCGw6xW5nrpUhvWMPsDLENWzHe19/JlzUSWoRjOI1lZQinpmVB7Q/QsMyV0Ht7yX0UusI2Ac/YniARWfkEzlnB6cWDkkVPhUfFrWeVyqDY80b25aojTj4hqjg3Q6vihAsrUwB6caeSZlurHgJwg/qhhQmHMK6h7GrRJRlgcDxAhkIopc9NrmFJlF6a4DIcEtJxe+zONs+zX9r+sgmbry6KKlHMnF51WYwcUfm1KDlVfuImlxFQPKg/xZw1JYR9Jd6epYJFiNGGso84q6dDgvrv51sWBLqViDM5NKISJaEjMxRDIWbEbvjcoaIceo7GvNNWn7mPv1MmcyoqPnTGGPrSOzt/vHjx//j//v/+9/+J//57vXr8Zsl71t8fstR3u6PeTeN0ByeXrtQXn9xW8WBNWaEoN2JN7FVTRoN2dxYRwyRwDC5Cdtwsxzh6dTrPAo0CzIoMNSM5kLCFKhJTauoT0sZsg/7+gMQaqRyghoCjE1A9Halm4cDvXgDtcRJXuXTld+9svp1+/vf3M9/eo6AuxPT+/e/fSXcbuc7u4dEuW+I1tw5KimdckgW7Q5kdt2DgZSuY8Wbex5fd5TriRZGsdiTVgER6ELx1Mlq88qXa2t9ZwN8w8LWfu/UtySQ5Kz4lr40HLXIkBcGBPJaO72Rbi9s3dkpeVfQL7tPwsGTdRMPYI9omryou5PLmR7tK0v7gUEFaIK88Ho0SRFVKB+KAUXXVAVDy4uW5YgAYv9S1T4SXEiyGMy7xFxL0PggstKLVeLG8CjAJEzPRelkHi69HhNJHXmI0qwN2eyOqA55PUF19q6uuFYg++xjEMlGWd5yWOJbLxeip4cE6wGbCClrGJjw51FMRn884iLCpKrRvtmttZy+WcsI8dPfq0qddbp4tHovs4MHMfoGERc3m7BDzMcy/MdyM9/X8vuYjF6ALuV5O4MrOq/WhQYl0CQNQjqKB8k6X5y0YLNp8EasUKNR9xnQcOK/ha3UFrSdcFW6FsRHHS8ElQepgioFdIY7mMtTXlzZwhW8xOW8nz1wKwOPxZO8QC7a5XyIDDNKbFA3qehjQ6GsQ61A/Y5SiNipJC5j33f99TMqdv1ZopzjpGUoDFuSEkTEZd9Z++IOJ/vxp5/+ac//fT9T/dv3pxf3Y92ehefvd9+++H8N496fVMfEoJzZtXYWnlQ3rHc7HJmpWRlJdNWWF5nyEfa4ZnDnhHFgK2k4YKohkluIz+tZLd+v6BBwV7X5zn5tcJROM6z+zxEJoUkF60nzkyR1QqtmIFowa7j1c73H+6+/uX8m4+nX33Uqxl3g3r79ud3P//YtrOxiPaZY+qaIViOm8ihOUGpNbQtTpHMMa+3K4L7fr3u16EJUgxXLSoikVpZEwiZU+n6fR1L6aPogekS2HAo8ASX53Lh0OIFAPToNqOt8LCZxcXDai5qdi34SvgcwqJUlnwOCqTJV1MTUSnNgyuyIDVce1NXfW2ZoInSnEtYbKrNkpaVZo6J6ncfPSIYU9IqFp6Z/pCFuw9YTkDgqllZcqXK6qZsSJRyRcIL312nKwy0LQ8rNowVW6Q4R0qYUzkLjq6YtrBzTXq1iqF4mHIBiapWAV/sW0QLtqy5CK7poiv8fNnrRSqGSKxK45ddBFr140QZvJJyHcFiVX4sFb61cUFQpcFQRIgC1SqpvFrOkMFoawreEVQArvBfDmCNAl0RpEL/lUVoLw4MdjRF2Tr6AUB2d0JBayhQU9EcYDwLF7Y471rG9Lgt/iZbFYmtVmOhFresWcnCOmULLdW5XaDr+Jn67KgsgkeXvETZmP7pLJi5oKanSIorE7UM2nLML2WQBVCrK5mOhzhWuVrvakmfwU+eEczKlsYce/QzI2aOOROcyDb2fTufImJKd3d37e6UTxltU+4DIcbp4e60tdue4zZ6v7uM+Z/+9//988/efPn5K9zux2zv+hdxurb5gTPvcwTVWuQY4S5TXNWQQEbYiGExDHWOXhBE/eSR1MwUI0uEuLaqfEKFzUusH83Sz/Xylvo7XpQxxAJMwTU5VXipeU3jWwPaWno3PEaEkkk5WcQmuo2sxkxK7fxh3r97/e1P56/fx5tr9tPd3c8/f//jn/+yP13vtvNE7kJgBhVNmrJRzKjY1InONSu0Zc4Errfb7XrJlDpBzqmQ1N32C3XJ62wa0HCRF5VMNycQLlWVEKgOlNanixM7S3Ni+zuxEnqH9AIAW7XlaiQREwVRecCjciFCtWYr6paabN1g5IVOH2Bbdbkg1zTtcgoel4h1LSrgkz2ZQdpBYYBMZYt2NCCLI5QvZY0EC4JjQeUKqWGJiz1MEVDVUacmprl0kQky3fgppTlba6uLeBrHWZbCVlJZO5gWYb6saLvFhDpQioNCXgUH8rypFyzn6dM1Wq2yAMEFDX2n6T4OFKJFhdsUSl9fpXC+ZlNpS1bbE8WEuClfrc8LHV+dOixUwdIiVgxUO7EMYxxFgzVfHtRwXshjxiKkCbnQFmwFp60hW0MKCuPWbq/gY1m8upXlrKsVvfFzcBlwOSQTqi9/sEjMQ8+4Kg685Twk3QdfuIJg1eGu+Oj46zhM8MGlLR4NaQ0YKSKFLK5QYKy8OkgnqhfBKEnOyFVsdwQNZSFV6NV3kax65FTCvE4dP9SeRARbWPJJApwV5ZhL59CYc4BCYGr2Uz/dnYFsrZ3u7npvp/N23qK1gGbv0ToYPD88tFPvd6dIfv/nv/z9P/5j9q099Oztdnr4+OqbD/d/eIyvB19dbxgavgh147yUvcjvSr29HPS1IQAqjOdK/hZNmdL0+cjikpzdLLF+BNCc/3YYYSFBLvreL762qcnEtINCVXRl5AiLSRtM/leSyeypv0xSQBEp7hMzyH7/jLv3p29+bl89n74cOl3HvN0u73/48eN3P20R59M5eB6ez0UOzQklOcsZhrOI04JzBsjWGyLmnLcxpsuhV8WWEaQzo7PkC+UB5qz0rWDoVCtaJCewSHIJmDlfYikS9OSDCh9mTnODxuN2upmrtKDyBLGcLzNzwlm3+pZUJnKu6DqlUfK42vuS3ZoDY33BpxyFpCr5iZcgvMbKGy1gFZ0euMcRPysxCzCitdb5kgao2KVsCZZJPV4wP+Fj6O5Jq7D0gIvFXkxBI71IBZajZi7aPYkMj6xxCclx3DU1M+uTF5pbpGVxA/WHUTa8dvm4NaijgFz1FtV+p0ZVF+zMsl3Fn/lyuCaGAU+1KDthTlD1nnCVwJGdN1aysVpEyHLbNjxamZ2ZqaqaiapIrqz0Muv1y1JFA65o08ylDK1QZcUrzhlEAOj1lTXYswx2QZFZvsH9PVprYw7YfRWd16CVQuRqIVbJA79YhNvA5WSsBs0g5TnRa65NIYUXQ6YlNbHPrOjkOAjpCuwyR5r1ViVvUhFqHrpdMDi4svQHWK75QVgTB3T4nsJ/KF4s1pgwcgKTJSytBXFWtSHHbBGNgTn3fY8Wt+tzm+3U24yYt9tp6wyN2/NAP53vzg+n68cxpi7Xyx//y3/59re//ubLz+L1SYzrBc+vfhO4nR/zbgPzw8k5qlnpDYFSkRBHTaGSruRnxvK2YaaWFoRGyjGnFCvACYY+8ZPiJ9LAioyn96TFGlhBruprpkaB1YL/ApQ1zUp1kI67OhVswYnqKuMvDTUAxJRw2uP+Q/vm+/7tu9NXT3z9fuTp7s3P3/31+z//Y46ndmq3oWxtXK7RGUxFTAA1XtL6sCAgJ6zrboSHluxzjDFX7GgIsf5vBa4uICfQg3MWqlzM1IuJt56AdL5OSEwmPDilgEUpJlBC9SQilVt0Le1B5YPDLikXuVFTiB0g1A7bSEYTkYQye49YkKl4GsRUdR1NTTd3soKwsMBhnNoKwqUeffWaJ8pirkwzayB29YpiIJhzgmwRS+tXtowL+Ns01V+VHQNU3b/BcFPCxSs241pr2FrzBK48JoK06i3azIO31nNR3ge6tteBe6JFSJW7lOc+rQqvIzQxgzNnviR+SkCB4nnD/U1rRNrqrFfBFcGFjZTVtL+8hb1KrKg8ipaoIGppOiORDjVSkmb1ZVjmfLGplNuChpCeOgAcriuzNd/uqvpefQx5sCAAW4s5E8pozcDCUYXjnqh5AJWywWIJsiJWwdyA3XT12/PYDst0Kp0CtoNOWCuaSxxGWyY70dUftmhPKJN8wSALQqWzyqgJCfaP7icKgMfo9CPIUCaSCY+asnYtjI+KearYWusQGBS51cbBMNU/tG57kDWlywSbq2yEBoq98lU9pDlu2ft5O51IzDFCGM/X55zbtt2s6VJoTBPJYx/ZBSRb29rdFucPb5/+/j/948N//HcPr15fb+9vT4y458NvWyqf8iFS43LuiqDbyyZX2+c6E6hNh9wk0sFkZYqXJrdi0FJd+7j1gwgzaVFMmqU+Uu3dyqCU9ZdQzZurM7rXb1EBCjcZBSNYHQmXh6hxKikGWuvp/joRG3sqJu+u8fnbh1//fP7m2t88TaJv1+vl/fffPf/0l87G6NdL4tx7b1skmTMFYiIiWjTmakhJkOJkCsFu5iDGbj65MZr7/1S97Ccm3pkJt0jCyu4lqjlzGQsBgWMwte1Llds5a1b64HRGxNKYIEPdNyBay5wHLz1y1AGltVVRrgxNmi3CSkBaortk+Sm1FU+o+KJiYAjLxT1xLKRsJcpK0/GSgFhEwUJeROVdX8QsNuixLmBGq1aiZmsqeDdktox7PZIhKxZoMPteHVK56gk86aFAhq04TRdp8deVvfQbzulkrGmeKnJPMzWLHAcOdcNxK46dKjhoL4VVE7ryFXUAirm0JTP+prTaRBjvJyIqyHMNhAPqlQBr5KGuXSyTB5AhQTRzhnOSrMEYlcoul252MSqCWV2E1kPaZtroV7bkOJl4ie3Mg7lOuARp5LrNYJX8+X8FFirx05bIwRgjgs6hm7SpMKaqeSHjg2Vo7Z0Jd0ZwI4ZBny1zjituQaXxS5hbh4Be01XOjmKvPo2/nMEF6e+AW5pAvlR1o1cVzvJF9iMrMiszj1ULgCXtLd39yg2gqqtwSN5Mc8w5J3uDgGht206nOzJy5r4PZJ62TWPu1/32vCPntrVgTM3OzY072taj9xZbzA71v/7puz//5fubUufePr+fdw/X7fXz/ddP998+t89Hv7tNa+3DJfJwcwLJ+UHzOIY9hzd+cXgyd8FoQelo9mfCFomcLu0q8VK0DeZp/ivH6GteEt3VGwQMk6dRQnQfAPoJebSSoLs2KjUtw3EQ01r0RuTYB09Xfv42Pn97/uZy/uopI7aHh9Pr69tf3n3/J0ycz9u23aN3zLmxbRmboleONJtFlB6Mg2Q0r86cCnQf5DFzTJlNLcqEQPCluBoCMJUzXQq4LGtRGStrHgvtrot3/G5a9khWlO0rA1aT3iBcMWczHoGgS50aw00ReEytyjqkS/TYNMHF5VYce+QVSklqi5Sg9T5LkLryGpkl0smy4OFsP1lyooUoTT0XqaJUb92NwcO1QVKLVgVoQt2KZX59i0vb4gte97zKa5evEavHhtZpXREDX7xyRf/VKVUz59F93ruhl0jOnMlUVQPUHSkDedAI+bIo4SlVaxVz5srQrhOvIlH94QtHCVwa2U9e/8jeLyL82KIKWVZkY4uUtsV0XHS4CgOyFMkWrYVjPUOQrOBpBV8Fz/kSa/sPE+t4lzomtXz8sXYCesEVo5JoVeNiv14yjrZCWtAaQMAt2OgkO0GFtzhndSGvvqYl1QQRsI4tEW0VKDrB5W7MrrQSqszXWj243lQV5R0J29Ly+GoJEptnR1XMVTarjGA1Ey9nzoOSsImvdfFf5BTIWIMgiuR26axrHR28K2dm61u0xlDeZoIzd7Df9otGNkpEi2bZQ4/NFAwV18tz9G6L0aKNzMvltt2dnj48/8P/+aeHz3/9q189iHy+XkdusX3Gu32L1OOQBsbobVRDJKy0drTKxLjFVGGcMlNVehMg16jOJdw0xkKKNYUcq5COQAhj4S8Syqp3j+kFL6DNRDKZFJyNrSAXxcAKzvv6TgSpeeh/AspJ9Q05c2bs/fzL6Ytf7n/zdvviCS3jzN5v1w8f3/34+MsvbIjz3ZxojRjZckpjJY99YgZbj1RETEaO7NEaWvQINbeXdgkuwl2PwtCjcv6wlRYJJumaGhSL5LgIdTCKJc0E0ATX8eKAmJloXNV05JjJCjfnso08AiJB0ZgzVntKCJkzm9HcfGk4E903hOVqTPfmmj2LTxCvbQ1q5hRKLexkZsG/1koUWBMNpYijKABwmjQWKULMHPawpvaNS0nWiBsUZZ1zlTiQiMry19iw1rT4iuVTox41Go9AHDL1ddQFAQgEQjPHoqhKMo9PZncc+cjySTSVRZDK9OCjeEH2aK3RfRHAuZqlV3Cw6jyKElrnwM93tFCdR6vOnGviBaRKPNiwVqe0lXrFkphjRVysTq5racoRuImbWa5CVZ7D7Caoy7q9mP4FZatBA5ecwd+1FFYZDWtEBZHqqB4KxV2CS15AIFjyQ4E1SthPvRKCmW4Pttj8iv4WrF96EB+ICbbFFkTknERQDdDB65EL3PpK1hYuSEFLkuvQwF23alz1crZRW/XCylUlNNbumtixQIWByIDlH8VxZU3/s1rbB0sr7CAbNISMQLRmDkshhtmkIUChJX2Skn3rmfO0bZl52/e0DP6W4/p8Ovd22ni7WRD47pef//zHf359/3cPDw96vIrtec+tfXM7C3k9STnep7J3VnwMArC+CTlRXH6EXCzhQeuHEVtBu1advFdD9XZciitImXshxpfQyPDSk0GSiNBLsOx180wYANUJFED1EHXQ3PzjySSbAhoZnXm78XS/9/tf+PlPD99+t/3qOd7sOiUjb5ef//zdz3/9M0KYoYGcYelNzh2YogW3wkwg0YSUW2pbNZI+vEwX71/HnjVzuM3p2xYvl5NANVxa1EfxBwtTAQz3PvEEWhDKAmVrGKaT8gGK061JhA7Ip45ByPJYY9XyEoj17+m4fKZ62wyh3bZBM1tvDuaOW++T3Fo7mIACqnX/q5uTsqitstT23nXll1Eu/biqwsyGowVBtzIOEyOai0RefaEj8EJD+NsrRkxRmVy9lGroTSqngmF6vcjrQA2vdxuXxSdCYoulN/FPRBAZZeC8GkvIWlvGFocBMkLOGhVZMrFYoLAo0MUS2PKOMcK9yeBsMISM4JwlhdNyjHTV+tErRcX7rOUIHX6tjlPRCFEkH/NIlZVrTwiqEQuoxp5OTVeU5zcqv7c8L9eyF18HribmWJF8nVesejFEsDsyqOqgKHKQVVtRT7WIJbv8yJxRJDyWS2QejQFUW1yhXHCZZ5kNdAsL91nwQmZOqbxTtXk6SEnI2f9lZlARDWx3a0iZeQxGNTNYbt7/luY8ahSOkJpH0+lUqhoPOIedEe7hSrHafilqjuvMyvKxhTUembCkpkUbY0Zn23pr5y5K6Y69ORPBKUR0cMw53OTNDpSB8/k1lbgl9v3HP/7p/u7ub//ub2cEpaEmbL190T7LQeZHnPJdjhF0uyQohb6ui5zYiRXwVtjGNSXVZlICo6GmTsrDMCpnGyuQN8XqwdRrlrLZbaURlgvw6oeTtgKSKHefdzsW1fdxQdOjiAZg205zXFtrk/2xv/r59O337esPfP08TnO7e7h/8/bdz+++/+7xlx9DirZpoLVt5kVzxqk5G+kal5xKKCYbeyojbX0yxX3fW5zIHGNer9d97sBpSU8qfYn11jYjWmzO0q4dtsgAvHQqZk2bu4ccp9P2HfawkZmN4cKwiqgtB5rJsHSw5icHw7XrBR4CYDKiRSzsWWRl9YmqpDGO4AMvTBSx+iSvQERUKHOhyBKG8aUbXR0XAwCtvKA/MtwFITMlWjiklcDza0o5PcEJh0rK67GGqwiUR9rnchcraelkhNlMTGUwEh6DXO9bOS5zCTa1WGDSzCfSKXGtg21fWwj4gJG2sDgANBiR004rorWcc2ZWwoUVqYTnvxsopNK9uarorLyd7ZsWnijDiZKi1gahXChfCGWwGkStg2b4WTFGpc98moKcc+YifdZ2r4X2JxdtDwi5IL3X8eArpfppkL1oOh/6qWhttbHypxxqyuW/eFjSjFXAUGGo3FV+USqtWuWUuyGq07F3Za6DWSDzcDQuel5UJ+EIsn4kPEMYLQrRQAq0RIoLFxuHpWfFFP/mJPYRzWmZf0BMdXf8kGN5jwA8xqdgwSi1oqFqqF+17c9kQjlAJZJs59PZY85vHGPfU5gTETkwphS9j8S4XlqP2Pq2bcjtdnnaTuepfHz/+Oe//9PXX39zv8WYe2zteR8fo2/9s7i/bvtVT9fk81m7NNe8y6jhN048Vjs1qHJHrFD96DTujv0JrYiN1Er5YjGWdKaktUgAcy5JdQVZMj5Y8zrCP59ZQz9WNMoiJggwqxQmlqCQABFtYPuwb7/cff3z3W/fti8uur9Mbuft8nz98Mu7d99/n/s8bX3st33r25ZtYM9DfM+JwJTUc3pT9mjNKdaEMoK90XXOI/cxrrddgCtwChEnSlBdjcWKcUcJVXHIDVabAxwQQxKPvnLFZlTBlTPRHhMSCGCWWEVGflXzZZEBoCDVAsjjKswqPnJ9TYItlRQTFpkv8tqa3HVLDyhuA17ovtqoRZmhMkdcNvoQv1mywoPBt3KpWFGtuSq2Ji9CNNX/fpF7uOLPMWc0BqlEzjmw0LJSbnGzqNZlzlKW0udqdejDzbrKkQeNI0WEO3scds1ovmqvS4yEOr3lfyFUbRKkzFExHkUmA2PM3prre/xsZob9I4fzSce5KbF2kMvh2XSUwyhreEAJAmsKddEzx85V5hpF04FEi0OV/zK/xN73JbqvQ/WCyunBrlNa+VC9RHj2+g6t2JXozU1sSqYzx4gWNeZNKzxxlPAS31k4CodtgeVkKlFoshWxLK6Wn8zjlJQOCSuCUzQmYMGEN3vm8lqrFa3lFylBGb1Xxm2tBUgVVgArqYWEtVtuMVhe1tnSTyP88Bmdhx6mBMRVGGtScbWK0+QcaqcW0XPclKk54tRHqoXFvtrHGHMoQETrbc4ko/Ww3iGI7bwx1CO8ZFM83z1cr8+P797/8Nef/ua3X3/5+efXp4+P+/Xa75/UtY/7h6nMvP21aWqfp1PMoYhk0L4gofBkUVJHw5Cwy8LiyipgLuqbXN1rq09hHOUwiNUr52gaypoK+0I1rGTPmhNpSGJhnyz+Y1hsZcsavbuzNPbM2K776XL/zfvTb3/Oh9nP6mdlT3G/PL777k+358fe1c5nta7WzRryFKnJ3qflxZmBaC3IFiERMxOxCIoWkWBi2+7m1G2/Zd1MX4amqXC7lOJCHPrjqKIpnFblO3BfDccx69LJzVtMsKwGM/bBACC+ZJQT08jHMSdW+oBUMw6roiEJyNRgBp0iDiiUiO5r06JufpmWsigL9S9kXyrBhdWMPM0pzWpc5Bt9TKHw7lZTCiurxYraeRQfRyvm3cjW+ksL3Q0lcrifeyKaqn4r6gmlaO4Wy5k1AVqLtqUneZFHgCJ4Rm4suWbtU7qWKiJrkhUTGTDlV1iaFdzUKVfBSRapVGYt1uaiRwcwx2zuSOoGOW7qcJDvAcyqBTuWu2qRseKf4v6VWYqJRUXbcZc4KmrGy2oxW0r/F/68fCSFmopIZ+0FhhV3FfCjItOab6ODrhLkEo76ahNeDCV6xfEGuVCOGQ0l2rHfrSyIVnxMlcohUukxxy9YvkZ6/le9Cn2K7BSPw1og4lgwqzZVzDNXar2a/6H8S055BNuBeJ1rWPcz4OE8Snd6iaicsxEW5srCv4iiGuY8EBMJZTK6HWTUoxiXQROrtImtn1rrZEP03W8Ntq1vrY85xvUyhdyznRrW3fIVxlTmLVrXHPscmPP+1Rd9a7dnaEaoXffbH//5H049X7/5b66Tie1xKOc225c63yf7+SnG9a+hpyn17ZRIZvZocOWt321NdiutdiMS0ZoL+QBGCyCtGMUsH2pzPheSYg9NM6Y0DsspNrsWIIABNUajrYWtgs8dAEoRvYoLjcA8gVLq2wZiBsfYHtub7+NXP/CrZ352i7sZ23Y+zzl++Ms///L9P4UuRJszjYVmDkjoJ2mIE2jp8VVF4Vb2QgCbxbtNKSY62Frst+tl360RKc4BR3mnqja8mKFVjAKQtFYwq9VOKW1Bai4hVk751JWFSMCt6WiYKEaNPyq7T3qkmrMvDEhTqUy3126GZULmbK1xXXjGqsEr6+UIjiY31+1axVyu2C5O6EhsVtPvEtQvGGRsw0oE+SYua16omQenVGEAvTj2MCUptKLHrSQYzKGJpV4Ca0JW8FAWHARAzmoRuHytnUW0UK4JMKocvN20U5MV7GSqt+bJT0ewtRCoRMyZx/yJIp48NY9uiuevYzLdCcPnwh1MyJAl/DaXK094hEul9Xoxe2X44Izm4uxfdkcFyPzDlQRFvRpfZPkLYRw+BUdwvz4WR3Lkk26jL5bWtsxWnPAU5ai6z47KzdCjz6PF4gyI0kJhfRy1yj2k1adJtSlZLaLAWqbpFraufTS+LAO7sgLrzKUSaM1gBW2NNKuVXFyQsOCAT4BD6AICotbE4DJePjFzTLCSgdZpO+p37V1IyWqlsKpLPnlRByulRarkSUrBaD0QdO7a0RiJrW99O0Ez93SCkSnsK3XYA9CcGaB1zR7hcMl5fqPTdtKr3Mdg73EbH3785R+EL371q7t+Pt+38fx0GZzzkv2M06+y9dZPunyn+XFm9kbNSV/BCChTuUqEvIsMaaAK2BZ3kazx2W4165FmYcE7B9SE3cVciYnWtip5yRoqx1oQhI70moCFOlJQuDrPo4QYnCnP1523gd7V7q+8e7t99cP2xft4s/Puunexcej5/Yef//TH/cPHrbXe76/XcTqf2La5y/F2b3fRx9yvGg3uTxYtIpBjzF3VAjZMZI3hulzdZl5uN7czYLES2Vs/WJF1tbSoEilhYG8YWnjZA0cBhDjtbsOKoLr7VdmOgn92jghBrZrQZZGIgEnOWWRBuYfFwqrsKArKs9TKAesV5FxV+SWIRGjJHIsi9IziivmwRB48pkkU6qra12WSTRt4vs0B5T5h01Ny5avzkGXhtdyb5SWCiKn0XMM5S2ZnjGjyMP21Da1YgeZYqjIoLXyqptSCq6OZYNXGqtT1y6SHghtqHoxFhMrRonp7EKsW5sh8HdZGXLPqqnNVUdE+Hulyh9WCpRbFSB9uWnmEXSszrHW2zIpXCsBnS1SqhUXJhxq1cvgs0SFWBjzm4uj8siv6OpipfEkByiEbq8FDziX4KF5qTkWwH+a7evC7QXTjEtIKyxy+OBxzSFx0shZiZ10cj3StkGT5PVeeWgDhj1qdp6nMKBrQleKQPPi0bk4FhrWntHzAXf4X80i4raGvSAX4i7D2a1fcJ1DV8ENLLqvj6QGi5gkorc1lyU6TdAJMQQ7ADTFyzGgtGumQPNrIvUWfc/rkVb4uMZRW2kQy5yQ7MaC4fny8u38d7I237e6+M56vj08fP3z317/+i9/9PhEjOVJtO7+73Xj38Op0av0u+8N4/vNpf5/zeQOESUTm6C0CbmIaSskaLzEBx43gloHWN7ZNCLFFbOpNbFMxXf9EG1PN6TG9yt6xulBLE0pqxhzSTE1U55iqKgokoVASylXUJArRsoBRRDt/1N3bh6+/67/5pX19aa8us8+I8+n++fnxuz/+/fOH9y26gvsU2MfOmJPZ5tgnR7S7M7fomPPGznTUnzPA1tqeI9PtG2J1mVYODOL5NuYEOiXkMUG6ruQ6nEdISydPXyrnHJUa4yeQMwE6BeICZNuE+ETr56tk6FCEkJv9lsktMVWwCsdylf+g+FcTCy+pS3vZheyaS7VXYFAVPChhmxaRG8UPxbJQJRpb93DJQ1EGvD5ei1YuUf/qlKCVkCbhvGtpMSpWPFJozgowZZ3FsoaEBaPRwhy6PA/cinUANUrXrKRAMEW2UjCn4ITlwruu22uOWaPofx3QJLgMPlwGt+yWv5tkzJyNhNi6+9YVQ0BWO7nyzVq0vVOTdSNkBmJRcDjO0QsQXxWZi+LwYgZSaqgRVo5CV3rgk//YwpraXST88UlcQSsrW1lH9+WXyWq1oDnT2VrDmy4rPArJL7NevU/T4xSWD2SlYz6Vmggple57ycObe9rVOVse1/jC9ALrnEQlFLlk/qvJ0XLHxQWpbt5ainXZ/GK+lyrhh69HpSUrrCvIUqphf7bc1Std0yC44679YMXSKk4AQUTEbdyYW2MyBxOJXehjTm975kw2zUEI0TiGIpBo7iQUagolNezWMGdG7yBy6vL41Nopg3PeYts23j0/vf/uT3863b/51RdftvPDmM+3hu3hzXNcg+cJXNReb6/y9tPD/vO4/hR5bdGAOeao3lQKsWV0RIitne7AEJt6i/P93M7gNuOcZLJntIyG6H3bAjbvgnpyAwO9ZSgyqR05ych51ZyB3CiNK+c+U/N2g5I5kLNpcNxaDo1L0wUajgXZOinenZ7j/uft17/c/c3384uP7c3A3aU3YMuZP/3ww09//p7XfPXFF4HMkWTL2xhjiLd+2rSPzOs+EApu25zTnfo10mnFFr31sxsqNET0kAbJ2xjX25jV3lmLMCtSX3C2R0XXVnlYdXpTUMs9Llq0mtuA1Fyt/ul+nAoPfC9nEm0Vt1ZMP/OAeL5NXLieRmp1UqNwIl6ycAVxUMMv6LI0S5ZVhDoZvq3GSswqDJIZJxOsvpCZi09YHMW6slTCnRVWt79VLLYoiqJ1lvjaz7ZaWtpTsgXnJGOuXg6r6RgXwSDIHZ1WmmXFElyBB+0qasWMJZmZLVqGNEt3LC/xotFQQUu2cFuFfIlUF/u1ilG9CxSKbmIJZ+tpVreCNQcCiwxbAVM5f9YfHbb3EOeqGJKXvzWzB3C651R5Cu9JegE9EtmjYGzEbZ9lOXtRiYVL6n2rXxqi6nwPTHwEcIv5RPYjeCfA1gwtIpoZIVealDbOBxvAJ37C/H7tHCqPJNd8siTRB0u49kPlKEmvOA7DbSXHf+U2HUjWczMisVqqe3nduLPaC1eiDYmMLLqHjv20sjXFiOFg2YJWXldiOKjkakMIIDJnC8yZfWscCiWpMW7tfBqXPRARkftgaM4J+RCml5O9FWRI5JwA8yY2AWJD61tImbOdG0OBdr1eew9l3L16/fjh3V//9Mf77fTm7jUj93Fl1+inR+Xjfne3tfb6TeI3bbxtlx/i+j7wfNoU47YniG3GXWx3uzacztG3nRtjQ2vXzL2feXq1C3NC0eCO0227u7+7MqKddnVu56vuB0+IEBi46ekdrx9aJEHF4DmAPDuFP8bMzHGj5kYEMOeNM3V93vLKecH+HMhYiPtx6HL64vmrv/3l+loPXwVPt9Hvv/xCiKePH3/4y3eXjx8/v9+QJNvMa4uJSOV1Dp17P51PczzuOclza+dt63vObNGZbnucU3AZvNTckSKTvWHwdt33feBMWmwIIZFusW0cYKjghih1dSVwzrmYjiX6kNw7c2p6gEAshEsTbiZm2Ffhou9sZRIrG+sumL6GtkQthERa5QTXOEbhdyy2OlkTlU0CpIcVLwpiMRZaI6MPI7XiXXPNZbyK69LLMyktlYh1Q93iq+zISl+bTNKSeCz3UeojkqxixJiZFcJ7WhCPC6sgsUXOFCpVHIdaiS/ZLJetFHeEgtiZUxUklfvRYmAsdbWu3cXGhpNyR4PVvYYkApzmBhDR8mgrpcNnyrQLKmiA3BT2hRB7CRwNco8eCxQKfXvtM4820iwYIs0smStBwHO/SLbWZmbQCOTo8FMMEXg47lqQ8ptG2onFDcppID+PLOp1VirYq+TK1lKoiDIPWxqs9M6KdIg1ns5EOirSnJ5J7vopr7MBFhPmlNy5EKvUDmXZVXMyM5OtElCAcs4FagRqzqL5VhWcCrM7oRB6SRqR7kMrHKkTP2QWl1rBfTlzVav3YrHcN6sIbDs6xyWpAHuLoHK/ztPOeYvWMYXM3jcK2MfIKY0Wm0FLpXMSgFrvc5/RQXJm9gjNOZWM+dk3v3l+vHK/5EXX2+Np206n7XLZH9/+9Hz55uF8f3p1ns/ZHraBnPuMlud+vp4xIb368u6z3/d86vp4zVvqqrib6hlntm3MuEn97iSFxk7ylrmrCScw0DlnYrKf73g633pHnG6Z++lNvPoyX38xedZk25qePn788J/n+w9bRxdmgl2t9153zAdw6w3dMf8pNDG3PW+XAIERwKlvzlI87/kUp/305bx7s7XXc8/26jO+Om9ol6fr7WncrvutTe236BNzZGukIjJOyrxGbAJSVzLk2ozeoRwzI7P3UwuOcSM6EhPMhhSFljOeb9fbXi2MRs7em8sYbAnoc4UDgJi9yrqsBX3d7ifd19QXo6oGYyVSxaMa/wC5Wglni6C0Bk0vqFolOeuHFxOwEDsKHFUQIKTQXQeCCC35vGRFo8dg0QMeSrlvgF7tbgJQNaTPZRxg9jUMnEAUX1o7vBRLa5XK5qF6OcRqaFBItPIHrMqDVYpl4UnFLwlGBJpp5+WOjnAF659FOzvYMscV0WBSOUwRlc4HR+EJTW4fikw77EX62d35VZb/rfqYkvFWdqE0DLWdtldYU0aW+1krtgA+sfaXh1vSUSYNpTwJh2Dfesk3waNbgZ82WBSZYy635cUL3fYyIL0sFQC3WVTJ3JWAm3c6fKjQxUPd1XOickwSjo5vxbW0BROw6r9WucKhkT/iQKUVChnGGOUAq9qrchpGR/T5rBA3WmlD/Wipoj9hzVgensTfeWRmaA/oVB/M06vCnWDFIjiqqwsXICxRxpHnsS+ts1CizwIqllaSldmshjlgztEaxuW6nU/ZAA2SOQfYlMpUi2ynqKB/VqYsWutbzCFQrZ1EZe6X5+vdq4enp6e712/y2rhrXi5xOu3Xp/3xqsT79x/a6eH+tJ1fnfMU0dt22ubTx18eP0xsD1sXptrdF6+/aW3st49jjtvQzBans6ZG4rZfNci+Ic7jtk+QbRs7Hh5egWoSWnMO4PGSl9vz7Pev/vZbvvny/OZztjuIPTg/3I93j7/88mPO21RG9Otlby1bcN9vz9f9dN62U98iMDOA1jBHIvrU/b7fUtv9q4ett7u7+97v9tsc4NweeryZY969eTg/fMaGU/T++ed/83/5b/+Z8/bhL6kZ1+u24bRtuY9G7PtQkCGwuSo42+iMMWbRMxFj7GRLUTkDbeRo5JR7Im7X237bd7CZU494Abl4QXGHSAELbrhB3tEdc1k+hJBhqhuVs+Ph9uvEqrrAexKLEyTkmDmrX67T58GYKpJBzhg7t4gqhLRc6EUB5MJTUwMl5qSNQrIoYLl8KxzWvxz1ChDABWCL9a3fIVfLx4oK1vosjF+wqnIFzpIQi7hZ2lDnJ1HGgp/MAay7UcLTmjRjS0RWz8j6vMWM6LA2hYIXU10w1NVq/vei6lXuJGd+2j+ax8cJlvGXeNUzEjylE0dMUHS5LcxRkpauNPqk1R1WZ0EV5vTiHPxQbaD94tFDQZqIjeZ51gQFMuaYbWvLjme07kUr/ZTXliWuLhaFOvqqxjHawanAw1N5mkUlu9kd8CYqP3gQ9JI7+dmJqAQj6yTWSpIBTveXjuKYtCBNeMbkKr6oG3N4rophzBCZmznCz6UqsEded6+WueakL1F2RVN2zeX5Q07xHQFufWzhl3VQM2e0BkxPSLJHL0bNQbGmHwGBFi0zg4pUI0LJHjl3VeTumyJSvTW/HbBShm6TYADYATcQJjVxur8Hz89P13Z6dX//WuL5669j3M68e5vz+fb8y5///Pikf/Xv/9vP/8VXd1v7+PbpEmyvv9jHfJxXxnx8vHz55n5D++zV69Prz6+PHz788h6KyK7UnEOxRXT2robMfr3dti1O53uctrFf27nPqTFyzMTA5bY/Zdsv8+vffYnzaSra7LcxcmIyJlqPtt1t522Ly9MYe+8RLdT7GGMfyY2NMYFo7bSdL7cxA+18z9R1H4+Pz+3xgmjsp/MXXyC6Al9+/c3tHO+fbpen0c53D2/u//3/+B++/dvf/fBP//TLD//8/k//x+32FDHyuscJ59MWcRo5pIhw+4Sca0ZadHS0qbnPbFvDkJREgh2ZLdrtul9usY+Z3EwpyrmjyubZxhS8gLLE2rtQvY8q/zbnXMJst6K0CmiVPQLCRGViHDxjmbISy+acRuI28WYqYvX29EkPN48vS+t/beWlVsRrfCqX7pdawXmKkCvXdWjMc0E1F+Amqzpy5Yfbp4NjCnmT1FyxS8HMwtkr7ONiNLg6hC1SqS68WlWJl+GYc8bKKjBWdVUj3Lpj0UqtUiMvgYD89aSRMqwDqXCp2MWDEVcR5AQQPUhCapVlKe7L9ajL2TjtXK7WyG+x/OVICR5zIs1kOAqxz6j0NWw640Cydu7uKVxNjiEuBknVk9jtOXCUFrfe6kldeeyQoiAFJLUWC6z6hJXRK1OJStkSbFzEykrMAMiRbNG9uIHqFlRZZSyP5c6r6WpGKF2MgXWmwQhmWvsvj9DTiw73qOl1+kWWdtVKoARyFWfFeg3UbwlO6LASAg4+ssKxo/34ykYZzRVJR2jMVLVLZfOpDM8hMs0njxHw3jeuMBCLIFrCakiqmB4NSLUuhOYY3GIMta1JnNW6uoYQpLIUzqnGiNhyHWC2yhe2tt093G8PD7dd26mPfb89Pd8mtM1X9+cxbrPhw/V5Xm//zTdf/u5ffPt//R/+m69e97/889sf3j59/OGXrbeNW/TeXsfs7f3l+ny7SXMqdtw9PT/3TM15dz5vpy7pto8AM1q/u0vp4+VZHz+ez+d5vbZ+Pj08sOk2xxj7RXh193Cdul3HzIgxzhMcmfucc8QGwKMNW+O+X6+ix3cGiDFSwR798XLJmS71O9/ff9YfHufjPnHL8fT+8f7Nw+WnffLj9uarh/vz+e4hPrv7+Lhfrnl/7tsdfv+Hb7/99ssf//L7n7751Xd/+s8ff/xLxIipbhXEDgJsNX1wzgmqkzn2AaKd7LyjRyZdmic1BtjbbeyX/ZY6V0ydGaYoF/BZvruYujIhNmpZfWnCY42tCi9Q5uzWoo7MHVnwg+I5lznw/wzKI21COUrYL7iJjUGKA1y4QvngpQySbXYH5Vog/36WsCTTmKlhjsZIEw5uc2JTnBARjdWcp6o86tgvrA1NX9+yLxWQVJ7jQOclPlnW5qimWzk4LJ+1WHhocQwoSqruhq9QtYN8iclwyACtdE3nA5Q2iAvO20C/NOKw3huBQyKVJUlxwLfsRsHZKuywY3BHHILE4lFqagFfAqJilqqC6kWisnrogwc6raCqwohqgl3vXzxL5fjdjwfmGYGU2tZzJBJuUpNzHhXdES/s0jpcJtliZi46ZEHhihXqx32w+xJIqVAGqjwwSghRSxZazSbXMCXBsIPGPhEYY7V9KgWCFl7QkglQPNpOV5cM1lIKkMc0ekSmuwrUMy33dXBoOtpKQGau/LhAKD27nCuuXpyinPCn6CQva/gp6PGqIKtDAkofUCDetXZzoqkRJR1q0Xrb4iS3VcmpADtCkfI0T6/whNNpwQltzcrF2WLrvXM7Kxlo9w+vTg9vPvz40+XDz1c9P2Jcp7749lf/9t/9h9///nf/8u/+8Lsvv/r27u754/Pt42V//yFu19f3d5/dvTl1jqenxw/vzz1uwPVyVYsIJvvHy/Xh7m5P5D7n7SZRme3UAQq573vOCar1E1vTmPv1dp1j6LZfb8+Xx9PzJfr5aVfL1ohtjjjp7v40x/PH5w/7uc8xgLG1noHQPJ9PmdkRp9hu+y2nhnKM/baPB83bdttO/e71/db7/cN+2y/Pl8vj8/v2+FHSQ3xzevjs9Wen9mHqepuB0xk78ld/+82rb17f/fo3f/37//Lhj//59u6ve2o7ZbU5ybFFzELRFGogz9AghMlkJxvYEoNkRCOZOa/7FJpNVDEDBSHKKh2WunB3vIhBnPslXg5INaypA6o6huQaFSiD8Kj6zFUBvAR/+KSJP7AEyodtrcGwJVUhSmyXymYDW4jbTEjd8IjmAlpGNDYCiQnXkS40b5t1KLkjVk2WlyDXfXsxkSUIPMhU26cquSoKoCLszDzqcqpzEkqB+pK5ZImg5kLJntHrBT0mH+GIKLCon0CmQ3PVVIb6pupxN1dTU+dDWzuG9x6cWnnSMm/BF/5s1c8fjh+aLGFpNaJ0nU15XO9tsRu0WSsPaVS5Ehs6Wq2WKzq0m3RnVnEulsnCbTfdksOug16rcxL8tJGtoGihWU/u8qsiirD0LGKAsq6MBRq6nwduwcDEenqjIffqi9U90IfSLtFsCewkVm81AEqPDPWsA6g5R+HQM930NtxXduVznX92dzh/gRbmKEDBukifpmvWNfShcOHCeqCEHCvUq5VywL7B/KYqYeC38DJDJDIJZB5BU4FCpBrVQl2gMgSiiX0iLfS2NyGndWoWF5Ihd6zXZGvRWgBbC0TMlGE1Y7t9vDw+Pr99+/b64d2vv/3iq69+9cVvv/3X/+ZfffPb3375xWdb6x/ffvhf/z//5XodDGxTD9vd6f4h8jqu+74PtjaA/bbvqa3z8nydxPnuxBb7frtc92CDqIFobfA2xvBBGZlB7rf9Oq9Pz5d+d1ZQI0nccubtMqLd37VTZuxqVg8H2902c05JiZkDFBvmmPs+W+B5Phm/bO3UTtjzkuBAIqczP7231h/aObA9PT3f3n//x5F7e319JK9oT8/jss/7N18qr3NX306//bu/+c2vf/XuD3/403/6X9/+8KcP7942jvO2NbboTXOnEJYaMkZmTpmvU0uiCTlvg4xovcdpzNvtNtPzKhEMl9H1xXEWb1vZ2nXgJFGBVQ8jl50b4a6g/YCU/1VFe6qoocM6lnaGNTHY5WCVmtJqCk2rkzNzi76QpROETc5lHfzvAu048Hnh7NKP1lO7OUdqPSZzZishkehMmq/SC/peTsp1DDii5EqXJxS52kssL8bFmyxhBYuYigWQWUH8aj7ocL+WeqUZWfNV1rdqSTCjhYfxBVnTKeqZqyih4LMIln9yu73lSmo3XHpxKFzxIk+pfwqlNqx8Y4RSc87eqkVKlfJ6PrBptHIGLyuYOBJM9LnJKqJybBequWBZYU4BV6evK35Y0aoP1WqOZG2bT5VYLSu0eKgSIivaChDdKAywKVfCvdLhOuJQ9SiyK6t5EVEF0NXfiNaFLaZuKab9W8EmY4fahFJjTgAVtZXZLtGnll8Od6vhssjUUW1fqsoy+pX7RU0v9V9XWfNKhWmlsLG6Py1056YXmm5eHeuKNyzitJxYJd2JKthRhWY+Mhs9tKmNm7hFP50yMPerqmkUJCg7w5k3uGB5CD1agin2wJTutjO2frtdWvDdh18ex37/6qt/+e///b/+N3/72998/dnXXwV0ec4f//rj89PTx5/fjcs+9/z9v/j156/uPv747vLxcY7rbd+FvW1tO/Upnbc49dN5u5tTI6+3y+P1cmsRpw1EsMfIC4jem5C58+7+oZ/uqPj47v3d6bzdnRHRHh46U/ut3b9StKtG3/O8X8flkvu4uz91bh/ffxhjH7dJZN+CM8YcAG63oTmVON/f3d31ferufGo9JvR8vVxv49S2+/P57v5+a3z98Cpiu+23px++b4+P25dfD57OG8bQ5ae/PDzcjev8+Bbx/GoO3X/91b/75v/513/6+z//p//j7V//YR+3odumkzQbWw5tvRFAa6EZ4cF7lIaSrXfr/FM5Z97GnFMbGOFGC/0TmoHkS5BbyBPriKEMFxeQN21SFGtZDRZYxmL9cKTtrPpHXR8nD6o+yMgc1loG4fq58GCD3lC5ABtfJyJr/MhBedRjOLwuifTBWMh8sYqKKetTFBPjUFcuygYvH+pbK/k6V6DuJ0Hk0ncuwFxLYcn6ccdXkSaLLStC4GALCm+ZxHVAh9V0006iRaQHaikPMxL+wTKgOFL6xGqKkLa5GUXfV52UKjCryMsmM19ooUUTVWxUuHtiMjg816GsRikGSy1C5szDiXmztEIYf+JqtVGpCAZTqenemlwiGRCtoD3JCFjkoAP424dRn3QXtwmP8BzDY/v8uhWJZVTWmmRm9iWPZR7N5gJww7LC9TXJk26M78wFAYnRsM453ZkvhdAYNSeANQq1UrSxttkzg5dxJljjHr1UXtYS+C8f43CaZeeXr3BEWci9jpObjvrOZdHVrI5BZtA8A0hRiebq6nkksdclYiE1oIoU5aKCGqo3hdxaJ4gItWjK6lpjOUP0FiEMjzNt7JlxGzMIYJuJhtzut/3pw563uIv/7l/+h3/3H//73/7u12x6ut7e/fzx4+Pz5cPTFqnr0912+s0ffn37+OH24ft3j7g+3W63x7mrd57uzspxvc4CrR7YodFbKE791ZZzOlptza0b2sP9/cycY7Jx3K458/7uXp6eOPP1w/bxw9vt/GU7Z4JzpnreIj88Pe/7jeJs2Uhup95SSmmkRttazsygsvdzSHp+vsLppU4geIrWT/N2u+37nDNOG/t2/8WbV4qn58ttYD4/fv7FCafTV2f+8sv+/P49EFtr735+2p/36+vXn725+83v/9AR5/v+9vs/3j6+Y46Q+qlzIqUtSCJDc4zWNygbu2VeE8qcjJbi5XqdqXPExIs8wYapTplD+JXsCrZStRwND9dJCRzs/BFSLtl0obyqjjRkjghoHgKktPrZRQQmHlc/B1TaeVq9owon3OqVsbLBkEpQUjhVh3Fx31CUbJGU5pwv7+XsdbozS918v9TKOHJdhrp1jjQqjH6howo5sQSLL2/Eqv4HXMKayVUhqmoNpuirH4C14Id+yD0BSnqjVYpH1ohjudprzqOEotJ28NQtLabFDMraIL9Zcsn/DFOL2chyfukBNYRcwLd8J8oXHajyiJac3VxxH3DotrxJpCpD+Wluo8z34Wm9r64tRwlM1VrMOd2fR+ZjFKj2LvYA9mrm9ADXazntYTB7jM6t1HC4FsEtIzu4RtY2rt4VFaRieRkXbS+8X9Qojq7YjmOLBRWl3rnywSq5GlYd2hqjilgtgUAA1ufauy/LjhUvJ4r5qicCKiioBAAQaIlRsrCDjcIKElaTLBLRGnEkmsuLRN0bH2MeRGHFLAFBQwqb0UJ6s3UyR2/njEY1jZsp5vBcksREogfJ3k9zzgRPrTHafrve9dMct8uHn8d++Rf/5t/+x//pv/9X//rvztE+fvz4D//0x+eny+3j8+tXD5H47MtXr7/8pnddP757/PDzuDxfOxrZAojZYush8nSdQ/tswakc+z7m3re4v7vr/TT3/Xq9BNnAkQnOy/WS0tbb2Me+j30f523MMRUt+vnj+7f7ztOXlxafsZ/OD9xy358umqk5ZmZsbTttjJx7jnGbyRw4txan89j3ObT1LRBj7GPcJFxuz1PYzidRKZ1an4EcY9/HmfHq/k27PCKvTz+//fDTD+eH1zo/nPqZr88DbLGN04Wv8P7nn//6/fX0+vTq7u73//LfnM73P/71H68ff9bYNXfNbDhx63MMzcgcHWgR+74fzhhStNNt3y/7bWQiWjW2n+72WjJ2LG+wENsLTYECpCW9Drrm8QC0K5AoraCkVUa5TqQN2lQ6SoC4LP4iKqN067A+h0VAHJbFziGhVleJKfUqkio6uE5uNZszrb2MbCWqW2JVa66y0nW/pNIXLjtQdiFstlfvP1+Wguc2oOa+Wlszh1flmz84Km7xzQ5H+X75OZORlaSRS++RpHImcunU4SkbvrDNKhoPOU3kUgHJTO/yYnZDwMHpO9rI8s0gSpmyQjugCpRXB9fKbGt1NNNRDn0wglyLvtiz4mEM3wsjHK7CZ6W6SVdz9jKtc1EPR9jk8MiNVoqUqDTz6ke7HEzB1LXn7vzopa+SRyRIE05ZjTXZXZ2x9qZID1XJlafDZxVHQixfHZXComrsoKBVvsxV3p1HCxF+og2ojhRYNGb5LgirN2GF4m7szEq5gaaYdDCeK2xUIlqaak3NGoROfKK7sBMoGq/q7Kl6zaN0pY6p+0jIQ+39Zq68AFys0KLR69M09l0tlHOMvTlCcX9a2tkE2KKFkhizdctZ+4Zx2/d9Xl6/fvPf/Y///b/9H//vn3/55udfnp9+fvvT+3e//PBz79uvv37zN7/7Ksd4/OXHd+++yzlySJByajaeWoc0Y1x2ZqIp2jZyRmwNcZszxxRjU8fEfhsAz6ez/aqoKY3rfn26WpS19Y2MfY4Inbeto0m4ffjQPvtaTe2G+1N/vuW43aCMhnYKSLfbvrH3toFqbWtb69F7O43brbHldJ1LsQ/jNvd9ttMGABq9n87b6XZ9vP7yLh5vqb1p3nE+Pl322/76q9hO/Ra8Kfbr+7zO7fX26uv7ODlZfztv59/+7g+n0/mHP/3909vvxri0vrXetM/WujDFvmcKqdaQ1X+tAvCI2xj7SJy5bmx6ZOmiOF0iqKqf1UueDTXhdDXPcaff+IQtWpCCshxbq2VKsUeqFbFZdTdWgaGZAkyEWiYXQSmjbf6iiBZ0yoLGYL6kxcRXgs7qiDJY/ta59BImdXvvqFeTgwxoZcIPhFU41fUyBaq02uOYz9CKRY7oweHFkrEvc7sMnt1YzlyMCJUZrWWZNkjVbLFQJOCSUsfdzo9QrdLza6FrkgHUrFeswp+qwPLxdlpiqqptV07E5BcgpCZZlcZY+nUcrXtsZzx7yj46XqzWUYuq8qKLrADMDvll8+jtoRcnkDDq1fKixbYdSVrHAWBlJwy+sRYWZZ6K30YxHFW9DTdAAIDqjKlRIUvZwCDBXp6S1Q20yLQjo79cHSqUqmoBc5VZFcxOkPvoVBi4MhE1cZTkUbxWJL6qdqTAuZGG6mcr1LA/KaWdxwCENVasSn7zLQCEiFEyaLXWTLbW2jJiRc626kvo6vDCMdByA+V/lXNGVMhFQ/pgdA+RtvQt9l0dpIIefdiCrSXytJ2ysuScyXmbp+0MaIxbCM+XDyn96g9/87/8P/6Xv/vXf/f28fLPf/zTd//819vl+tnru19//fmbz/rW8pfv/nT9+PF2ud6f+tb7edvMcCnnFi3Hfj6d4qwIzjESOm9dysvlGoGHV/f353sI+9jnnK2b2XP8zX2My/VZQ55V0GK73G7bqUfrY7+FWt6e3v/jP1zbw/k3f3sbs2+4Pe/Xy/UsbfdnKW9zzzlbb69ffXablzFnal6eL56JliNdjDpGth7nu3tyjJEn9Nu8XW/Pe7tdo/XWt9Y1dDr1edvH5dpzjtt+fbvvz/fx8Nn57nVn76f8cLntmP2+c+Tt8fE2r9vp/s2XX/c4/3T/6v2Pf94f37b7hsymDWE2ghNNe7K3zMFoMOphw+QcE+e6G81GVi9YLt2qE0UAEmt84GHoV0ggJ9ZiVVJCCCndu6FCb8fi04WdaEQLUPJo8wWfjW+mYSxJ+XgLM9AiFuFTsXDJTZWMVskBt+1ax7jEoJYnJBSM3sLD7rnkiYBWD99cIFWL2fjEH9SPLpNDKmcRPmWo1RpbUJ7nZVlTInMUBlxtKcHqAOb8bWUo0sOxZx4dY6paAhBUTfBt3ZMGbZJqEWwt6+KCWBqXWK35FqMAWdX64i2jDJEpHpTf83BxHbTgwX0VRb7852Hya9kWzbIcZBHU69mwxFD4pI9a1SrUr6/eUDbzEcx5kPtqvbn3uCmdT7tASHCTycoelIs8xKjuGgsbTczive0jux+v8vg5LXhfOq3AFFvzboZRuBcdIU35KDlhG5X+NWGzbpRAMLEiZAFVZI1FMjrd5WZcHkVWuYcAoOnx07ApP05GLXZVYaDeli20Jr3ypcYsFpu0/EwF1BWeVFRtYreECiDYolcM7DYSUQXHIUUNfAcbSWagb6dxy7Fni4jeRaI1t8bMMeg2yMp9v+555d32d//yX/3H/9v/9NWvv/77//Of/vzTj4Qezvf/8g+/fzhxHx8ff/rxdrtdr5emuLtrr+4ftr419H3sl4/PyTFvgPJ03oAYdW+0X/fWt96b/ew+9mC73facmRN5u0QLhdq2dcVp64MzGIE2c/7/2frXbt2y4zwMq8ucc633svc+tz7dQDcaIAACIEBKNC2RFESJoiSSFi2ZkhzLlCVZGdEYieMvyUh+i0cun504Hzw8YsW24sTykCjKFMW7KBEkCBFsoBvdp/tc9uW9rLXmrKp8qJprH3nkEOzuc87e737ftdaseuqpp54a0pi4pGGc60SYlrm24+n88kXaP7m62NtyhraITADK7sho7G2283QHxPMy+6lo88KsqppSJuYCKbImj3URNm7aQKVJRaqLomxGMC2aUsHNJqf9Ro2P03k+3tq08Oa4u7y82Gz3TMelHs5T1UY5N8JqmjhfPHjgtP1t4vl8QyJcmClJcz8gzCk7w9hq5VQIiTlJ81qbfLDbzBDu7RbUhFz7Gw8oB+5yYUY/0vHY+eClU5Ea6kDnIthryN5ti6IhXsrHebvmGMw7ZMbsa4uCsvHClQ3N54bUddic2OkQIHS5pFqYdIaaE0h7meH8UbQBu3+9hz+HQLH87rW01gF2xC5Ts9jyGBXxyp06P7aeRm3iLxqGAoEULUjVCHVu1GhBp4SQxw0qtPcaPMYAIhqC+wiFJrJP6XpwUQ0u2Sw6sZ7HrQ8zQw/E994F0P15gkAxVWVf3utf0fVgBAjkgxq6OnkGrrde/HlcJSc1eraEXvIBoM/DRo4PbsdNfkzUUBGgT5O5n8d9EHdLO79WElvZIzRayFK9T9LbGP7Z0QjZH0TELqFBREBQo5RMGzg/oZYinvfk79ihi908rXVRra2yJNDup9YzXaTfXjr2EKygEgZB66QXdnPdOAm9dlCLsWFX5Vpshg6SzltNFgUWAphIFEqdX0N/imJ8Mrh/H9LpBJynEOxPbSSVzhfqehFdQRaQJ/IHISqCGgNkApNqZIY0LXV/uQGx1ppqzYmZkgIQJeakBsai01Lb3KRS4TfeefsLX/jyZz73aVL73d/+nbub27feeeudN98emc/Hm8MnL46Ha9CllLwZxrGUIScw1VqNDKTlnFVRZGZOUqW2JZe0NCGCoQwpJc651no+nQBJDZgSYjLQ1kRrRaZs2Fob8jAWbnVp0pgZSYWqCRnimBITHyeZTxO3thnTMIx2Tag6S1vOp3E/Sq3uhgyNFlnmegZEBKSEoAJi1ao2zUNJjM3UQLfbAo0ABwA7z7OqEdLxeChjIaW26Ha/YUMFkjrrYjIf5vPJTjM+eiwg2802DZsltWmqhco8NWf5Lh/uOb2NQzm8LO3upZHV+czMFqW3+KEjTG4tIk1aa6qKvsGDIjJFaABgIlToy2HAwl82zPvvhwO9TUc+a68urHfhTX+l9aR2DhOIgNQC3aNZ4GgAxMA6HSWCRnvT0IUMa4vCuhDbNUBOcJgiclexACFKxCI3iEE1X0Ecac7L/wBiMbfVQVu3soBOfPlJ6ZyOdglSJ1d92aShqhqCiSGDw3R3IPW36COZBuBSeu8VAwKCEievtwh7jd4Bm1/mrvRXwJi79gvcqWMHkL7bA7G73ID7F3g/JlgZcOsh7tYavkPJV2V10T34QC92lB69ded31ZhJ0QzE1zvFHUDy1fDdQNafBfTKjyKuehfXu8qK4Bcl0l1nlKATRV6WeMHnNFgfqoL7p8RC1RuMxj2PwS5GQA2s7e0SUlFC8a8SUfKFMBC+PMF7Y+9lxdVe55ABm+h9zlkZwIDbvRUQhBStkh6AmP7yXGoh9MRgpNcRCa9loyqNixCFhB8v6M25qODWhwFUjRhAvVoEg+h1eK0CwUApuIE7Ea3bz9diHrGz/n3IOICIghlHKlYkS2RgJnWxoqCSCJnJtFFJMjcF1KXSMDImra2Moy44Q2tWUxrffPdTX/jKl7a7/Xvffn85zNv99se+/iO7XT58cv3Be98jbabtYldK3gAAMW3GAQxub66X+cSQcy6c2JoCZB4SmVZDESXEXAoSLbXuckHDcbsVkVqrgW7GsTURlwswiTRplZk5qToHlLlVaac5DQaWlMCaDqnAZthtSyZqFQ25oeh8kk1RgXmuZrodN6YmS9NmeSC370/EtU6+u813n9dlTpw4ITPrkBpmYH80CaYJ0JY6MWNdEudMIgl1M46AcJoqaptunlfBNJ7yfjMm5kxzWyrU0yRjLgjGm/HJW28XxGuVerpzWh9AiTM2Qc4K1mpLXBi4iWqrKs0fJVXjRNbVmtQJXPd4gIBatuIMCzlDSDVgfYZVidlRm6sS4f75N4+tYS/oWhQCUOzLO62jVGWm/tB3KgHd0A0QkVNy7Said0ZJ1VJi8FUnvco1U/SGQm/CRC/SohUS4dX9JyIFRgxB7lx0P9oeryG00bgSRAidzgdUV1Wa0/qyBn0fYWNOYhKxyxegY6QfNQORsMsMiB7FgcUVCOZade06QId8fjlN3UJVvVDvGidQjD6NAkbBjwFU3aq1j+D1dLZWdRaaeuce3B/S/TZ6XQbOAZPdy/NX0sjW3xGBinanmz4sh/fTxNbbFcGcx1S5f7CoOiLF4KrCcmh/n5x9N1GkAUITdS/SdczKejpDAnUfDn9z0hJ0ZBxVYUj8/SfpOgNpBmZuZGp+PEx1bZrf8ytxKMw9bB28i2qXGmi/zYj9pKmJ97T8fagqGGHY8vgdD3KtZxlUsdWVzkBBfWw6sD7E8bPVSqTfFjBTdGvw3odA7K8P/QHwb49spBj9YutVFzCBNVVqZC2lDSGbAhCXcTOZAZhoYwNUYkoqrYLmBxeP3nz8mU9/qmw2N3c3bZ6fPHm0++yGWj1+8skHv/+qTafNOAxjGYdNztzqPJ3nVBKIKNhpmVARQGCpUz0CERc2NUiMScwo5yElUhVTOB6PTWUYN0PZmFpr0kSHMi6C0lpb6lKXealAlGEYyjAMo0oDm5ELEc8nnWE2xZR4++jBZsMAjcByojKO84lFbWnCKSdGZGytigklzMNIhs2ktQUJM7NJMzVKVEoBAIFWpQHCMGZSN4kEZkAAWSSXgZhOp5mIqKTcDAD3G04lt9oIBdpcr6e83YzjLhXKeYd4bq0BMJcxGVw8fEqcrz/+YLp5rqLaNJOQg0NkTkikaMBIoqtCwdyKDPv2RosNq2sXzjtMoF1Q6ARCj5gemMQrBSJSCNmP+cRqfM/KxkZoEGtqagwm8VCJiQ/KijVGNpOwY0M2APeFN4gWIIVJgIIBmk9c+jIVclJZ+2kHA/WBPUK0kMZ7LdLPFNy3WaN0uWfGo6nQrRlCju0x0F7bMuicqRuLgmGwuN7iM0AQaxjUBK0Uk5+OoHxB7V6hDwA+aIeGkQMcACoaccfAcK+1wtgLBs6SU2Ja1SU9YvXlA57wDGD1jMCwLfVLpH3iCXpfs/PytjZ1YleKx0hFBBFNKSEAIDs5gYSdd/C9QwiiMaitirFx6fW40/9p8cTAWu30GsjzmWN10BUWrLg4suL6wrErDZB9JZypiDBxlAqqiJisy9xCDhyEYEAPiD/oTYk+bQEG4U7nSoGuG3V85DsoYJX1w3pjw4ohhAq9iAs8RQBgSGDajwWo947F52x7P6SXqDGO2NO39coxnkIAN0LVnnA7+dOlotR33SGFj1WMLOJ9elhtOTD4JkpIHhMdVdS22BlLLuSbGnVKQ25tNoDaKiA9ePPp577wxR/44S9dPtgd7o63t49wOS2n04vrj9vpAItudrsHTx6WlECEsMnSlmWup4mpzE2nuiCnxEQI2pSINtsNElprsniETcB4miYVHdJAjCDSamUmTlk8nqk4gihDAQRMKZeMAAScuCjh6XziVNKYRRbTBkDMAwImxs2mAC828n63h8MGORmAgi9H1SbNz1KrLWhmbSklIpBqVaqqlFJ8WEDEDLVsRwQEVTNNnHxpS53qfNaUUzDmfgZM0IgTFIQmqtLqqVpbKG+2A/CYThWnuZ1OE0MuJcNmL5dPzOB0/YkZ+FZgQHegJFAhRKO+x8JvMhqoYCJUAqPu2hW4BwGETJvAGtMNLGIHdpSPTZQQRSNoBi5GMjXR5g3GCL2A6ljYA0GH2ugzSgTm1OaqceuENYWpCYpqGJI5eKFoX4b0putXnUEwiNhOSiHtC/VqJ8d9JCZQnyNRn9I3s1VZ2FFijzUu9zQN9Bqej27xr10YAh4rsbnTYihturTRUSyzFxZBJK/i2x738XWhZdgYQfdHIC/0eq84XP7VFPu2Y7dX8zRFwZLjGiZiKBcAiZ3U6CRMzJFFdO7zt/G/oPIJ3GNHDQjYt07iaoW3iqO8UvBBkD5LtFL8gHElESMDKphFERYEhpPZ2CsSi5Ep8aTiT2MvitaisWOV+DH9iYj97eGZqoYJ3VwLuFemnXX3qx5Jhpg7iR5P9gqa3TBU3bwnwqz5ZjvoIbuPi3Sf8f5ee9nl58LFSTFWDtzFEBCCIg3jL4DozkR1FK/U5ybAM0yUMM47Afp8pZc4UQlE3x1eT0xm0LcdBOuESEQqzTlNVCKEhJRcmIVZlaHVCjMIQEoiTJBTpkat1oMabHYP3nz65NHjh+fDdLyd5vORZL795MXp9jqzXl3ueF922zxmsHqel8VApbZSymY7TsdDExHFcT9IE7G2224zl1ZnaUJIphijSCoGoCLKOqTx4uKytqoq46bklL1ZQsjE3FRTKcmIkas0AFymeZYZzIacTTRnZi5VjBOVbSICNEmJKQ3EA3BGIgBi1LpUdD9GRRFNZADSqhqoC1+RKSECgkgTEcAsBkMu1lDEiCwzGgAKGUm1hmIKAkwiOE3zxeVuO2xP00GbjZu8GXk643mZ2nK2qcrdXdld7MuGgUHq+XSm7XbYbkZ9VDmLYjtfG6stC5GiISGbmGUkp+xdRObdKWZ/NnqsWRFZZ5vDQN//7p79JvK9pZoSu8EPM/da0b/M1uFHdiUPISqqhiO0RzQN84IEWsFLhi5GWJkRD2SogIzalBIRExEjwrq7xlY9Zecc/Eyyc+LgxJGuBIuzUgYIput4KiPr/QAXYqwkC/bE25ixI3NVO8aV8b7ka0yTGfbhW4qVaK8FVV03xQff41dVfT1DlxHeswsE2pQhnH8kICY6NUTMXqCQBp608NLRnkACLYc9KhiIueYlkGUkhj4YG3sRdA360KFDaz7pTYCejhR6Z9vnsnx5nIcUhJhZAECVbhlN8Q0uv1oteu55SO9RYJAsccnMep8GErMjEIyx4/W5BIzhitf6+Yi+FdkzXPw1GqilKCg74WPrDIjPqHhzA7wrq+s7WkfszKAPilDfq+e519dlUL/+APdkSudv7v9hazuhp5V494hhGNK/LOaB+wn1W93zUa+27xHEfaKK71v51q6LhMhH4fvoSZDiYfBJYmYzhN6nYgYmBROTBknNoB6vdXhQRtiMF9pMbUFrZUz7x4/fevPtN9/+9DzPH75/QGl2dwCcqLV9pt3VRS7FVNt5Os0CoCIg0ojIVOpcVXXYDqaAWXVpACKySJ2b2JCImFNOapE0c2JOLE2btrrMBlibJKDEA5Oof0LKpi2lnDmZgiKklEW1LU0NypCXqaHBZtziMikiyMTQNhmp2RmBMNVZKiMOAc9UTMUQYdiUnHJT0bYg41gyGKScRVW1+c8ys5KTgS7zItqAUZEo8XZTZmbiNJ0WEYHaEJipSAXv0AGJVtWlWWuJkJiEcD6drcnmsj7Y7pKQNThNR6KEmS8uH1CrU6Hblx+b6ICIhDqfE48Y3AUZYKwRCT0i3j+KDjdXTCrmHLq3NM1BdTxjHaQFl0gxzNW1MfHVHQ862uhmn4bg6n/osTdK6Q4czXtzAVIC2SkIcIyyeyhHolWfbav7SRxMU+d7YwkfgN5705iXYUhRRZgJBuZzpO2nZ00Gfn7WGBMYNUTxq9MZ9MAdV9J6Ke8ii+7mEnA3wGkEAgytUM+gPfqbUxlGjP1HRXLsBAjAml4cy97bDnQGzrn3yAGG610LogKcJwMEIjKFPsmFEa99AgC9weOJQDkx+PI5NWAgJhWVrihAZx8pTMLMvGgJi2ITRaYuKYq2aMhb3LfVP3tnINdKyLAnWzOvLWy95AGQ+xyL3w9V9z4Ims56IERAs9Tb1qHqcUdyVAt5vVuXdHuhiL362g2DYHo8cftf9EEE6DDdI3yH7RalNqwB33+mKrq3qB8rj+TU4YWqTzB7+jG4V+p2mBCx3Looqd83BHOFATi3AGbIBGr3Y8OdTIPo/EkvFaHXPQAAKmIsiXlgJjC1ptowF7QLbQrSoDWTKnKCopdXV1/+ga9835e+Apyfvff+xx+/n63KqwMCvvWZx+N+BwTSqrW2zKcGYGQpjT5jac1yzsNYUsLapDUpA+22l0BWl8VOS1MxIQJDYhFTaT6ewEyn0/GMJzNLaWhVSgYAm5clJTLToQzDZuCUVLUdBBCQCZG06fE4jZsNC6EJozEKxH9YLsyJUymckmhrAomQUmraUs5IvhgMQIUIOKWhDEutizYzEy8HGX2DQ4sqqaiJVl+LpoiaU7KtgWZZBCmNG2zS5joZKSU6nxdTtTZjItXWFHZlFLR2PprMQ7m42BKcpqW12uowXOyfvDFM26Yw37wUW3hpOWcEMK3ammq6nwXtdGJ3gfBnCh28B6Ef5AGBAZOPZQKY9bwa86Jm0uFY9NKQEMKtyxBQQZw7Dswp5u3lLvVU0NC/Y8hOfJEMGQNCMPIrIjYIqcnrcdKpmHh0O9PlhhMr8+MxzlVGthpbxolz0IVM3aHBIzUaQHeEvH8PvUbCcGwAd1CINxMhy+F/VAZqyN76Bn9jFDSufxspiNcE2FuXBmagxKwiCIDIomLe7SWKK6BgoB5/wSw8jhDRQPzWhiFBB4rQ+9694YpAxgAmAOQbDFFMRQ2CcXqtEe2AXpnR55IR4tX9SZDmW0b8gwdgXhW/GKQFQp/Z9V/+Oh67zcydNAE5eHnsZlKe69WH19BMEUh9LMMpHPSb4ZwYrY9Nv6oeShUJTYwJEyCYdKVjTFD3gm4ty1ZNnNOgHWAAuPLJokPSVQG984pBnWHsBY3cAhAzX2uSguBo0NBMwACRViO3Hoc7a4Q9+0P/EbS+ZetFZTwA4Mg9HuI1npuKMjGgdZLRrxgamKKhS7fQ1gmgqA8RESgR58Ryni0tnAA2gyK0ebFWJ3kuchA9768ePXz0MHN6/uxlm5ab22ubTwD25OnDzVhSyQxwOh60LgyGJNIUgMwaI5rZ6XZKhUpJ06mJ1lyKMaosrLQdRlBc5hkN21wpiYqaGHESXRJn44QMZpaJyjCA4bxUI293UtW6HJbEmYiQsLVqREhEjE1VqiXOy7RYQmImTkDcmjXVZizNVSitLTWN2YsoQmAveEMCD4y41IoE2sBMl9pyThxUAAzDBqypARqmZMQ8z7MaGVgpWZqJCZuhokATwZwIADHl+e48DpkY52Waq8CAw3YUaSot1dN+KIh0PEs1Ok2HnDfD9uLhG29dI5yvP26tEiZE5ZSB3IZTzULkqWhRxzNEoYrWNWeA6B5/TmbG5IqGsbNZdLQ8AcQTYuC9sEAykRLi1CB2QB0mLU7vIjgPo2YqiuH1HIbApt2yJ4pUxL5oL8zMOwXQRyENfEyyR+vwLtXwZAbzrRi6tsE6Z9PhGvk0LHY6J9pv5EDVOg587QRZlE2AiMToVLz143lfd7hGqqsaI+HGaJVgb5T7gbSO2MzEZfl+DD1aggERqWoEGZ+Q6k0csOh3exaEvpnczJA4eDB1CzYy9Mk9UjX3b3d6FdREgf3HAqqakjETGa3pDRDDMK5/dsAwQAseLLI1AEDohOPZ8M21PX0bEGO/fQCA1hp2sa/fKFOXNQKBDzl3+AJm0NPzPdUN2KXDiP/GXeg0CSTE1YKvj5S7ykkNOLbPrSuaoTMsGkkG1hSHgfR77yGq6HD680uhcVJIrS8a7aRhPyDqAuWokiyM/3rpYQCrLWnE95Wteo1hDIV2fx/QuyyG5mWa+aZSiOGQ+zowumY9fRCEERIAMMci05womTJYA1/8zYiZqClMqnNbjpdvPP7S93/ls1/40m4c33vvO88/fo6yjIYPnj7cbbZjztN8rqdTm6dMwIxm7POZzeYqVkrZXm6GksyazTKMIxLX83yY7sowpkETpwVr0zoMo2qrZpwAoF1eXajIFje1tel8TkOiBG2uOVOhzJlbk2WZC6emTRRTYtcO5qFwSn45WpN5mTOwiOo8JQY0OJ21Tg0B3DYnZ05M0iqYNpXWTK0xJ6LkDrRLlVJyzuk8TYisAgCo0na7y9ZmMwCyXLKCMjGpQVOpAiKgxpRSzjJLVtYEeUjL1NqyjJuUEShjbYmpIVqbZ2TSuhzOJzynsrvYbr3eOR1ffgKbPVLJw4M5H2S+sUxivVDsyuMQypmteAXA3HGjExXai9b+125VE3EkpC9OIJiBqURg7KO63Y0AQ8fg7V4lEYnlSOarL+x+j6MBrRIbUUxd+ILx5qGv54Awo0A0Wj+I54XQkoRpBYaNQyyuAlFxe0MLVxkAxCDNARBAY82CH3IEC6tFp46tB75o8jm0Y4zrBjGtGiIKMOjOAk47RNRylk3DPdszCjMychPx8I09d/oRxd5EXa15xXybOwQfppY4ObW1YktAF2qv7WOH/Bh76j0yGYg7HfnyZFVC0r44hCl1/boFiRfykCDMO0ftVEoYLfSo72bUtA42v1bCASBwp4G6OTbE6BkBJ14l/p3AcLfR7prqnyTAMN0nyKDFLTgN5+SZPQMQUfRAEZPv+IKed9aSDrCj8u557TXA6qvmz74FmiDrzxiRL2RYq8TV4cdciY7WqaPgOvtCmn5nqBvGOTmrK+4BsDhLEdvXRoc/rIY+5d2zaM8TcUXCfpQwHNzxXgYaj1UfjUFyMZt/fD/iGsMKxmQlY6pgqKANxRh8wqACS7m8ePr29739mc8DjC9evDqeT+dX108ud0/efLjZbtDgcDqAiUgzEUVmiArTVI6n82Y7mmrKzGANGBDmpZXCpYytVQM7Hc8pl5RLawshYE6kCyEy50wpj5t5amiy3W5ba6dFShlAobbqGvOSS2Zmy6pGZDmX0+mkopwSczKAJosZiJmAZiQgFtXprLJ4/Z4RG6eEhJRyMjSUZV6aWasVrREiUSZCMchUABcFKONGWgOGuc5qcp7mRJhEylBULZcy6TRsNqZW25RSUrfHFUucEo5Y5nwJUrWezghUNhkzIIKItLkaAjEh2HK6y+PucjMy7tu5HW9vRa1WJdragNM85yGDNQAkDpl0nCLnUrgTPSsnEYhHMeZIu3DZOR9exRdqYUBL/pXum7PaIHZzQA2xHLrzrgb9GJWIdQDlIzNhmI7d1AbiYGKIuDv2c8DWtyFyYPV1/ibKbFQwMkQiCXgM0EfiPVirOO8cfU7/MWJmohSFTQdXoHFmo4o3aO5C4ZWTErF6wnDmq08ZxYCpX/J4h4EddcXRaorqjmSEvlrVgtxw6WNUDBbS2qClYuWtoQTZFZctvP6DOOnRw8FtB3wedgyMuheBM3NAgIpIQOZ9cF99KBK3Kkoi3+C2jvma9cIO+uMR423g7nW2MhfQg5ythnlhuYbROfCIFI0+dV9Y11hGb+W1stIcv/cE3QNrJHQEn8YNWrHnJ0he65EaMBJ6r2OVUa4N4UgknXMM1iQ6MAFMfGui903jTq9iYQPwbZbuoADROItHgACNDCE6LQYubIgnzhsdfn2IyHVtEIcW1oK60z+h4aGQx6jFclGEMPXpOtng/izuZC+TQge0mhlFpsS4PdoKlQ0DW1WmaqbzZAhEVlvL4/bNz37u6vGnpknmu4++++1vFZOnj64eXe2326G1eprPMi1oBipgIFUqkJqWkhBhv9uOwyYxzKfTLAIUM0HLVEXaZhyX1loVNYS5AsqQUpvrpuQybkBAatPa1Ey1iVjK5cF2LyKn47k1M1NRNdCpzQPl2gQRconSLdZuoBIR5eLVoMiiyyyJEiOMlEuqSwOWcSxqsLTF82NOySoAKCc2jQffTKdlQqJcxirNzMZxgwxj2qVhbk28H5A41VoJ2N1KyjAutY6ltKWpCac8352MNSEzQUu5aSOBYbNtdZa2cCJt1lQ240ZUz7c3abCchzeePrm9O93c3M7zAZDK5VOqd4RqcgLzE2tm4qnf+Vx/SBHWAUL1DO8n0uO7qvT+lKKsxSUauN8cGESMj8oYSVuISrW3HMHAQMBtoR0ORZM55BGIZKAx9YmoaIkCCYFV4oxIGCa4SuCrAl6HRbauHguoHopn6xwpOO5eRYuBvXpkgrBiBlgVsR2hRblAPvsGPc4ZQVAbGPtWg8V97Tjf/4xoUULnmwwI+ugDrOgtwrcBuoVwlOge5rrWyMMccSyYQqDeuzSIWgw7M2Y9wnZ+aEWyUZSYm696zgiKKQzd/GaSFxNOvpv1dSnYGz9kIP4cBTqPDhKiq5MwEkUvqyjAxz2NEdkIuyuUeemo4vE+zCtD1AgRkOm1trOnE0cXTOTXWHpvZy0PLB5wSKbgqN6vdHQqXCnkyw+9hmOPp721BdBBS2+6er7REDWtAdUMmLFvOY0D0ylF8L4QBP72FThBwaj6GDPHY+T5whCRjNQPjzubEmNfmxzwKIYMPc9ovMPw9gAU8bEavwkg0p8kf7h7oen/FUgNCU2ZEBESyYYNVQkJLYsqE8g8ba4eXDx8cHF1VZt+9L33z/PNBtpmKPttLmQgUqfzfDwycymJvOZvCqLDmBOzk6KZubV5mpf5PJVhyClxziaSmJpUMMs5i4nf7EkqgZkyCHhQMNWc8zhualMFUJVaVcAAtZQxD7wsjaWpqkhV0aXVIadhHMa8adrO53koYxk2qm1emiq0qeEWLy7LoS2qtTUlsNbERJZaExNnZkqWIt+DGRCgq/tNpEHKnFJWlXmZy3ZYpJ2Xqtq0eeJdKOFm3KiCLMqJmLhKUzBO2ZAUqgkqIaeM2aBhk8pzI2b2xgLn8zxPp0PK4ziO5+lEAvtxTLz1e7fkXKukYQMqCg0hi0irhpbMyfSVu7z3GYhg13+PMTq6HlSJBdEUMkQyBSQUiQ6WLxMGVJ+9BANEBlXnZiwIkygITI1cka/gJi4gqOTyTSVKFiYEGKNe6zQVxiyUmhCygqq4aZUvbwPQbkNPgNAVLP5Iax+oQTNz50QFIHWxhkF0Uw0MQFSYOPgHgNgIg5G6mngfxY2XlYh8XVys3AB1e10RJY7JZDUgAOROxxgA+Opm6CnKB249S3WCuV88r04iBxj4LQi8axDIsotBOiTx6Eu2NhgAPMd7sA/jHQ9onajQtcoxI0JCFMUoE8hjfKi8rAuFg2yPyTboyTPcmy3GkiJcO+T3Tx0DB/537qEJqOKukoFzewpH9I0L0AlsvzSBrSFyOQIhiTTsuTMoIh+8SwRNU3+/njDdwhaYfa2G+jC37xiJ5OykDvVnp+v6I75CT8yRGmLpaOxm9LuooBDmSkh9UiPQt38zvQZMPMtZlL1mZuJXP7T8YW1qYbxiEEbh1q0tAMAUkcG8DlZi53NijzNRdoGCJ0Ev4/D+smD8rT8yJgyWUIlE9KS2pQRNFk44bPdP3/3sbjs+f/6xTHdvPnkwYmrTUZZ2kjpY1VZzQVRVaX4xKCcsZAbaVMWsaquzmYjq/nLLmDgVIlrabKKgUsZyuXkwyXlaZjUbhoLACTClnFJOhVQ9BKQmZxUdtxe5tNaYmIecUmICujvO03nKKZUy1Ca1KaOc6tHAmIk5qUoDBbPz8XR3e068LySn8zS1udpSOGyRci6+5dGCwFVOCTL6mrkqej6dOaUyXhgqKOXEKng6He8OB1CbTycANNNc+JAy8zBwKZtUhmGRxuPQpvNSmzZRtWqScm7EVWopI5BJPSOya4+GnJdZW53RhIFB2vn0ivL44NGloh1vbs+HI5eRbAMwtaZmKE2aAsWRACfE4pEzgK7KIBcwOixSJQB1ssBLBjdq78Nf3ql1+1qORmnHdwaE3QLORfAGQH03RvAwvjXF/FtVgQmJg5T3bhs5UeWw2cyhMYaTmoBB9/GKkGQ9lAfR4fWN6996c8NWv/yO132Q9bU/U2YOn83Axq7ZI/dzZvfAsF7oUB/ZWQPd2jDQ0MxQV1U68YwhVAf0eToMJU+snYlDGRHd4zl6nNAg5m2t0+D+JwdvRthbsr0VAn2MlkAltDprtMa+rrybAkEHq51vDrzZhbIrdo5r7gGHPCR2u564thiVSrRBXgsx8ROwr/dZM3VvG8SMbd+PAtiNwK1z9KFdjnAOMQFOiO4K3tkbE+0EFKR4tGLHZU+V2mvbnlP62gCIXfMhWuqPlHV03/+5emNHdnBVsRGsgn6E/iJwf3FDR+FDYRhpPG4sdrLGw7T21oXnVw3/234wVBv2pBKG0a8roH00Rp1tFHdx6WwQ9ra9vzkFAFRDJhL3LtdNTmgKoqIT52TEUiuXsVZ4df0KFT/77rv7bX75/vdQhUpSsfk8Y8K6tISUUvKlaMtcExMaLVOrqpeXYxMhtIv9BgCHoaiYmXD2qR9kJoGWUxITRsjIRLTfbZlLSmwI0kTZhWg4Druq7Xw6+VZnE6mzCSiCDSVvyljG8XQ8397elszbzTCda21tsxnm+Si6NDEdRk1AmWqVpS6KUArzQMQsVQyMEKQpMzdZwEhnGbYjU5Tzm/2GgU0tj9kaHe5uDofj3c3N+XisVVuV0/FUcho3I2cGQTJ68603txdmoMjJMIGZMQKaNDFVSnkcR1mWBRXVcmICsKZjzmamy9zmGbGUoSxNrVWz9uByhyJ3t4dlOoHxkNm7tS0kfSTakHvkJeygrk+gArqxGsbuFz+zgD3o+G+CcA+uOdYFBIUSM0QOHgWJ4T4oe5npv1FVc7Y9hq0IDXwpd2dL2e1pTcEY0PrMAPZWNbjSARECzkfVD+Rd5tiDDETWZSYx22xw/0FieVnvRnRU6onNEa7z0f4R4ht9wMIvTnT8okvS/QzQYhK1T0sE5tX4UGqARkRIIGKIjrgpXIZisU0/m2gARsCK4aBnYPfewBh/Ev91n9d7KzEMLs1NlJ3z6ItEsNNpHciuoN7WVm9MmQUd2DOy/2jqSFzVqIdlCJbEwJeW9gCorw1w+VtdfxZEeeE153pRu7ZFV+mmrZ/N86B/KjcFsr6RAu/97yxoPQUkSh5C7y2UAFy6Hao4f8rJIAQ/uE4dG8SumCiSfTWPAflaruCK0CxIGI/TzlTE2EovX9bP4b79oaczAKNOipnFds/wf3j9PhGQAIqpu3kggpkyc0g/fdOxhVQ2Tq8jBSKnXdX7G4AAsSQKosnfnx+icMYTzQlKQkJUEUqpCVaBJ+985urRozrdjGN6+uk3t1Sf/eG30PBivyUyBGQmUUlUkGxZFmkVCUsqdZkJE2S42u7MVKvmIS9VwMXboilnJC45E0KTdpoOJZfdZlPGjIZl2DExM6tBbW2apvN5nqfl4mKvCZZ5muclZ0LAeZoxsSGY2HbYEtj5dDKwcUgMkFICbKZYW53rOSdAxFJyKQQM9dycXM0lo9bpdK7TlBKNY84p1VoFwKQRpFa1WVNAQ9nstikNItpU7w63L168+vC7300JmQcFTqXsOS2n4/XLl2a6KRsCfpWwzhfjdpc3SAxAMJZxaZNCTaX4HZ9UzMRERDQTjSnX2kgBgTKSgtXpLAbsHV+wy8t9Nfv4o2ewoNaZoIDpsqhCMjNVTewEVi9m3VQMcFVK+BMF7kvsSKZ7GlKXSHh8EKnoE7/mFs1O+GqirH6OQF9Dvh57Y4eL+57fQxCHq2aICsDM3XknIG7QJ8F4mBL4/yEgmmF45sZETiQbCHrXM4SfAz/gfUQgPgxiNzBY2bGe/xDAzTXIYDVCi0qDCL1EUVVAEpX4m0hEvV5fq3sAU6CEPgAhqoSMCMzkFmqe2CiMu1HVTJWJw9oIbFUwWl9IBfcxNAh4T8xrGYG9CcG08vtoXpgFhx7XwMKGmn2p6mst5eAscL31sKJXJCRxAayutzIaEz50aEG3WEf0oSSzTj4HGFbzn25RtLFIVW1xPan3kZ2RIfTOrjtjeJuz+Q+Koq0TR/5d0YfABPe0fJQwrhR2AXJI+EGBXB1KLlyNJGfB4L/m8WRr+ekPk+c5MzRPDNFFeH3OLf6TOr7o+c3vXHBHhu49hwiGhBqlNwN5YiZTZS6GsnoZJeamGiU5rI0yiBly5xJjz4zdXyTrMpDYC+ZXigAUGQnRGCn5gB/QuFWhy0eP9hefPr765OrJ8Objq+X48ccvXprJ7urywe7itEwqi7VmgHU5uYlHLnnIbKrDUOrcxnFMidsiQ8lS23E6X17uPAQDwuATWFKZeDq2q4vt5YPLvB2neTkcpjYvmVMeBiLcXT4cN1rnZZ4Op8N5Wk7MtN1uWq1nbTLPzrSe5JhLcfC12W21VakCJjmneb5DBcKcUaFWm86wX0y11cVMhyGDKDpcZVK1WiXmjIgIUbTWJgo2jiMALbVxTjfXrz7+5Nnd7W3KZdjtEFIBrMvUmoz73VSnhKzSkOnl8xfnZdqcp2G3uXpwmYZkTKQ5DZiIDbEtjROaYRWotVIeznMbUjaUzAaA07IAgC6igCUNVZqBXGzKab+/e3GHgFw2atPSugonpoCtmz85Uo4+VH+WFDqRYr4fBoyIfRwJvMcSvmYsXuCvzT5wJUf4Bq1zm69TrxGZLRRHQXEgIhoTu2sYRXSJSUGMIBpYvss30YFMuBE4YPSBr+6QTMzanBlwKeoKMrGDzv52LCZjodfVZl0TyG5Fh47NENFVNWYWNvvMKj5Z6zO3wU3dW7DguqmRg9UnZQvHtohoUf33LoHnpJjQdfRsgMjEPdp6NiSAbuTqAkwmcI0QCK7u/BHKSULA2/ODUg9I/kYjvK/Q1x0z/bcG3frJQsxK3Q8cwUcuenfBTEWZ2WIjQySALiDrZOD6OHgZGCty+5LKePzAL2mPZ93YzyLHmnM0GnVqxDEiiH5V7EjxHn4KOqazV7iymAgmpmjIGF33ns4jnfWAf98NsD6Mov5aQABMHMxMZCfoZbatdZPLH/zGinUcBnhfAUAXP732Q+NkRg4hUPOyN+Z7EUQFEVTCvKk3oW3tJHleNRAy8CfX8VpQWmCIAXHUFPu2GADLiRGNGXLeINbLh5cp6bAvj/fl9OLZ6dV1HnIaRiI816M1YSYTkFpVNJeiWqGJA6g0jJSRwNppamBqOk/T5cWefXaKeJpOKeXlPI3bTUnlwYPHxGle9NXty9vz7fXLa6k1c97u9lcPHu+xGEiVBszzMhEyotWlqvr+eqhN53lmQk5cSqm1TdPMhIfjGRl2u935dEZKOSVpskwCi1gDHmgYS04Jhw0Yzoe23Wyn5YBlELNpOoMBklAJiSIoILEBz/OktT77+NnHzz4kyhdXj4iGeZqW0y2A3l4/R7Zlmvebi1wSqVVZrj++a4+qyN5a21/tUs5lO+KiImpm0mphVmQQU2MVqVXa3MqYEhXRRsxtkZx4ms6N6+bigQgq6OV+K5PAAlpTq2YATS15DwljXMWHQ7ibmRl0PbVbq6uxDwqtdK/XpATSlsTZMSAr9+oWDMz1bEG1m4VZNPT/dQuyTgQZEzv9pOaziujrKlW6jFLNiKDnIX9OO7MRaNiDS98k2CUNHha9S6xqqNHM9fgbEj71AGEKSNZbYp4P1PWq/ZOhqhChh9M+0wUiyilsmxH6jFWvkyJKq28FiDAioZuEe87ZAAzYUyxgEyFyFaIwJY9/GmGgz6U6pPVMooIO5iEIhKDsDAyMEAlIYyu6edcBfDg5KOiI5tBpGw8BEcSgD5cZxByG34II9KAmCNjHtrE7f0TRgr1r71kEbKXCVgLGO+w+WmUR0KJBEn5BTBQrhiIaerDCsAh1kcLr2DqIR9OOrD2/qWqCKEQ1+syIatJFohKLoymMZKMxoAJgnnfWiQQLIWrEfSC4V5RCIGqIh/W11kLUQ9ZhU288RI7z0hXvKwyLGgcBtXf9/WnzXhAihAO4gWOq7glirwGu3pUyADewwGAvdW2G9IQbvywqG4znKUpPa+eUNgQ6pPbo4dXp5qN2uE005FKIUVXromyAiWZrTRoyS1sMXH8LAng6HoZxA2qUmEzR+OrhVc6F0JRgOp6GMU/nEyOPw8jEgHCc5levnr94+cntze3peCKmUoZxd/Xy1fFyv3/85A3UBgiJkVKaprOZMCYkHMdNvTvkzMxkoG1pKefjaULTcb8xs3k++UM8L3ValhlxOs2bpQ37sWzGcbN7/sn30JZxV6DqPFNdKpeMhIQwnac8lMw5oTInNDLRzPn9j95/9eIjWeaLBw858fUnnxDIspyOx2uF+XKzvdhurcr5NM2nOWcihnk5EdO43bcGRIDGlIfT4cQs7JvcDTiN1abpvMx1IUDLw+AUJTEnI+YtD+dq8+lglC8uLkW07jfnA8zTXcN0nkQNDSnE73FbqWPDQIQQSnnoT2ZM5vuz6St30J2NVYlTTKUSmah0TXenMswMev/QpfdRzas/8EgGomge+nsTObTUftJVLTlzIWZkQF2v3c91wNLVmBMwwu46qemwmtw83LSzWNrBrvaoGZcBQbuhLq5SUg+07Ad97QGAudhfDRCIeT0vHiI0hkLNPUG90Dbz3yqo+SSKqsT2PTDAXpe5dCWk4d7WRV93bL0m6L6oBraOgQaBC7bewb47fnUz80AeoT1GoAMBxuyA4Tr0FMZIvXhcvxHjSkTc8yaKI10kL4YMAMQSo3plsnLuEJ3GiMCA65uJktOfvx6OzG29+5Paex3+szzPA3AULmaKlrBPU2EMmESuQbAEzjPGpwd1cpPwnrhTAwQRWTl3QFgbs17Sah8mVJEwugNXDZjF/pWumetZq7cIoFcOiIDoPW6/DD4XR/cFXW+nWLD5wYB2crGnXBGNeQoKDLMWUNA7B2gOxACiv7ZeydDmQXRgCAFMDFNgLhc6EBChqFRrc9nu0SAXO918IucTGKWSmVharQBgQmU83h25JFEBkTKUnBMBNmlMCdhUVQyRcDpM424Ycibiw/nICJAg5by7vCTky4cXw2b8znvvf/DBs/PhXJf5eJ45DfvdBjkdb+/qVFudzGS/3Q/bcdxfyDKPCNJqKUOV5TzP42ZEwCbzbrc3tXlp6IvDIFNCYgbkOi8IpqJi03S8XaaFNqXNKsZIRc5HTQgow2ZwdOOarc1uD2ZMiUg3u8ulNgA7He5Ot7ey4O7i0eXFLg/jTf0egMzT3W4//tWf/+s/9dM/kQFfvrz7xjf+4Ff/2a9891t/dPPy5eF80ovF6nI3lEdvvJkIjTGVglYJYZmXhMiYcko2jqaqptNcFWAYR64ASK3qfrszmGqrhjyd7vbDMBc5A4jwbO28VFE1jsVbvqTWg4Rad9LqABBD9OAHxzebExKrNgIUvTc07pYABmjuv6+gCQkxJrv6XhHsUzCupHQxHKAEUHWX+cBEfho6KAPuInMCx/Jeo3cCtcOirkoMjIjrf3XCwGL01P0aPOP5EeOgajFIHx9uW+saDHLDJGgLZjZV7F0NpixOVfc8CXD/vS4ATMy4VvCdgo3mMKABELGauOiFXFYAiN0q3Azs3mEiDjYYIBGAcSIAi0Xf1tuUIZcKdawDTR+CW8kBAAjXTKepLTzE+nWNCO2rCsDAV72YKjMBomrwOtD1rN51V4KeObGJQOcg6P4O93nf+zLF18w5OFYzICZkjKk9M3MtuAOHez79ftKtG2O4NlMjAjqX5mFMNDGlkAlHCQXrY4fgTkSIiCItmCFVCLc87SpbAlTfJ48UxZGHUq8DwZwQQKMYwL3PNl7gAFlQhbhuDosKLAbK+o+KQb5ewTjht1YMTGYmqnBfclqcFUQNa3iKk+DdGWS/KIQk1h/GqCfdAhcAEInNfTGaCCAwUEI1QWptOeQ6WM3zcUo6J9XNbpu5mGoibtPMKZ1OR0DDamyQ8jCkjAjWrKkBWC6lLXWuM5ywDINKO94eeEgAlgYexj2IQkpI/OrFq08+ef7q+q7VpsxpLNmatmZE8zRzZgF79vGz27vbi93VW2+/MY5vpjI0MSJcVJdZck4IpFJzyipCmJZ5AbRShjSUVmeoMGxGyDDPEyfC2kwWokYAOVOrS1sqM5kqCPr8MIARcZVWxqxiVRoBHQ8HHsc6zS+fvzzdHC73D/Iwno8TIZvWU7352hfe/Bu/8B/83F/685dl66fup7/+wx/+3E/+43/0S7/4i7/yO//qD89zNZXz4fqWSVq9fPAQUmpNmraS8zxNTFZSHpi0tPP53NBQJIkmJrTcZDkcD4RpqdU3JiVOu5Ffak2JmtBcmyhgQnOTJA94QewiwOqhgBKxg2Ja0/8iDAP84GJfneRSlrDlgo6fjGKm18GPy4p6oAuiyP+HGEIiEWEg4HtgqrH1N8KdxT/VYyKFiSOgi5mDqY7aBToAtGD2IxpahKXoKKChhl2xL9d1ozEnfUlNfQeUtzHRA6djPO++um2kAzpCab593PuqYSXsQdA3IXZa+L6Lax1Qo2s7o5kMMY2E3hRFcTZC+sCpgn966EUPJvQkK6qrH6T6PYIItt0XSBl9/hndDNWgPxKvQdUoFDBgeOBjL0s8njpZbArmS9+cGolPHQCcEABVxe9NKIk9S7rndq9RECi6j1GHoJM3RIhIroO33rDxR0bNwvSsk0pw79lmQH1rKXjJpP1TWArtC/XaFgB9RtEZLgtjE3TlLBLGPp1g4B3qE2hYnQYedyS1cpLQczAgoGu60B83f46ip+zBvWf2eCW07g1tUcFFTuilei+P+nSc3xnvswBSN5aw1buqV1rxMKyFAAQtZYSs1rxh72eKkFWWRL3kBwIwMmvziR49GDMQNNUWm7lNTCsS5ZzqJGp1GDMQllwYsLUmYtJaypwYlrnWWtW0lAwolAqIoEHZlkePH5soIRPozc1LrZUyH483N7e3qkaZW4OhjNYaI0LitrTd7kJFXl2/kjYtU33r02+N2/F8OuWUpUprYk3aMu2utma4SEXCQgNzWWpDs5Rya9JqS2UAUpQTJVIQ1WagxJAK2SxgQIlsUalVQnJiqiag1ioZDfutkR0Oh8P5mLabIQ+AuNjy8SfPD8eXX/nCp/73/8f/9E/9iT+ZgUhBQRFgk9P3vfvpz/yNv/Zv/bEf/h9/8df+4T/+5fe//UFb5oO9JKQypLGMu+12rmgGSElEFt9oyYVz1SZg1s4Tc0EmQKxamTPnPNWZhEhhGLb7MdeFCKjW1hrhmExRzdBXWnSG0WO/9CkdBXPf/XiGY/DSuosyGDGsQtL1eUJng9hsNU7oHeCAMBQ4zmGg9tpakYnJ9T4unQx8amv07VDFR60won+EU+e+CXppC50CtwhnDIhE5Cuq0BshkZw8HWt/j8rErpmgqF2C43BJCCKYWDAyniwh+iNE7FopUO2iDyMiEenoz8AHncA5MQjvBwvxEfTeKsRUMDCTgpH0EaDoQTuVg71ZbCawSrR8dFbV97WFtjXQq0Vk8yIv7hd4b8S7zvcUMPTs1e+kU3NGSD6ZJz289pQdQQnXaQ/0MEuOrb0RGinF1ugL/jE8k2IfUCCm/jeoakyEzKtSJpaTmUHMTzg6IHc7ATB/d6s1oDel/GOluGEaBaODX1p3Ulun0hgjn/fEDR2cu/OfoyRwL5Ho8fqb1xV6QNdXd5YrJhmiag6GJ8QBpmYco2tgFo4Ycam6vXU/SeuN8Qc47licgLgGUY5E19761osox/o3xMH1x93XIRigkcUdk15mEqLKkOHychwHrPMpAZKLNlDBxFRr7BgAqWKgaZsASao21ZSYkdsiHXEqgC5LRQSixIl3u70AmLW7u+v5+qimN9fPjfLNJy+X6fju973z6K2nmOl4ri8/ur67O+fdHgGHYTOfz9M8q54UnnMpbz55utnujoc7U1iayNx240YVj+dz6pvzWm1tkd12KLv98XCsrTHiOG4HJTHAkiGTnYUHllp1aYmTgSKzqTKykjJmI9AG2oQJUsrH4/H585cGNG6Hm5fXzWyZD89vPvyJP/m1v/t3fuHH/8SfzJDCxVfBiHzMisfhR374S59/5+kX3n3r//MPf/nXf/1fvLy+mdtHzeRif5m4lDw0UyaYlkkdFyOUlJlTrUvTOnAiJEIhgGU5UcqZaDqeaYPMebctVQaYtnW+FTUERuwrcbFDOwALuc6K0B1+eKvLh7ZcISSMuW9ritoSu+tD8O6AEf37mVMJYQYTh3KB4stMjbEDYQI1IUALgqcTORQ+Ma5Bws4hgOEaSPxZplXIjz0sWXhChG+L19bEnT/qypYQhQRpQ66w6FlBLboiRBbcFAQPHJgZzBQ4UaeBDFSZyGcvoc+IxXxXj0TIFOeg4zfTFREHGaMxhQTsoiMwbdYJi9c+6rp4qisSnYUIU53I3PdWSIFk+8ZmcCbfLLqW1uuFIJoitqkJGrjVvhvJQYB26uR0AAKH8a/nf4yG4n2VAQE/PGDgyvd7Xopg6zUfURD6iKCAyGoC6ztV1xpZtDhexxz9GcWoKgAJEr7+rnwW3KtRRojVnmsCWy8x9PIjhBMU8lInZtAUNarNuDv33wv9oLzGN90XO7A+EQD+mFGUSf5ExChb7GvB+EiumNL+QIMTurQi/KiX/SJ0+m+tTuJ14sl17qnnCx/QdBUQqOvOSC1zQlHQadxd7TYJ25JUCBkF21IbCKLNtSby1YgkCgq2zGpWl7kxG2NZWkUEZmIiEV1q3W43iLDZbXaXm1xono/HV7dQK4KALCJ1uj18evvo3/mFv/6nf+ZP7S/203z+gz/43u9967u/940/uJvb+XSalpo324f7y5xQzvPtq5uS+OLyKg1DhcqgCbGMw9ymZWkmJq1xGmo9GWJrfHt7qK1O52mLSCNoayCqKoAACc2Mc9Yz1aXlgRVwqcoJmAmabMbN7eGAZrv93kzqXJsZIs3Tcry7VdCPPnn/B7/6/X/r7/7Hf+bHfrRA8naQBxVSMEoAao2A9MGTy7/yc3/2j33ly//tP/rlX/qlX//m733j+Op5Jrgh2YwPeMhmkFOpyxnQyQoCbXU5I/FxPhfYKCExE4pJY7AhZ6ltponJHlztjnIAwzorYiZksAbUz7p1pxoiBIZV2gBgDrWCOVEzFTFggddG4LGPGEGHys4VdEcVQHemNCMgdS1zACFHcNDRb9TeSBQh2/XgDmt7t68zoq892y780V6Cd4gJ64cAAEOxmCYFQxfPBQiPzxHlBWhAO1hPpdfEpp1bJUC2IFgNDGJJmRO0XkMTgqGoBHmtsY0DXrPl6RuBe7IFQyQiUFACFF2jIK63Ipo0FBNxAYBVMQbnAF2BZEoQbnroA8zcuX0kX8+JwVajGQhoDOJZ15L2qSPsScD/1J2wve1vzu106j8iOXRkG/RdTDt38NzZqBUsRBnUB1p7CocOaNd86h8dENz3ibu7pz9l2pk6gOhI+0+L2iLujQ8ZYvK+iDsjBpKJhyGW20XwDOvp/qhBVDTxhK0FjTu7GXAsuVR3UDKLFQRdtoUdj8QTHIUjRE/GoPNGFtQ/9gGF7ni6vhODLi8LmBO9hBCBxT2LnAzW1ZzxlqOMiHrZ4poKAIKbEQWl5c46ypyBLKkWsk2izX6T0ObDCY3GXWHOsrRUUFpfa6xggLUt2+1GTbU1BEzEUpv78LuwFxD3l5tahYiHbV5Ub56/vH11Nxbebcbnz54fTqeR4Wtf+NRf/V/8ez/2E18fUlJQAH33U29+7Yuf++fvfua3fv+9b3/7D+E4nafpdDiOZYTWzh9/fHN49dbbbz99+IRTnpc6lnFZFqTMHSaez8fNditNERlVwSiVQnk4nI6qknJSMEyYGEvmxDybDaVwJkXLqq1VgoTATaWU0uYKBK3afJ7Z6Hi4OZ1Od7eHaqevfe3z/8nf+5t/4es/uk2JQF3UDQRGBE0VNLnJOZEqpERf+OLbv7D/qcf7/T8s/Nv/4neef/h+e+MpPUwj7NI4Js6MJqZMOE0zgSWiuVX1oZmUfEAEVNGMEYlsmU9MabspCwMStqpqFPKFAEwdnUFADu04JcBX/K1/HXkljfENPjqIYEpuRgJRu7pGwbuG/h+KJNrUuurfMXOHn+akK0YlCmaxkxUdn0VccuzVy190TOuoTNbdRwhgoCr322AgoCchynoy7iFol70QECEhx6FRQyQvOGLFOaCqMvWIZMDkc7l+nlwLbsRs2jmOoGnuYZ7P+gKA+mbZTgRDDFGB1+rrfFOvs/xcxiaA9VcH2sopR33fg0SvzoCZFdQro05/YZArAGHUE+0QgygCOrvQuyYef4ijYw8AqsKJAcBnZLBb/Aam8NgK3Ra09+RX2qf/hFiL61YrAL1I8h6AucFGHySDuDhgYOwfgkxVRIgRIDYWaHCH9y0o6NUoJzLR5KUsRcWInnx4BUTkSARWPSd2wgTCayfOgHZVz6qy6tsEPGEDejco4k5oYLuvhHnshZjLDk60P62d3HNs42o8L2PBQAG5czmx4DOOK6y1W3SserF4j2j84kbu8G92PaoTeP6OHRqx2/qigVRGezSUstulqxGloVrJCRSsqRGYABEVA23CKanKdrthwjrLsixDGTiBNgj/lgpGWnIhpkw4lo3UJkxWbbff6HR88ezVs2fPdrvdj/+FH/sbf+1nPvf5LyeI3K5AOcE77z7ZP9y9+Zk3fuVfXLz3rQ+/+Y0/yJx0qQBwmpZpXjbDq4vd5eXFQwSypudpsVbH3cBId+fbXDIzjputiZpJojTk/bLMS5sQkmkjIxBhBqwqVVMZUkkGTdW0acmZmRBoqZVzGTebhHR7Ot6+uhlp++r44cvnzzbj/s/8mR//3/yv//aPfPULBHC6PdydruepbovBWLZlP46ZKBuoKJEqQFFoQPrk6cOf/Zmv86YMZfMbv/Wbh+sbhmQPTA83Vw8fplxARLQyl3k6AiX0tQrQEqPvoEJAqRUIUipuHceAu+0209KaqrjLlId9f/ZIUeIPEMN2MVTeAIairpEzRCRjD+4eTSjE6X3Gp59uA1SI4pUQxR1qiU1CimBmDGTgCyKBORvIGifNUERyysGjW8eWHdt0JhTW7SwxqAPoIYmhx/FgdPzorYpwM1NyRsF7jxje0YhgAFojcN8PZyE6UA2XMIQwwfeUaBBqdDOIKWffShhkgBoCaGL2fANe61uffoIQq1i4k/WtYY7FQyYfU8p9fB/7OAWYgZo4GaJm3SnT0aWzY766y32cNBwvFKwPc4F1TrlfYevcdBj7qCFR9jlTMADjxL0jcZ9m7/NPaNRXLqQn+hVwoN9XEk/VvXVsXZjgLxM62s4v9R+hwWAhIkb4BYpVyY4w/Lr6use4UFEyYepEXMyPYVQEhoDIKNri8XIxA+C66sE0XCa8NiFCjeSDPbhaR9bxAaISs17lxSOL0YUx840HoF0+4PQ8xphYWCOhRWMfu2jM/Al0R+kwEgF47SerAQj05ThrcepULkKfJ/M3iBwJD6Nf4Q6zRoZGYoZmyfCN7Thux5rz0aAZJCIhMoRWBYnJwNSkKpETjrTU2ppQYkqooimXmIBhZMChjGRQSlLT25sjlXx++ZJzefbB+9Myff6Ln/u5n//zf+kv/uTj/ZUCNABszUW1TnU9uSg//gNvXiT7xsNLxPqdf/3B4fqw2V/lvGVOd7evvve9j+obtt2U83w6z+cMlBee6oxEhNakjpSZchNt0lSUKXYflsTtfJiHdJVSyYkS0QJtXp58+uH5cD7ItYigwTiMt3eHzY4oDYvpJ588r3U+TIdFlqsHVz/x0z/9cz/1k+9/dPzGN3/x5YtXt3fP33v2wXKW/UhKbTdcfv/3v/PHf+DzX/7CFx492A9EBo1AVQ0TP36y/4t/9k9ejKlc4m/8xu+eTnep5N12bPNUeAeI2nBZZqaNaR1HnOcFBJJlqapmvlgjfBMSE5DWZciFLc+LiDAT+aRV6DzIRRDW/eid4ryPAoYxZ0Ur7ugdggAz66F21BtkMTqOU+3hCAGJxISQFMTjTvgY9geXiYI4AlJVZLa+nQkR0Vxi1F2dI1hhJ4rA04tjdNNQ0oGP7DgDiuACkvBAAgTvCFgU5KJGjClzKCjMojUYAB7Cq3INl2aGbh8JCN4gFQIywjX8YWePXT7kAYuchLDuUhcQDBH6EGxU4hA6Jbyvl6JhA4CxMabfjhiDsB46e0hwHzpCAGxiQOqyTlvbkWFV+lpiha7E79cJ3ZT+NS7KINTwCKQKHet7ocM9EMG9KDXGhkH6HbfoV0QGTSnd93bjbYRO5j7EYlwitW5P4pJ7NQCoolE0BIcRiMLfgiqSm8HBGhfNAEilxaI1RgJS9Oxs7v6BK7GCkZhDRgwhBug5L/7lwRBXsgjWhzXo0WhzWaxriR+hIU0LIBPZNBq/XSwXuct96noeDdWE99kggjtST0wa+cEQkXFVKcel81FpCLoOfSjA18X5YyhVGDRDezBuLPGzRSeZmcbTtOxLqdaQQUQNQAWZEyElJlBRESbE5IP4oCJmzT/mbr+tU8tl0KZVdBx4Wc53L59f390uh9Of+9mf/Nm//rM/+oNfe7Tf+dPr3VZQSAAE7ugAA/EPffndz73z9m7/4B/pP//tX//t0+3dXBVA5+VwqvM4bBI92GwuidhqTSUbASHOyzFTabUa+r6hyiXnYQuLaYPWBNQuxu0IkAsr6M319YMnFwlL4opMbZqgDGkYOQ1etR1PJ1W9u70+L0cehs994Suf/+qX/4u//w+++9F3ZZnPx+lwuKvTkspmuxsLkSr+8i//Aej/+90vvP0jP/iVv/BTf+IL777z6HLMlH0k9dHDq7/wk392c3W1Gfa/+s9+7Xx9kxnMrKmO220poxqiNBHymNRkOUpNZfSHWtRMFs6aiI1TnSYkWBY5nKxKyYQiljJ260eOU2uqRgZRwGuUuobd2Va7xz2AkoFKzwEW1sTdOzM4BgvU50IzFKkBuTuMMgBRScwIpiqACIkoXERdpRPFCmEHtoEoFQyMuPd6vVnqj7Z74FjwEgDdoNJtwhCRiFDFKaPO0/jkWrffiQ8RpAlAuGK1GE8DFxS5dTNDp5U8YHEfBwPrURh7YlNFAJ8c6qUSBGGCHQvGpcEgaZr5kqg4yyvEvE+80LWyQSEBkKpFmyZgb4QUjbon2oRhkYSBDM1AdZ2aiFakx13vH3RKBMAby4HAXYEZ98YEfA/XWhwYQLQKDMQE/NOCubY+dKUYGakz8FGGdAspf6nYDRDiU6Im3eGOGcBMZPU0jbtHfX2dN0qZTS0hhibBvdYQDCjckkHRDUUJ0dw0EH3MrzMtQVf1HhLEugaf/IodeWbrmvdAIB1CODUWRVMn8DwVOnyAuOpuABfoBoKZs54aDEI6C52yRXcI6tIzDIrJ66x17xqQqa/diNQKQH0hjgMMVAQvdKDX70TJ3bSmprekdWSwNE3zuNmoGKJynHZ0fE5AoICMnJKpZkpggmRqTZuoSUllaZJTevHxddkkYHh5uFXAl+fT5W731/5X/9FPfP3HP//97+xTcnVaUgCikhK01rTNp9OLj6+nZbpd9Pb6rDbouX3tq1+t5+UPv/XtRHqcl5SGpckH3/sOoj2gx9oMDI/zXAo7hdakAg5LnQRQ0TLlXEhse3e+LQj77WZbQI5tOi/nc83bbRk2h5s7TtwaV1Fr7dXdS0o0bPdLnW+evziczsfz2aTuL98Ytvv/13/5337w0ctU8jBuOG0uLh/IVoayqdqWZUpAqKmdpz/8nT/8/V/91v/w3//Tr/3xL//sT/2pH/vRr7zx4MoITCEV+FM//LVtHlDxn/7SLz/74P3do0ePHj5iuGho4J3anLFa4jLVO9VGTbe7caC8NFikgZo00YSUksqSOE9zO82wzcmwAhgSmgQ8i2chAJdPRFnHrx2kBRoMX/PeCyMgWm1nnclwtOHkKnbTGiZq4lYlGpEDzKzFmYKQIPukcuA+VWR2prSjJfQoRhBaptctDH0uiZB8vaDHXAgva4/F5JkmRr7CaNE8N3g+JGIRcWykou63aabcl0dCl5C6r7L12r1vUgsVvgdRUWVmaQ2ItAmCEfmWDQ1UrAa+aDVIDtdhOK3sNl0+lCCAPSt4069XBCvJu9ZFPW2BQfilaTi8RM/UM5CvSse45bEbJCbB/IoCaBemBgf1GlEDK16Nfef30BrAkMjJRAqO2sSzgRlR2BlRbARGfE1xvFoerXnVonm+EkSxRJp6PylUG/GeMEwjHB27wQYYIKgYE6UAuvGr0/GRJ813PeJaIXiF06ub9Rr2ciRa2X7P4pPHJVjruCiM7H7QwVW66FUh9YPnaKDzRVHrRBDv5er6kkFTiSsnrIEBROcdeuZ3xbHjfEDfehTXDNf61fqXg/aDFJnewFSEczHjk8Lzindlt2gyYmbLiVFa2SRZFlEhSgAkak0MuZGgEas0MEvOU7U2z3XclLwdRXQ5HVMpORMkaW063N0+3u/+9t/5D3/iL3z9rQcPxxTuNAkACBSml89vP3r28em8XN9cv/feRx99/MlpgsOrM+dxs9sPF5dvvvPuxdWjDz78+NkHH56m81Knw2l5/urm8RtvPn7yxstXz07nE1VEEARGglorpbTMyzDkYbchRsRKCU0ARbXqXN3KAigzpdKgVVnykJsMjDzP88X+AVM63b385ONP7g6HurRxvCTefPMb3zFO4/CopJE5AUKTKW03BsXklRKcpmNGwMHG7cOi8PL61T/5//5Pv/HPf+fH/+0f+Bt/8+e+9qUv7jdFSBHoj//A96sl1fTL/+yX766vE6eUh1K24zhUyefTmZEMmSC3OoueaTNSckBoItUU0ZSZhlxI9HS+vjvUhztOrndwPGHW8UaH//dAv3sE+El3pr8/yet5XztIFhFkPavBovhy6VbF/VfQp3AUHOKBQZOGZsz3S9vFNCHD2nP0IIyIoISMGqsFvM3mlL2aL85ytxsSUOdMEftwT6wZ6SuP/CyHBWQPPgAiAlEHhEybEJnTCk6j492F9jEjG5yYv3DoOBGQiXtx37tyFNQA9Xhg7jOIkenUfAgD+uXut2btEXrPL4J+qC37dK9zBwhd3gvh+YcBEH25IkLA3H590UmeqBVcd9WbCP42ON6qk0XuFaxdV4idoiFm8NTi67lecxl3MaeZqWvL/MGJvLXOtyJ2RyPo2iFPe3EpXmP2VKuvkYmNEoQYsylk3rf371B1usTTa/Iyq4fWoLmYogkOCKBmyZX+Gk2WLmA160inT9wC+hSGl7pIIfpZGagghnqQ9e74WgGaowG0nkIdk6jXFkZMcayoKyD69ELcK+pNafJRSB/46AWZ4/q++dqTQhSl6wWF8Df1lw4iqycxMXWm6e4scx7OWIA4cYYGRECETWoTMZOcOXExbGYmzRCBWbUBJMubQaQBQtlsUkki7TzNhLAp5TydD9NxnqfPvv2Z/+V/+h//ya99ZX+xX0k6VD3N52/+wfvf+uDD737347vnH9/cHD783rPnz56B6XZ/dTzNuQx54Pksn/nClz/19juf/fSntC23d6dXL18uMt8dzy9evHp8+SjRME3XSLTJnEqaazucprLZcGHO49IMZG5miJg4qREYZWYmzClNsxwPt4ULYJ3PM2FZ5sM4jGry4qOPPnz27O72bj6fN/tHlw+faipaKw7bgUudhWyo06GBUTLG5nGoDNlqNSjS8qCf2mwuJ3vx4uXp7//9X/qdb3zrb/2dn//ZP/ennz66JAKl5atffKf8/E9vt+Mv/pN/+uL5s2bw+PEbl1dPxRYugy6NSDfbndyJUpNWERIjMlGVRpRMG6Yi0ohoVn5+fXrr8S4XFquMxkzuRchEovdkZ/D+nRtC14ljcBZo4cbZ+5mho+8YFNc45RyRMzLgAbX/oZoigpqJSYxwmgByk2rMjOwcpZfUAIZqRNwZgwAz4VfmTI/FGgCAGBj2QOWKIIz2aWSgHkbh/q16BaIAvkdeuxOCrekPIiCqdSgaSRR7tFohcQBb/7IAyOZREPp8D4ZVcsDcHr3DJFjdU29t4fqqta7T7NqYyMSm9noUjkzQyZ9g2js09MsVf0g91vc43cdLcR0zdQoj7gJjeHyarqR3D9wxfSZhzReBu48g37MmETqxTwb0mBT42vq+moj/4PWZ3yv/xr5fAVX7gneIj6wG4JHTv2H9OBZPYvI8bj3BBnmHDvQN0Iwo2FEf9o33Grc4YjhCXDYEoA6TMK6gr2yG+y7/mga9Mxatai9ynLjXmFEkMEBGU1eMdZ2Ez955LugRHtCYUBtEIxvBCavXlK0e/y1G4aEzSNqf6n4Y4L7689+HRsL7w0trR+GWBkBiZjMBMgAjptqkSkNGlGUcd7XNhqqmDKlV8XuqqrW1Km0z5lSS1IUQd/sNoF5f3wjYD/3xH/xbv/Af/PE/9pVtLFUFADieDt/81rd/7V9+83e/9f773/mj8/Eg86IqbZabuzsEuj4dU8kX+eqTD169+PDZ3avr7/7RH10+ffPBo8dlsxk3m5vbm+l09+r585vHj68ePZ2WQ51OgibLNM0zU64yPRrf4JwNcFomU9hsdzxkaW1eGoshQFtmRHr16vrh5f7q6tHti1c8pt3V5Vg20zR/76PvvXr+3AB2+4f7q09RLq3JMA6KyZjzCLlwHjfTGZo2YG3tnGDODEsTs9TMGnxMTWbktLvicfPB9179n/7P//fvvvfeL/yVn/v859+lMSXQr3zxnc32Z68uL/77//Effvzhh9O44/Qq533OWSgJIIptNtvzdDicDtvBmFNiNrCmraTMiWSpGWlp8OpuOp6Gy5IABFSMXWsf+oSgzhHIUMF8Fxhin/UPc7M+N4AYEzprwu5VtyG4y3AcQvep5XD+CTkCGBCRIhFJE39QU/eUdvM1P29hN0DUi/Yw/QQDlahGmP2E+4MeYRIRGUAAfHGAds9eXNUx0EkuiHDp05eM7EjWxBt165Rcd6DHnjl6CumQvm8vAQQztWBTiZyeUveQMDPu7YReBgRNjIjhztoPqDc8PcR1Lx0I8q3/st7njggb17ezMREWADFcTAmpaXOQ6q9okTCifgK9Z8/9l5p25+hoHAbT3JMfAQGKhw01dN7G1lLLwDUsobU1JUKVmOmFvgjTOqQIBS3c/3QwwND8eBkhGOxGyFutp+CI6c6o+A69iHcEYKnnCqV/A6lH295pS9XeYcL7bIH+bPYeGRKoeI71/Ysu0nI7jGAzLQAQAGD4aURbxP8VdzEmzs1i9ahnaYI+SwJB0fkD5zYVEPLoPkSAQeExRTpGT38AACpCRGadHYuP7Qek44V+Hvz6ABGAUOKm2gTOWpTYHeFbbQxEbEoK2VCAEBChyRxVNVJd6jgMCKIIDWxpFZGQcZpnU0HCaT6d63m/S1//sR/7a3/z3/3S25+C+1/6/kcf/NKv/at/8ou//eF773/y4no53VDJlAkAtFHe7BBzW84i+OrmyFj2j59W5BevbnXcKvDFxeMH+7cPL2pmubs9fvNff+tzKo8vH700mU4HUFSV7TguAlOdi6kRqVgTycMFYNJEswg2mavWpiXni8vt+XRKQ8FEUttue2Gg3/voo7ubG05pGLaCdGo3yxkgjdv9NjEY120eiXlahMlKobocWE6FDEQJFEsCJMCmnrDTaEZ53F4//+5/9V/9dx9+9NH/9u/+nR/4ymfLtihN3/e5Jz/3s1+ngf/r//q/e/7Rh4p89WDMqYhUBS15yMCmepjujtNpN+4IkxkkZmZqTQCgmSHl0ySvXp2fXj0gbL6wlwDEAAyoN2i9PO8MT4jDOhIPtBHP+Rq5cA2jToL7kXKQBaK+MUmcQjFcwalLnA2ZVIUxLFCctEFmfyRXthkB1degO6ED2skA7KIdAMDVBcFXHEc8pmAc4jtiEijOk2G4HLukxHf5RkYDX+Z4Xx7FrCvGfgjwSYKA394KU+i0T/BDYGBeWPRp2x66/dSaxl5JD5US2lPq7Lr7dEUMcRunNd55QFhLLuh/5TU+RvLGuHeR1LRnh2ixumQfAbuM0W/C2hFQM0UK6BkcTlA38Lq6MwA788pAUWyHj16PukAR0EK2Cp3FcEMzss67BMpeOS5ERJQgCePVeg3nAa8H/p6ioJdngKGoJIK0RvG1vgtrnl7xgqsFIpVGpregYHqZ5V9GAQ0Yoa/avtcar08N9EY0rf2TnluggxY3jE2uyYsprwjLDolA3ct71T7bmu0QYlR99VO6r736+7YQbq/f2dsAgAgg6Kzl2rIDUAE0NTRKJ4EzZMUCijJXM7NEIgo2U87MjAY5ZVNJnETEBE3QtAHivCzsth5AzImYdD5vLy/f//a/fvjw8qd/5s//5b/0Fz/z9uNQjSlcv7z+57/1jV/9rd/6tV//tQ/ff6nzGQ3yhjfjTgkbouWUcSj8sI4vDXGZlsxjGhsxVQXSdjoe5tNC9Jy4XW4evnh5/N6338vJLr/41UcP3niBIMuCJS2izHmazppNTLfbrRm2NrcZU3vImeelLqIpJZlPOWehqdVZVHbb/Ty1l68+vnl1U+vy6OmbL1+8LOP2yaMHano+SkZdlnPJ46OL3XE6vbq7E1Vr2pZbkEnYCBkoQVSHZCSpUOJUFRad0n5UOfzSL/3K+Xj8P/wnf+8Hv/IlKCmxvvvuo5/4yX/79nj3D/6b/+H46m4su82WMiGVQXVZTHkcaDnPy5SsDsCMTEwKCqaimjkBpvNMr27rabLdQMCG4L6F6iNV6BO1BuuZtvVRdqZBgShalBrYJgzfocNI7vsa74kRgABwfvbcWDG+PeBvOClizxPdqkvv8VwHceZoChUoZt/WP2le25qaRD5AMHOjzdBx+A8NvA0GBgLgvmMG0P3RABQ1DI1i6nc9rX1q1RkmFPHZTyMgBRUJlgkRmEnFDLC1hi6/7QEnIk23uHeKxkLedE82+acIDUqndX1zu6uDXPqpuL7OSlohgFIHdc72aB9FFlAiNKTYtmJdJRrXB2IONjyW1zRpa0gTt8PzvqOnT0PmMKfzryPvfwAQoLrU3e6DJAIgU8ffnabuIQvvsbz1CiIuVLRnvN8er3T/FZ37gLXYApeF+s8yX4nBrh5QIlolMR4qsa8Pjca+wSqB6AEZwHlA6q7LnS8KMI3Bo2mX0pkZcfIoHym6NxV0tUAKpsfI7qUJEMUB9NMDENpPUAOgro2L0iCgGYAb+OB6t9bbaN6CgKCsVqzgJ2lNVl4oYDj28tJA1EUXyQyYCZFElDk5G6tmSCRNAKFWYQYkU9WUE1RdakXRYTve3l5vN/vzdL5bDm996o2f+nN/+t/99/7iZ548iBHuBX7zX37rH/7ib/7mb/76s/ffPxxfNqP9djdut6f51kgMcRhGgLJJO2timiVxLiwm1y9fLFKRczUp49YIN8Pm5tWBUgJtkMb3v/Ody4sHbz1983L/4HQ+n0+3m92u1goInAiMDSBvxsNxIgBoC6Iwx97A1lpJaciDtkqAJZf5fPzog2fL4TSU8XS4e+ftT/3kn/+JT3/q4R998zuAClS+/UffPR2m21fvPf/kcL47nlvlgtN0ABRi2+z2RJBYASrAAAQKDKRgDQCvHjxiG1/ePP+Nf/mN/+w//3/87/7e3/7i594FZkrwpe97S3/2xz+5fvVr//y353occePD/WLWUDPnMmx8qfzSJBOYmZiCYWvaWDORYb49zS9vp/1b2UxNhTipkVk3UwkCJU5N6AcCS/mklNf4CtBXSXS1PyP6Y0eICmpxOo3At9YpdrqxU6IdHxFCB4/Qh14sGYCCEHBgOX8PFkyBeKBxlAkW0niP+B7vvKCG7srQQ5kBYHfx8SMRcR36hE20D4DNVFTJV84i+hiYS1QNwUSRkPi+arEQVUfRH5tREMlCIUJMsMpt7pWAcUS9jRfI/rWSAqK6isse6Q7IuqDcSZ4V97v5BMZQb0wBQ7Q9un8fRp1nZhT2lPdxFiJnGzEh3O/BBNf+Q4/jYQ6oazHRMzjGy1Pn6ykGeokILOxOMYikYB78fvkHQcbYGv16ORqQ1T0L1U1J758jf6F7WtuvVM+nnelIwdfdN6kQgxLpWw8pNuyspkBOiN9PQPjP7UPThOw9D+jdjFhnBzF2RzFYEX0bA/O33csL12B4tjN1xIbWr6/5QVypelyvg39CW4XbGOMtXm/5dQxX9T4WgvdorL9lz5CRQfwLCQjDoZvM8FRtEWjGakLIhCyiaghNgRSJEwOQIaKAGhuA5cwOrTKnWuvFuBlSYb74+ONn45j+9I/+2I/9+I/8yI989dMPdn6fnn/08v/59//RP/qnv/rJ9148f/FBhro0FNUF9/M8LcuMF6mMvBmKznM71sOL4+1t1e1m3G0fPNp/+Ye++vjhxcXDB2986q1ZSHl4+fzlJx9+OM3LzauXL5597+76k8PtzQ/80A9+37vftx32jCxtTnlIxJvtnggFwYD2271mJjQSYwJEb2CIKKuatLbbb5fzfDwdTFouGalst9t/5+f/8p/9iX97n/XP/Ftfe3B1gZam0/Ti+vaj91++970Pnz978ez6VqUdzne3dzc3p/N0vDsfb5bjtcyGdKmYWQ1AuC6bIcstQcbLh49S0k8O8z/+rd8r++07bzylxInsB7/yxb/zH/77baJv/M7vTsOZKFFC5iwitTXmxKmYSV2mPBS3fjOzVBiIRBrxeFyWlzfTW4+HbU7W53UlHnMIvONhVlUB1CRG0DsEsx66KJgCQ+jChvCMA+tEEMQ4WHc5BiBCVFgVM2tXLWr2/s+oCLrCKCwYwnTT3JiNEAWsN6785QhUwsUhagnqsSAAnZnbpkcrUpsfcgZTAiQkBRPRPskbrxD4yaWZvb5wbwbiWPzLieLzh8idVSUlDg1odwH22Olt4ECNkbqMgBWCLNLYsdwBqBcxBGIAamLil9i3X2EQQfFBw4M+ZrfM/8ZtNSWWI4extrmAFYJPCdFkL56gr3AMrhmNgOPVxaDfd/eGU1EP+qTYx8W7xCqAMhgFWSiiq1E+Qe+6Y8Qox6iwKkARiUhjqyCsrGBcFY1tXbhuqIc1CZGaoENtNSZK/qlwvYmeZp1p7Ct8Xdm5xtuoqmLlGyKikq0JBl4HF/0Ne0EQLhHevGVSbRoWuJEg1soS73VVaGiEHf7DSvpDtJX8afTL6tyrgUVfDH0/kSuYqJOC8e4hqiFv03krL4pO6Ec5XhrAPythU5iaTmLNqFdwqs0opxgRMEEDVRBTFTSxNDCCnafGKavZmAsSLfNS5fzu209/9s9//Wf+4p97/PRBIgKFpvr84+fffO+PPvvFd/6jz33u+uXxw2effPDBtz95fn17fbPMVWAqVpQUGev5XJh2D6/efOcLT5+8/fbnPv3mW48uLsrjp48IdLvfbnN+8eK4GYfb6+n6NH/y/OV7H7/8nX/xW7/7r37dTEQnIBg2A2WcjsA5l5Qy8zTNmDInTmWoCIw5p5SJxqEwkZk2kYQogK0pIoro/uKi5CwGZvjyk+tf+eXflNPNj/7o19548/HFkNKD8plPX8JX3wH4Y6DLdBYwcae/WnWe9TQvH7+8efnyRogXwbm246vDftwudSGC8SI9fHR5Oeayv9puR7DWoFFTXfTycvvDX3z33//pn/q/fPTiOx9/ePXoasRB1YgSaCPmTdksbVYTxKzimj9DwFpbQUTK84IvXx0Op+32YRGd0UA7vU+UnPpzrGOdOQ3aOXhLBDDfA96DRqBJs1gw52fHnzSKriPqPeruaaPDt6jAqZM97rPihuvxAJuoooGv0MI4doRdxrJK9hANuEvc+snuMtDYEgyd6CAidVGkRpOPco6dlUE1BMGASNaRbyj/yVD9kuB90kI0QhDtDnlB4ROiBW3mx9F3aWEno7oYJnLeaw1cg9fXKgOiOFtFpKKIYBrGR4TkaxL6inAAgL4U5F7Z7ndEVa0ZkrneBKknt14HrNyKRWq1ztGvhD8AoZqprEjYAmh7sDYnKiLhReveVpir687kcH7u4TZoKzO/Ix5TexnhTDiaSvgX+A3qKb4nrD7AG9oesjBzRTNLPWxDF7xDbE6IonZtT5E/ichkaiEn8sXCXit7d8zhDIIvZvHVQt0k3dkxMzEjEGnRRVnvD0RGDOEuxsoCzyg+DYldGkSA2jnVtXHv5oGAXof2wtYJrJ56PA97rwINXM+LAD7C16khIMT7da0EoCgiRFxVD01nAyNUbWqQicHMVKApMIoKAWsVBAQFayZJiYxzXqQxAacsTdUqmP7Vn/8rf/kv/eQ4QgJQAGkTAr3x9MFbn/4TcT0UWmutye3pfD4tN4dJoE2nyawB57nWhITMTx4+3G+3u/1YSiIEM1WxMmQEfefNh0h6OLXvfuca2vTydP3G4wc/9EN/Ynt5cbnfDdvCDHWyoYwIgEhibbPdTJMsi3JqrbVh2wZmASPUeZ5P52XY5LLfnp+fqBoRE+PVkwcIpoq11n/2P/2itWVg/NVf/ZWvfunLT55cfv773nnj0dP9rjy8utpdDsOu9MdLR6ArAAD43Gefwmu/VIFAVQm0QaJGQN7vUlgUjFRaW9phaikB/sTXv3p9+Jn/7D//L8+31/zgkTUTU05EkMCQRJrVqZ5zziqac2rapIkyiYE0vDvV27v5ycMRgTUmpxS8QRfFZBCEACtCDQYTeiDxmOUdXYqmUmdgvaLuaufOqBKAIFGsRiFovnTPpUUEpqbk9isGgIYMfR0HaKftMWIwxCm1dePeKpXBrlPStTb2LusK6b2rZhiOb2gWtvzQm8kxlEtEpquoNCpkNyj2Yh5dySpGXSjZU6mvqzUiJuLWlsgFXmEToJLnXSTr0Ktf7nsbO29AIDGFhMgMAFozIkIi6OuXA0GHsgMwet2eRnzZC1oobe7JclVjppRchYVEMdfkazgRSU1jMaKbKXmjUdVFlrb2TRV6Fox2rPYf7f9aKW7sAxbM7jtrmLjnKi/M2Lqf22tYIfJi8FcY+SLqKL0/Qy488ISqZuStzZ7YnF9KUSOYP2RRDcTqH4ynGtx+R3uTG11+gOamgx6H+wv71XQdsfoqA+s0HwAiUdKmChLdKujIP65fp/HjmPmzRqudRawCxv7BqOe9AAYKgJC8R0NrhyrgQ3QcvMrx6jd2EfeBZAjy0TrYiAvZIZWYnRsIMgCZKCbknOsyRxlmFG+AkzWJ44UkKgrQlrodBkXAhHevDo8fPHjn7beH0T87EACkEf6NXwoEqaRUaNxm+J//ov9/v/WDTwpAQOeqN6/unl/fvP/x89/8jd+7fnndELfjuHvnYrO5AFLTamrDblymGRXOx/M4lkoVkIjz8XRGBAMxtONpPp5mZNpfbtrSbl7d1FZzQmnGRGgmIo8eP6nneTOUlFOt07Pnn9z++m9VWd568603P/Xk4cXFO5/+9NX24vs++/ajx7uL/WYzcoolqv3zrh/GLa1ccgItuXQRAAhGIgCCQnk/UipGWsby1//6nz3O0//1//ZfnM93ZdhlLst8nmslYgOrrS1NRsBE1EwWbYQ41WXLmNMwLYfr6/Py6YclJ9Cm0SrsvLMDFCJPSqhgBoRsAKJxqMMqJupxNLfGVAFv27qwJpjPtRmrxL4tFlaPhPgVFmaKfU2jV7vQu1VdzGAREInNQ15HrAhB+QMg9q1hGFAaMTRxFlVzvKYjco+hqqbEJCpxPh2n41r/UGcAvJ5Bg1jE7fE68qj5ESYIOzyKeqhnLO8TALiHM/SOhQ+BmcvHowsLBgARatUjIOvqH2PaRSUIoEBoEuNIQfR7m9L3fZrvSSS41/z3Ys7i4mOvVxCRiZxoN9PE5JMV4XMAPt/lD0nMmKqKBzCmeP+rIMc/t6kRsX8MU8VYrRFIFuNWAvTltJ1vRBON9EUkIillg64b9YCsK0kDasYMzrQ7kRdsUqdaHEsnQEBGivo2Um5cRG+gmMumvGoh6Clp9UREF9281kaFThHG42yenWJE0CkZX6tCFA7aGj6EvRzwBBAj1vGCCn2y3NW53r6FTqqtglkAievUCxgwJ6BwfX/e4EIPKejVLPWNo6+dfOrbnJSRAahqm4waJATs6jE1sEQMhEZqrptVUlEjQAIRUTTmBGyGuMxzzhmB5mX5b/7BP/74+WG7zynJG48ejalcPtgOYyqZmIiQGDSGjABSzK/9z+L++qs1CJb6vLRnH9+9vL79zvuffPDh89vj3TTVm1e3m+3FxW4LQHkY52UBmVqrzHnc7UxaKQNoacuCqcxSh33hlKY6L21urQFBbS1nJOGlLuOQeCy1NjMbxlFqY+K762uiUvLm8uGlmVDKxEKGh/n84vf+dZNWeBiH7Tufe/fxkwf73fj44fbp06c7SttCu/2OQDbjeHm5R4JNGckbRRH21UeCKjQAPbU23Uyvjqeb248JcJ4FSrprVjbjdFyYasKUUlnqEUSZS8plmu6kLWkYRJuYtqbUdDE0M2n08m66Oy1PHiRpABxnGzuI9yfAVm6991ep427rqsSOOHzfZzz9ztmqCRPHc62GSM3tgBCDp3RcEs+e33M3eWAEcuad0F1W1DV4qkbkOT9cLYMuxlhJ7F09R58eBDx3eMKIrqME6OkQKxRxcRaAmLGLee6hqHWoB6DiuDemSYPT7Rw9IaGKginG3kpTUeIEAGQofswjSPeLQIS+qXOlg2LWAM1MfI1an/WFTm/fD+tZuFBoEDq9UkGXSd07FpsZUow4xFJxBWJCQFCTe7Yvhh268aaXOxStV/PHBMEhua45tTcs1iVdQY5Hi9HAHLGFr7Wh67VWnq2Hq2i6qo+LACKghONbQOXglGIhRKf9XeWoioTAUVi5ziYeBDC3grhfG4Ds64F89sLMiXvsiqV7OpFMwccKsOeXQArkld1rnrMGEPPi6u1vH1hwisz64YLQmkbojv9yhyLsma7/fW+GxyeJ54GiTHdqL86wmyhxjBzafWMkrqz6uwzQ138umMMhAEJrYIpUlGhuOIsJkxm4dCqg2SrqgJCwIVCrlVNqzXJhV1qLNAYCgN1uDwi/+/v/+g9+//c54TiUMgyZbdgMb731ZH95eXV1OQzjOI6X281+tylcLvebUso45pSImdS0qWqrp9N5Ok03d7fHc7u+Pt0ezs9fvXrx8qY2OZ1PIpo3BTHtH1ykNJi2pUmTWsbdYZoT2u7iorWlZJ7Pp81+ZyeoSx2GTKAEjdg4kYG21vxIVVVRmGtDRKmWEs+nmTIPm83h1S2yZGKRhmYgMl5uHj942qze3Lx89eIFQb69Pv7hN7/73h+9L2o5QxoyT3UYhzfefEzU0jDsN3tKcDGMyzSP+83cWhJLBZdFG+hpmaZpPtf68fuftGaH46lpPdyeRSATVhEsqTVhbmaGyK0tJRfJaZmhtTqOWwIwa0TEjAiKhsjlcD5c354eP3wIvPoKYg+nvcZ1ItYRvyhjsDcdOBkaMLOo9sXlPh8UCs+Yg1p7Sr4ZRgGAROt99YOGGNvoyJkiRSOLiSR/A9hX94ZYM1D2SlAZABHGFLCZU8leJjuasp7P7o+Uf6kF3RuAjyLsU1fzOKbqnoluFo+mTowYEYv3gYnMQMVxMiIiErs0Dk3dgc6VExThw7Cj6dWU/x6I0UqqYEw7+B+jkxBCDlEJo2GhEe7dbAG8jHfOOmJYsMFmxp78XLKJnhEV3dDJQ4yPAoSlGvhFDhbB7eJ789BfPFzlgrDy4O3Rqvtp91nZHuoMQucD2Id8EdHWebYIvOC6UmKKnOHu97CKW+LZcmW8O/TF42dK6nWM14AAACKS2K1SQkUUj0GXyGgE2k42xht1wSmhmfrWAgNdHdb82QsmJYpN7KjDzPzYYLxyF8BZqLB61owR9P4inlYw3J97LdxTgMWsAGJfD4TuXkvgu0F9Nlrj/8EXvCIBShQHYN5NQoyETGv32gW5MWKWzKCaVU2YEkLyxOnDZIxkaqjIlGLkHc2JAkZgZDNNzEgmS5PEpZSyKaoGaaREDeF8OGiTuszvfeeP8jAgoCHPp3m73242Yym5pDGX4eJil5kzFyTSpoDQpE7zvLRaVUAZAObzfJ7PFw8uN/s9EQ1jmZaGRKZydzqNZQCtaNPFZjRrdT5tL3an28pEbWmJeXi8JaC5ShlHkyYKKqZzq+dlmaohcSqn0yFlJqREeZGFkKbTCQkLcxNZzmdAGLdFW3v24iNplUzHzRYQdg/2Pid4Oh3bNM3HU51byfnm9kZsWiYBA6bSlrNMJ6lCJRmQNXGxPBezlJrq+aQXFxdqiBnHzSUZKLSRrhBgOk6gIGLr2UeFtiiR1boAxyNEBKKaAIHyeZHrw2GRq4FT1eZPrqgAmG+8NgUzY2R/kHvh3110UAkZEUXX1bCB1AzuUZza/fesumxTIURDCjtfBGdsDd2awLkVVFRGxn4+FYzJreJIBZDRt5FF69UL5ZBBR8QH87m2CAHOqFmI4AKjApqYkfa1hvc8fj9viBCduPCAg77Wxq+Yva7p85MnHspAVcxis7iPWK+2BxQ8U0zAaqxojhRshKv+JWIRRmR1YYh2CmglwRGoo/W1kovgHDuE1yYE3d8kZ28AUJoAkecSROwG+XCPuJ0g6V4UkayCCAEmMlUJWny96D2wqqLbYFhnM4JANjFxSVaQ9eCRDSFkAtGKQGQkJsDY3eO3xqxnk7jnQZq65R9HrvdqwbokJ0WXoMNX/yo1QxEg7rnIwMBNyR1sO+UEEbNN1YBwfbTD4SkCJAC4CDcMSTorBz7tBYjWXXs6og/e0y9DHyrrvZNIVKgaXTrs2uxeS3ozM6alo2Rfq8NoivcuGqLrvvEeGLjcgQGJrLscGQFiM5gWE4NmJCJIuTVJSZAgyltDIk4+mydgZkzMxNTbZU0hD0PiVJskBQRIuRDD1fbRHb+QKsPuYhjTZrtbzudlaZsBUkkmi2q6uzubTdfXd2hYyuBndNgUlWaEgES5DJvRpBFz3pWUkhkO43aZJmLf8cSp8PXx2ZOLpwCqbRm343Ku0PTB1dXhcBYxNcqUa3N3X0yl5OSKdgNQToXY8oauX1wTGBK21sZhv9STgHBKU6vnZarz9NZbT1NJKlVq5ZRMwUAYGQ3bsuw3D3iPtjByAivEZT69OE369NNXiS7v7l7Isl+GeSh5v39Q6wJg83kqY16WExVaFnn0Rt6Nb1a4Oc9LE3l8eVW1LdOii0mSeZmbNEExV/UQAtIileYzlyJmiFitUa1DZuC8LHRzd5rmtrksel6I0OWVaoLuXRzwN4IpdfrPz4BofAmEMNz5aGe7LVhmt1gn8oiJbsIadUYENQuKXO/bbOaMBCCCKjRonDwNqB9PBQv9jLmxhCtF0bBPvneeRMM4LPbw9XI1hDe9tjEDEEVcNXsBUs0bHj7pgt6G6RaeiKtlpgM7a83AF5SDd89eGwg1QGCA1aKyUytm0HsVZnGNAinHW4MIPuItTNBoIayEVKe3wuEYwfwAk7lsJDhyNDNRJYSUUkBA31WMfM/R+EpwUPfhUDfddB2OAJDek9VO9FPY/tt9a8jAdTKM67iwx8B4Kqxf/5WdhwitkVs7WgfV1cXTVIlfX+1iHoQNvIuDgZIBiENSbBY7emNhpy+tEwXAFMMaoftBn2i552EAbd0sjwYxH+83PJ4MFW9aGfacH761r9Escd0h5imYeOV9VvzfDTp8FPu1vB0NXo3K1O+w+Ed1XR7Gr9D3mpeWqmqg4SELIb4LijLaCXBP4QVU6g2BTp76zzUxABOFqal6BWaqdUZik6Zg8zIlZmwInCBnIhAwBWNiJGweIQjBlBh9E5UL65ZpSYVf3nwC5nYz1kSP0wkJICdE4pJSHr0TnXISUUMFji3dwqyEyGQGCsoiKlKbAApYNrPpdGhVMNPV7krUFoSHl2+kYWytNmlyOA+bcVrqNAkgXjy6mo9zk9rq4qrDjgqUmDiX7f5yPt9ZW64uBsQGTerSltoUzFA3OZVN2ezGNjdTPR6Om3EA1eNhqq3ttkMpWZeKRosdCg9SqjWrcmatPsNzOpxyAlGbdRFWJLlbrqd5IiAgWs6n5XwCEMoZeKj1u8h8Prey2Z6qEqAapZzVTmnMcm4m6oVnk5CxVak69eayAYI1EUYw5cNpnmdBygCgTYAWAI6jhaGQsfBHoBhIRBKpYERAYCaiQVb44UXsxE6gOVwRkGLr8nxVFWmIHsyxH2oDQhOFzABGxKqK7CQmB8lgvhXLmMmrBiTsdidoEjGIiCyEKH44lNZZToPOL5v3OYNMN88W0WFDZxUCr1vmpK7RNrXwOehhs9MIcXHUd6346ICpyH0Y0NDIw33jDdTt2SE65gDmGkLEVYTlZL3vHfN+7sofY6dnsL9tMwenhB6HjToDZrGPzJOliJAPJptvUIFoQ7+WBQ1AzMiNKd08FF6vdrDfZcekgOaG+4DQUWnI/J2oJqfROnL1Asi7FIpIFCsZ7rOIdWEZAIopBnV2v7Ea+t3EGOMLMsnLj24UYdBZGWI21URMaEHlr3ShiYHT3Lhi8oiGzBxFRp8ScFADa1kVmd4IScODtOfwFY2secuTnmj8dzxrGFG6fwCLQxR/QcQiir5tJvaChjgNEfucvZu+eVsgnkh/ihDAMHTS3p6KVEOAhtqXP2i0KZRM/HkQhUVpUQVm0xmBE2ZEVODWJKWMiE0UScNM1VC0OSXlKx78ljcTM9XzlEsxW4CzmVizcRzZQKsQobh5uqExA/LpcEspiVRVyUPxEZDMeZ4X9BYHkmk7LovnsFqnlHi3vZymU8m5QTvd3EFKzARIU60qdSzbpU7nm2uilDiXUu4OJ1va+XRShc3FlrfbtiyAvN3tWoOU0nw+tOncZCqJ1HCSZZ4XAZqXevXwajrXXGx/cUHmyxLpeHfa7DZ5MFGrszWZy8BYqLU2n8+EMpSkzWqtmenBkwfHu1OTWcwEMOWCqKKtmak0IgYxSsPh7maXxswJEVvVMoxoKCKY8HQ8mkLKCVEqExgYoWvlEvudkrrMebMhQGstFnkQpzzOy/l8XgBTf/hi4bfbwmJHQt0aE0XNrLk/ACKJKvq634A9tI5Kul5mpX39uSbCwNT3p9Ijl2caQSNMbICiDYg4ijA/QSsAI+eMCck6pEAfoV2RsRgRW8CqNdQ4ECJfKwbBpWBAIlAC6lQUdLbFleNg6FqmeNMOZjuMjWjR6SUz1F4HWMBGIw9eHdKHStLXckWxE0g3PkIfuIZYR6iKTGLa3eW8ExLKTgso6dokNZ/dW4mRjmw9z7mAVdWChDMj9TFdIeKw/3YS2pRixkjJ7T8BorLxgOp/gADeYPeiDGNXkMNWvygA2NsDFHMbHeBGyYIr04SAfTeVGSAZqJvC9tJxjZMOnX3tj3dAdZ2b7vQHiBpZyFjNlBGTlwbsFQEGPxOFFBj65uVO5IOBmGF8TOsYAtY5iXjfACs4j7do/V+RtC0K5P7A2ppNNV7Uvebi8losAgVzJZKy+9453XNP6Bn2RX3+wsgd2hO6PtgF0uT2390Dqv+zf+x71zp/9BDVfEC4GSomnxsEQxVloDotTSszlXH0LoaqATKwl1nKgFJbKbnkpGY5ZW21s7E8jIMsdZ6rI6ph3BK3aTq1ppyL1CbLjIjS6rwIMaVsogYohRIa1KUSq5pmzmqKzEgEysfT2dxcHjXnoaqYCKVcxrzMiyi8uLnelGF/+XDYbup5BrNxSKfWFHB7OWx346KSCM2kECQE0xmhzdOBVAiBMxuSMhceNpudiCLjsujt9fUw5EyIhGUYDZK15oF2t700aGDNFKxKNUmUmKg1mXXZbNJm3E7LMm7GNMj5dPa1JMwp6CYUxPR486YpFLoQWwybAdS5IcLpbjrPMwqk/U7awpmxRq0momDIqYDMTREhgTprFq0pQF6qHk+LGKB7QxIHv+MbVUDcvE1djBFnElfBe6APh9ir3S50/NPZbD/kgIBACGJi60BMl9TFokQnW8DAgMEAu4d5NCWJAEFVkFJMRVrw5E7LOEXiBPbKNhCRdV44OG0i1RZrn7iruNcVhu5qE0pEc6sYdYVFDLgB9E/vUxr+002BUpexoKcCM28mr7bBEPxsuDqjrVnBCQ1vLsKK/iBgf5RiwXlYQhT/YvPmAfWtXt4ZpRWie/nmb8PfqAc98nZ7RC0DEX8/+FpUY07h1UnEQAISDnABbxHNkENN7uGnW0p4ygWIJVeoJv7TFMJjukt8/n9U/WmXJEuSHQZeEVEz8yUit1fb6+rqAhoEQJAEe+YMOefM/Ps5822IgwaBBtAk0Wttr96WmRHhbmaqKiLzQUQtsh/6FOplRYa7m6vKcuXeKyTCqt3HPwxJcItgZnMpwapRtdDA0tFDDLwloSU4DXcgD6MkJtcYlMZjz/lHGSWBh3NOnO7M4/Ya6XkIYw+/ncPbJ5fkJXmMYAZhJKQ+WikPg6Yc6L+GXDOk9CF1ksNSFNFiliJONlbijUIjQdWsfQ74HsChesMoXaIOiSGwpENLtPVwChNHj9rlNXnFecbAiGKdEbmZN41LHFmfzUlkgvRaG+8bT1ORxVOeDSpjEuWQEpo7FsL5cto33O43243P1646lSLX0nrbqzZ/KQVEOJ9OAfR3762pE0/nZSonuBJ6KXMz21vV3knker5Qkb53dJuX5Xx9MLgbHh4eoV773k3XdT9fLqrORZj4crk+nM5UuNc+TdNet5fnJxa+vr2YuRnfX546lbnMp8LijaAkvO718XLp+6bkpZzFWWS6nE/dbN3WqcymrVfjKddDa23uVKaFYWba6+amXKRMS8zet9paU9Vm/T5NsxNmJpHFFq33rZsCaHu/bdvD4wPmyVEAbNvzXvtyOU/TXGbZtq3V2nsvRPt2d+tOxkBX72i9dxBKmdQNZObee2czlKxxnUi73tfNupVSuvYg8Y6oMyqsVLSGIMA5Z7kYSPUoI0bcz6vvX/yEIXWUgyFkYbebJzCF6B7sZIcckWkUkcemrSTpjJALcybWWHbErJYIe3TW9voZ4scOTMKSXxEKI1Bw/2l4TPro1mMIbzBXswE0xUUI5bJbblUnyqgb/L/8PTlzwGA6jsxIKdfHyFlwO+pHHGD0wXHJbOrMBB0e3YxcJO5DdZXvnY5lO3lrPb8ZCpNRT4Vr5Cp/XUqcorN8o+5E4kQOY6fYlJjfLyEGAKOfCQZXfGGvcMd4ZachaBlAWFhpQJg81yNGhAtVR2rZcKwqYDZTz+iXvssBjYyPhYiw2UIcm1rGnGC4Z2bDVqKVTMMSym87E2LYTvEhK0/npjjSyBo7Gqr4fPHzyQIifJkPX905KDToYWOacsQgCbgncDbQLnI1zYMLxFLTqLt8yAaQbRdo9GNfaOqHmaNnlWZ0+IlnlxEEsmAPRTYDgtzm4VUbAw8wObh2a91cwc7uczc0N/Feppl3qa3xts8P50BjeGGCs0+uXVXLxCFK4pgLT2U5Lb35tt1LYROZFpkmMeq1voDEHFRIuFjvMpXpVIiKmte6QiHFjUy1mXPtepLSXftetWspixNkmky1a+vdatuFmaXMC7njvt6IpWt/+/jWyOptJ/bCwoXff3j/+fPTtu7LZene52Way9JahRqZm3fVdn64hMjezPe2o2tVVetlKkSi4WPMXJuxw6FkfJoXmYubbdtdXQEuVGAuRL1u6g7m6/XtLCTMte9t30wKzOdlXoi3uu1bf3z//nyaYay9MfumlZjN1aDk5NrW+613tSJMbuRmbHiFKgABAABJREFUag4z762rGgguhdTBZqpk3bRNIkWotb1MRFS2re57O80HpTKmn9Gt5jXOstZ83NUDrsYIcWOsmhrJsYyXECZCatFY60gYGBJ/o2GL7+6elmkBl4RZwniJFBu7ITbX09D6GI3gyuC8qTQwIqQXgXse+i/A2VDPQl1Z2EzH6A8DUY++NvxSEGu4Y/bgCcukXtiPGawmO8hysp31NQ2r00BDorYyHh03KPegMSFXpLkP1imbHeVt/jNQGhp5OL4IFrIDQj8AaIrZc6wd/KfSbCIAwpJJEmNWOIrCg2gUXzwNWGo0AG7R99DAD794UCOE02jpXrW4ScsPkk+IH4/6ko6Z0aHRCkyLxqsnq5WSTuaUYrTs+DDUGA6QR7ij8RhCAUtjPc8YFIzalrJ5RIbwEHuMrjEf/jCWGhU0HMOvIZuDgL1y0S7ir0TeRSYF4VRY+JFNmIlFAFL70ikwXyPI93QAeXAEcyBvY0pGMBrhARIBQ70XeG4UHcKBoub0GETRvpgryEkRxVrwmtXj9lI3r613Q2299U5SynLaezOnXquqqrVeK6BAN1PtXbu6UWvN3KzrPC2Jhqm74zRPZOi9leLL+bTtKmXa9946XGYHMU/x+NxQ917Xbk2JiImmIr32dd1770xSROZp6V17awb+9Omp92zsWVhEiKUIv3l4c5pnVZ/msm+7mvamt21vRvMycZnWlxdmcvTTZVnrvtd9q2qqE0/h4bTuOzNP8wzr+7bC9OHhCvDedL1XlgnEvQHEbExqwkJcjCYDtWrEU22dpTDJNM0w2XZTwzyfC08FDDfrTYiY+HK+nMo880SGVqs7SMRd27632rQ3N6v3Vu9NFd0C7GYDqXZr3dVhIdZwczfV1jftvXXde+sKOANSq9a9sQgG6y6AiARQQA4yUx9RO+pDHNMrGlVHAPujoLNAbCIc2Vjw5CMAhsFZRMZRUh1/QqMFj/kws5DTsLIZnUHc35yPUTQR8RdjXnr8qggACV29vuscPyBHHSmFHqYC0esiL2EwTyW8ATys/xMHpwyyNgQ3BpjnaJdHKT/gltFpW6IlCTlghF0LqkhysDGCeVBlHVALREKyfh0C3/y/wM/5wKW/8EgI7x0a+5YpO6REoROzGZEf+aEDMCROlvuwreB4ainHtfCniDSZrzswmmiJkNCIuamKSDYo5izH2yGPA8MHRBYO21BTc3czJo7DGRVBJOb4MrMDo5TlyRf7FgZYBEeYZxDcSwwEDnnp0PzCHUaWWjRyolxuKWOk44CljN+RJE4M4ygCXrnF8Bhr51fo+ayZiMx6vKtsZj2WVwR7NIz9BryYeGPWMaMnoDiyzkFNywuBUfuMJizvZojaaGSmINASMMhk8JjPeDR9h6IjawgnNKJmZuYde/emsaRdmFhO50eAlIx6dwcJIwsDkmlapiLCte2qoqZQdxNrLdxdWtX5fNLmL/eX82k+PT6Wuai6E8s0a71BfJpOfbsR8XKdBJCFZRJVt9qnaZqXCe573ed5JsJue2tq2qVI623b78uycCnTND3Ob1qtYNy3e6Zz5qmUeTmv97WUiecyScHpcj4tvFzf/OwX8+Xc+RMm6kKt1WmatKmpGUFh7phlIhdv2nvbNj2fZraz4U4s963urjJRmYjK3HprrZFC66ag82VZzqdtu+/b3R2tVeGwpIZrn6fJXYWJZyay+/2+rb3MApncqxPVanzi02lqtbtTU5VuOhs7kZkQK6S5gmP3OAvLJAXeuhoDpr0DrnZmEUjvvu8NmMPGwHNNIyhCbqCY0f9naZVDxlFrUnAZCDDDF1VwggED+4a7i3DXrMlH+B52JnHozZzD0YwIMoCI8U8q2+FmXDjrtNStGhHnPDhEA9H6+yCoRE8cRz2qH3OSKKJzU5dIXlghck8lcNwRYnZXzkKXApeXsaeXBm8m84o5IxVbR2GWMJcBlGuQX6vLBDcSAsAYWGYCOIaC+RliKQhrQFI2EJdR1ca7TWoNOE3unOCwI45HN2K5F2Ygd+mBQwCTxK8MYQkRByE+7K+HLdtr5B3Ij3tQBjTYAdmZZTsARBcYNTu9DlkPPD4/BxEBHOAagZKpMqSsw0xp0EDjD2iAlsNEM9EnjLYjkxCIUIIuJiPbcKj7RkLx3NFA7hop00cmiecXIdQNBh1p3rOLcYDJY1EFcpHboEiRhz9yIkqgwa8wA8KEJH0tAqXDCOpHWvZxOnLQQOEibUlksniYNizjjkpn9N3ZhBjY41BTksLc3G1QycbjNEDIHErcwgfKm4B9SgYUUxFiB7UO8zbNYmrChZ3dNbCvrhqmMKfreb/v3XtrNpXSmjGBK8xpmU5wZ4J2JS7TVNSbu7vpPDvTst4bkfEyM5fem6pLmS7X68RUazvNp1prg79s29vHt/B5npdtu18fHpjczIV43zcC5vnUaycmET7N5zKXdd2WZWKR7bYtb5bT9d3u/JvffHP/P//xT77+6eO796fldL/d67q+ebg8r7cysZqKTEVAZSKiWpuqgrWTf7x9TzMYfrpce22YiheeJlGWibkwrGoAnGWerJosrAoisvBdNjeQq8a4PgwQuZTpRA6st82U4PO8UGGG5c4Gje+OeJlm6l1h1TfVHjRZYogzROpetWoRYCr7dr9OU+31OpdWdd/bsRjuFRTOBSnu3QEX5pC6m2YYGyADvWKiSFb+uD4RHSJAH1VSqmHjmCdMP3CPTC+SPXQCqlk8ZaHrHsBs4OgB0cdMYmSaKDxzijvqQs/TT0fdRxTscgwU1Ye7EVKIOZCEiIxOJCGVyG1fGVOQBgUY5B86dthkYPSjGjv+KG42J04dFhSxXyyFVSmwUM0KdvyV+GX5YQdYn784OxgfDU0QeXMqkQO9Ad4l5fu15AdFGZrRlYJFaqoR+9Q1Xzr0H7EdIfegDSG3cJpJEsZoIWgqgXrBQKbGEhE+JxCmmm8SoxHMisHGaiwL9kqMd1+bxcwYNA7kSEHxLEYjEokxJzogAkoMg1wtgnUutYhRqSqNjQ3RAZnD1V1S1MDMA3eLcAz20Xflr1I/BsQ+mj7kSNfcPAy2IlmZZa5SG4ebiNIYK7/gUZKN54MEuRyu2ZCNlxrgHw9tth+tcVYUoTgxG/OJPJJEJE4mJO46rFVJQGaoDd0pvECZufXYeEkK69rIiloFgcWZJhgRWJgIaF255AGepxkL4XbnidRs3xpRDI8JZstyKlJqb/B+Woo51mZlKevzWnhmdwEVcoNyKd21lAKyqiazzHLq+3bf6syTuTGkaycGGGroVqkT3Kd52de1tn6eL9O8rOuqT20+zTydtet0vchyvTX7d3/5l//u3//7et9+9rOv/u1f/N/f/ewn53nxqRdZlrkx7La/EHgq8143zCc3FxZn0d6qddrIAJIWjRWctq1KmWVCYW60zcYE3rZNptnVnWzfajOdw/AG0N4JuF4v677e75WnWas6S60dxEydy4lFaquqVk21qXuWbAZXawZTUyruTtw73Arz1pUANdNuTMUSG+duuu47iJlFezrBRLNoyWkzhTMYyV4PL3z2GGzGQRknLK+l5ypGS78CwN3UVHsCIQQKdlpg6AYq7K4sZWAy7uwBWIwCK9vp+IswLoyu4elDX+DLcTtxTAKS18gcLzm643DPjJFkQMHs8KDTIFvfJMgxoKFUUtdksyCSbl7G2Jtlaa1j2fdn1kNmmNgHi2MlZGJfY8JBEW0BjIo/GLlfBFOPChcxHtVMJ6ZOEoGJLEeQ4yvh49N6Jjxmi0SQUPF4o68lNBAumA5HSpdJaBKJr5cI2hVwUzugZ0qVYCSFsOTMaZADrs4Miv2ZMQqyQ+ScK91zDdcAiyjX52B0f+xIo+mIgPn+PGmz5jHfcUlLxegXcrRJx2Z3wIESzw3JDgsE02KOTcwwDd4MpaESnOGm+bCib8sOjpJyxDzKhYi54WkVBz/YgJSoWVC0Y+jvQ0SWySskJOxupkl6i54wcx2NtpBSkjFwwCyUMt/kdUUQRtWjABlz4uw5I/HGh+DxaSQLKzqWrVLrtjclFoANYi619onEHaZQM7NKMGIqssAhkvrr3pXIBQIjdTJNMbd7d5AxVXWubRJilEkmEm7rCuDmNC+zLCcISSlkPIfADygUdbdApm2v6/1+vp6XmVpXLixS6lZdMS8TE15ebtfLNRycHt696bUXQmud4W/fvPv4+VOvzUwJ8vDu3V3p73737V/9p//wN//n//H0/cdutv7w0TZ7//UvHs7nZV7MjAwuUHMiqLk13HWd5xnMyzTvvS/zstdKIK0mhd2hrbsRWX+8Xm/rSzfMc+nmUoQdzfs8TVAnK6768nIrAi6i3am03tzVvTiIeldV5xLEKidhVu9qIhIIeG2Vefbe19pr62Bo73yajMxqZTATaQ4njRjG1M2bUm261xZ2kU4jcESxks2oc2Hrg+yPnLNlWTtKDiIKI4A0ICGCGQxG4ekDogQNmIbMxTObsAQziMziuRmz5BA4mfhuMUiAx6pegptp3oJALpgpotCokkZqcow/COAq2T4WHQknDcTdHRaCHx8TkAGeBgaR40HLhBG2M4PHmW3TeCYDnoiQeVicUhL53NLvywHTV1e7KMssVtKOsi0V1jjCHwAP6/CI8m5G5ZU9Gc/tFTPxxAQ8B6GJLxwvgGNCi5HHA0HLSWdW6hnnB3kdGI4SiOL21VEHKUW2+NTh5zO4xEqhOc8XzQfsQeTPxRSJewz+ZSRIZADPvie+IIQiK0dGxzsbO+U8dRKj0AWIuUSGiE6EwAbjwwHiVb6MLPEZNkyPiDGYySN/5gLMXEBs3cAE4uMQ5P/T1G4lljeoD6YHyzOcFZwAHfbQ0WnF9gYM0aAGWP8ay1PWMbqhJEskwye/0rxv2Z17NsRERJKODfFnBiNXorgVru5d3dya6VY3L4/d0bq59qZNPSdA5CbErTUqZDASab2p6jQXJgbsfl8/fv48z6ciZUWVaVEGualaYSlSzIgcVKRutfu+mz08vOFZ1m2fReZp7rWTqXbzybnMITZnFlN3K9qrFHn78H4t6/1+P53OZtpbOz+8e3n6xCJtb3VvrTZiUei3P/yRQOfraTlflsvbp9v6V3/93/7jf/irT3/8bVf56qd/KtPsat9+++3z88vPf/nL68P18XrmeT6fFxPZtxch8omIqLtq7S/b+vDw0LrNy1T3ZgxhEiYzM+3OIO3M0redRZlL6Jd7s9btfDqJ0/PnT2E+buZcSjPaW3eQVkOZ3K27saoI9a6X09KabnXvpjyXZq1oAXMzr611VTPFLAojNSqw2sxNewdpmS/WWynS3bdar+z73nrXCItxHd1jR50dyAlHFWYGkCWZxYNTnH5tsAQhEUH5mCcRgXJExuyqg1Q34N+o/uJXUHBAMzRZrgsfOlggxrFfoKFslAtJEgxhuEHTsfK4GFm4Ybh+AkdhTq/rarOES21aoldGFsgswYlMjUfDIXHTMaql1+ATaEOGWIqfSlIQ4QDZMEDnwOsd4XYAIkk2ZEbDNPOJN5vFb6ARCCukQ4uXiWJcej4gLIdZZxEMJOAI/ZmrwkjOBqQzXFbjkcW3EFB9vMCYJIfAOjw54DYo6Z5qqgjiljBM+uXE54jhhDliaJTYSoSnAyeLIBclBRJIGlVH1MHEEEvwm5gFlo7TGPwrT5BqOD/EQhg4QQaPiY7AHG9uWD5FKzoSS047Bu4UjSqNeU00I7EjPn5w+JAE2Oac6615dBKZnvOU5H9lP1qCcRNNh/orxl8cqkJyIlN3dyiOFjOl9X4gaqMUyeMRvzcJUuM5xpuNv2DuHjvgmNnJO7yZrr36iRyipuZmgHWtvVlvZrrMM4Rb79SZuF7nE9xIXGRmpt463O5bBYSYpvncez+VhVmcQLHpgriQnM5XQFq3dTX39f1PPsyFe61KLoErwfe9Fp9mkW27yTRtrTZ7mkhKWXo3N5qnubub03I6Pz/fVd20nnQ6ny6Om2lTRfft/PBw/epne29/9fe/+b/+j7/7b3/1X9XovPxU5ge5fnV7+r7WTU7vOuS3v/0j3B7evHn7/vGDPZ7mU6Nq7luzqv10mUoptu4Ga91ElmW57HWbBOpWtauag8zuIHf2dW3Mzk2ZuVWdTyczAZE6DKaQZZpZylZ3JZCUpp0UMi1ore07M5Z5UfNtr3vdjM3dFNxNC5eqZLW7dri3ahPDyKZSmjWHd1hoJVMDCq612oX3Wnvty4nh4UuT4TZnT/TqSYBchhGl0jGajKvlqb9nUvXAs8yMC0e9H5hQViIJ6+dpHNnCD94MSWqv8hqn/ChqFCfPjtDD735QjDIyDv4+xhAYCL7p4Y3uKRDOkVfcT2KkuXpMoZnglKoISqKdjxhAOeTEIK0PJs3xXhMooKBRRY2PgcskSpV702g4e1FGygwPIV3LRxLpdtCsEojI+B2wSfL+YvIDwtDIJruUhVncYTECpVEXZhAjACIhCOBIPyF2EGKksbY7PJatSbIw4ymOrQkEGLJ0kAGAIwv6OHcs8cLkCItshgh8LFdPSs7AOjxDY4QxInd1SB7LqABG7ZoJBGOakF9JtqkhDcvoncJ3N6fBroUjUSKGxXXEANLy68ZrtIxonzQf8hSI5cvGACUUSSzsr5R9kPtgKo/3mHTmfJvZc4SFrWeTiJjcxOfgdNcdSxJGRU/ZNo5nNxqnLF1y+hEnjJk0xm5gNXO18IlMbQAIBpJofqFmTVtXJXfVauOoq2rvvW6ViagYNQfQ76tI6WpurqYKNc1zu66bsEAErgRuqgVEJtVctJvbw/RmWU5dyQXmXc1u93Vft25a1+3dmwuaLfPMUva6kRRz0q1TEWZe192JlvPp+vBmr/fbdmcpzGXf9lq306lcrvN922SevFtZLpfrw7uvf/b7bz/+5//0n//mb/72+z/8qLft+vZnzm+qQ58+9r7HgKV2k+nM1J9ebi8vLz98/OEXP/v5xHS6nJujr3czqb3Pp0u44W9blRLTT57m+a7WoSBQ8UJSaN73tu0qZufzqUxnGK17rbWaKUge375ZTqe67+pmjcghht5h3sjA4u4uy7y1tu1brRvD5jeXtnVwARUYae29dRZmh3V3hgtUtbsGScfIVTvR7EZmMOfatNa+nCaKfjkqBnOKsiZgiqj2ExqIHnbEb8SVyLBFzoMU48jxJiE3xIQrHKt3dyceFzhIecH9B6k5YFOZCJZtRJrQZLkZpb1IQZhiEpCBwAeynaAoEoTh3JHq5g6WuGhMcDUNYUwWzwTARWSA+wN4iVRhxkQxGck+25yGdXxiRg4iEj4acoOzugXAZjagg5wO5JyPhYcKjI46/rCIMIcZAOfCOdeNJOa5VYbHJJDTfS+DVYwv450Ks+bueDjFwrLX70iYQ1Hs7kCgDOBD3jGKx8i0hFha5KMqBo4AlvPL8fW7cZli9mOjC0q8mkhK+eJ3ZKj1nA+NSUbEWDtqYs4hEAEOHdrgxLfGyodISs6vcRYBPI6dwOO3R8cVGUwNuaie4zyLjBLb8meOpO0RzYktW4TRPiZFAcPCm4df9ygV3MfwI/6DYQ7JTGmjbI+ERclGyNXJOQ7x3Lo9ZPPA68EfJcCBhuWR9HHmRjcsPM7fqEiiowSDDi9AM7fu6KZEZF7BC0xCQ9m69taDg9R3Nbcyz+itV+MrOaH1ZnejkiIlMKoawVrv5mIKRz+x1Hs9TzNaK9PWtLdua7daK8F370wce9hZyrqvxHo6X6lqbbuTL9fTu3fvPz99pDLX7i/3deXusK1Z2+6MneBqrfj06flmXd+8e3t+fJRl+f7l9l/+f3/5V//5bz794Zv1Vs0gvGh3n1cCs1cm7dRVCVyWy0mI91r37enztz9s9/vpNP/s66/fXi5LmW/3+zQXbfpy25bTDKbby/18vWyVtqYTS1kurbVaHZOJ0Hw9S+mt+7btRJhmeXp6Xk7z+XwpchEuBZORAz2bfpRSZNt3ba3MJ+1Wt66tfv78udU+zdP19Pa5fp6EujZVbVtTUhZ2pnCNL7HmUJ2cRcRNCRr9YOGpd91rq7UBE42dWkQcUM1rqAr64BdR2MeM1OGDcO6HX4+pD5AXgMf2ciYxQkT/gbHDCabGBGd2QLWH8RyAA0QCxqALBLVQmCMEnsSqGvVZzAmSyzNqoTzbr7Q6z/eGHEt80Sl72Ob6mABj3Dqk0T/yQ6UjD9Ke4fAn8MQwaLj9ABRv08wlDOQjZ2QzQFn5Z2KlgWwEzz3oS2mME5VZbKi2YIWACK/7GBImCvbXiMDx9qVw8vVjy/zBLD3WaSIWG6ib5sh0BObMNDikRQM5+OJnRgrIcpmSdRTguRPnGsjXKUU4F2Wkyx7G3YYnGgLUCnlugDu5CQ6x8IAwdGThXhEVcGZ9R8gPs2BneiUeE8G95CqBgKBiIhvkHHiRWa0nAGJGcCQiRK/MeRBFIxRPOqsAkA0OHI21BkjbtZAF2utYfjzHbGRTqZs7cjidy7NZSrYTHy2ekztRMGox2EojieY3moXX6Mww1MfOAIzciTLNIkcGw0CCvATlgMmYFWTgEKlZMhcEYInzR+yq1tXUDI3E3Ywh6sY89aZTIXM2t8KTOs1UpFCrXZ1LkdoaWRewMKlhmU/NarP9832fWL3SNC2n00zl5ELLQuu6OSq7GIwI1u3d+/dM9Nv191YJEPM7HOfzhUBb3fb7/pOffDWfLm76+JM3Msn3Hz/9/nf/7R9/+5uP3316/uHz5fF6PT82E4LywkWUSy9lIpl71ecfPkshX592IrAUkZnK88cfq5Den5/ffzgtl+vDm2kp7vUsspwXA4pMJSbh2qsxABEyeGtqjqngcn1oW9vrrr3uu18uV2HZ6zY/LNPpsm97q9u6brU14sIkJNDe1cl7W8qpbnvdt7322tp0OrW6L9MyFel1bW3b90qLC8GVulYWqW6td4dzMkUagax3diJQ7/p827etkjzE2QwyasgCAnaEO0foTM2UxG2mKCE4/B89NkBk5U3OSANRMw8ww1Tj2iABzeyFOWKBOUnWbsLFYle4D8Zd3Kw8tybhKOED1nSPWxARz0kzAh3t8OioMdZ4jMV8PjrlbEdSbx9uDh4LoBIPO0rvAIcPlJaQM7+IbkehZWY8tANM4umJTwFnu+sI+aNXx3Bqi1g16JDhvpTpcCAIblok+CwHEO0MdjUMCQKNJJHvDFkAsrAfOioGDQlIJF1KXO8oIROfyslj1LeaUj9Phi6+HOxk6Qo6ZsvRmuQbCfzZzWL1n/vhGn1s8UkDuPg6KEmZbgbhwAo95QUJwGQCJgxy45HXjjXp0YcKzEuoascxOoI63KDWJRaw+zFuHoIvzy8/qgwHuSnxP5kZRI47CnIPW3BmZmT/lf/7wFeF4r8fk3UaTxAD3APALMOaIp81XnMBDa1jtmpZmORdIDOlgW9F7UKZISwzpI2uJf9hD8yJox/XMMHC67HWfIcG6xpuSmpKsHkpVrXvzcmtQ9UmleDa9t1QJuHFUIW50FS1EZl1V90er5f7VsuFeFrqbe8QVfeuCiPuEzWzaSoyTWf3fDtlKQB9+4dv/5//6//848fnDTcQuppM07Y3005ertdZMX/6fCfBHz/946dvf/j9H/54f75vt5XK/Pj+w5uffADPMp8fHi9MDmownZjnye/37fxvfvnyaXv+/lmmaWvNdPHrVfv7ut72/eXj3/7dcr7+6p/9mm58ff+ugPatEpdFCju33lpT5z7Pp25uvaPIPC1119vTE5EJk8ynWuvD9fr08tJbo41bX+u+3Z9vaq25LaepTHPvfdvXqRQY99Z7rdu2tW0D+ba2Mrfr6Uq9v7yst9uKYtocs4Dde3flUSt4IYKquYbTk5kSyX1t09T3fTeNAZUes650DxuxmogpKY901BThvyhZ8Dos3ccPbIfGHR/4QULVB5OZshd2gExVJIs7wAa72mMqGsbuER+IOCrHUaYCyGVEA4nxY0KB0b6MO6DIVtc9PRtooAZhjQCAYodaxr7c5OruAIfDohNBiI9mwlKxxWa5XPY1jBIFg4NAae6QnQQsqtxBrIrscVRz7hT8VOfjcxko9mQl+KaeZKTkRObIfajTxriRiEKm55T4giFtG4DR1TBMPRqTL8tdABjjc09MCa/4Q77CqCnNkdVDdA8uTPYFWVGYAzLnSOMAc4Lxns0biCSOnJtG+U9ExPkZE98YvemAm8Jzc+RsjyI/U9TopZyFCwtZD6FT9iMZuiVK6Iz79oUpq6d7qAMpjDMNmlJICjkAU8t2jsPgNOqbV2xyFBpDjJczKWKOF813PXj4NkDM0NLol8vTBoSEQfYaLYofyXBUJWOU7UPSHT8GAoJPqDlUztFB5IRYWBNtPxCSb1In1t5NldIi2801louG0qT3tq4rF2l7q9vGVObpBHBT27bGZVJz5qk1JWLT2DVPa9PumKuqu9HMpffOgRBW9QYi89N8Md927UI0Tcu0LK3uP3z/w7//3//rm8d3Hz/9eD4JOa/3rW3V1C+nc1mWb/7hHzfTjz98++m779q2kkzLw5t3f/Jnp/P5dD3Np3lZ5tZQGF2bd+91bXVbCU3xb/9f/7eXj/q//X/+v//sX/yrT59+4G7rarXubB98wne//13dt8+fnwmyWiOj7fbMhU9lOZ1P07Scr48GrbVyYaESK0KZmYu3bgncgl6eX1prLy+32mspUte6Pd9KES4FEDW/r3d1OxWZqJj6vtdt32vd5/PC4tM0L+el3fpe67rdgE6lxLlBV+NW3Q1ehLQbYK5ahKgU6CZMvVqvvW7VugmnXU8COpSRNGJBblqhMWtlSlIpYD4Q+VCzk5hpXLYD6lFV7RrDtJSChXRyxL0Qtsdi6JgZhBWySLJDRujPsMLHEkUctyD6ANbezTGqu3ETBut0hAPPEBJMP3WO0I50Pcp4l0EUYKfc9AsSgpo7QRIJz3Ls8IwBBZcvZipEg0GJZKT4ayUYTEaH5zwjamekcUAmMCJyduuxFwbClGEkYIxCRMRGY4QQDyRfg45hOIEJIuQOG/7/KWLLOUiECxtwQyL7gajQ62cNgyD467x5NDBERBxQ+4C3AI+S1DH8hEYlPAALIhqGycTQoagIl+dIgz6MSN0dDCbKZcojf6eMLg/cCH7Hft9s6ODmJYB75tEbIhdpEcHVjZxYwn4nD0TkZCdhsnB2ZRBn7fAaOHN6PiocJg8r3TG3wUi1TEOCG4KlfBsYBNd41HnmUwsnEs8xrgpR0F+z7/IjQRMx2NmPaXD8a3z4yLA5SBisvdhuEaVQDAHNncinIoGWgtJryanH/F+1q1PT6tpDA0Mkqgrmfd+WpZ2XJZ5b3Xaiwly6up75vlVHb92sEU9ChamU3tV2Veen2zbNEwi92b5VPxWYynzetEd3d3l89/n2uXVMKLX5tjeQf/PNb6f58c3bd99//OGXv/hT/fz08nK/nK/V7Idvv1+3+ny7f/zmiXw5nU/nx3fL49u3v/rzH//h754+f95//2m/35+f96mwWyOCoPW9wUVRPn+6ffXTn9dmf/2f/mrv28JEzkWmZTo5fJJprZ/WF3z95/+89/3HP3z37e/+ztFOp/ObDz//2S9+uVwvrRvA18vVrZdp6t3X9a4GmdC71l7naeEi2/Pnl/uL0Xme5lpbOZ8pl6/5um3b1kWmqUyuttdt3+/7/Q4i7WaTySTuvvVtW++qjVzLdDKoO7R3Iqfi7tZZjAVtJzahybV77z5zEW5tr61106B2MBMU4w4FpPNPlZaJxtOB8AxWKHBc0ZClpisDRmfq4SSaMcowhmcmsWLForbyQRc8LJopRFvM/PpWMAYOHhAzJ/1U8na4k8OE2THIMAm+RusxAG2ngJssZQGU2/pStDWkaOldM148hGTIJuAIOBhhM+BpH5xygIK5n50JJ2Hm6OmJo8qGw5iL9sZSArYnJjUNey7NncOveDoRTG0qU8uCLLG5V/XQGC0QMCbnfizvHfXzF34BWdpnBetOGALv/IUID4OjYznyJIaHMQh84BBde3pdfFmwZxKgYQjonqYMEdHCQs3HkHU0H/mWs6dJcH60epnSjQaLKNc5IH7dqJWLmRfhoy4mCdvoMTzOs5E9CwXS7Y5gfaY2OEAgtm4sMo41HH6o46J2TtMU/icnxNJ2ld2TQHqYCI3uxo/sK8xxiI6qBGEykXpvMrPoyaIvM0qcNI53Yo5GnlasGFOm1x7nOFBDP0zhYM4iAHcn4iKlOLE5QK7ajCcnKFwEExcwYhKg2luvS9cowHoznrrM3F1vt9vDmzdmVFXdQeaFFuvdzMi0t743vb557ObrttV9a70sE9e9Ec263c/n83y+EsrL7fNG93meW2tO7eHx+vTDxzdv3pxPD5+ePrtzmSZVe77dzEzm6UqP869OMs2nywK3avvv/up/+/H3f4TXcir3uteP9/OHd6R0/+FbtI5pBp1A9e+++b9+9/YPl7dnMK3Pn/Xlfr2Km+315uxdRZbL2/PD8+fnuM3vPnxN0OX6eD6fSzn1Zmzoatttl0lEQESq6jCmaZoKJipl6rXett2JLg+PQqId5OSC1rs67dp178vDQiR729f7ra33drtNp4Vm6upkfLs9b+vW9g0wcpKc8MPUtHAhk1K6IcBUdtAEdxUhIYDZnWvT3k1mSkWSkPdROQbCTwhXuQPEx5A4BTRCgLq5KbEws3rPIV/sZXYzVRIKay8fuh1PmwAKAlJYEjoRHEWEHG4E4dcqO4tUgvkBs2T9Q6MIi6sZ3hIsmckGbTEczrJuotFMZxzBa3bLQDh4du4Iaa4kC4XTC8ENx9JdABTVq43y/nh7nmvfB3c800I+ZPdoHSLGhm43o9UAVcjUmcOjLbglbqoeKlTkhs4IPz4w/BHTE3wTEfjQFmQ6zIYn9sMwx3qEwRkZIe21WUHw67MdO8CX8YIDTgqzthHLhYUGLSrXoh3RcNAKgtvupvHd+WAG43BUen1ROoIxwUlwfF8Inn2crbSN8mPMEr+EmQoRacy1QO4h+80eDQm5GLFbdnGI56VmNDLnARDFYaIhWBtV+AivnpyB+JAEgsfwKgWBEWwJcYL8eITx4EbKoiTn5TwdDtBxyoG4YpG98usKBm+cLJKsvkBEpG7spNn8Zid25Jwo4YgJCugxTGK4Q00mVoKHuKm4tua9hUuuG6z3sJBqtfUoyabSW2/aVVm1lnl2oLVmZr2rkFEvIqLuvTenYnVf74WnwkJdu2mf5NIV3JTMtlqvCuL50/OLMB7PVzWbTlybns/XMi3ry8v1eupmE4twOS3Ltu0Pj1drsXRaHh/fuJnZ9suvPlz+l//pT37183cf3rPMf/ubP/z+b397Xh5/95u///H79XS59u7XE9dN//Rf/Wwq0+fn28fvv//4m29U69OPn1imh58+TqfH6+N7Ps3LaVHty/I4FRYQT1Mp0modHlW03jeZRVW5CIuHDUrd6jRNzL7f197b+Xw5L6dt3+/bnZzYicukpkyyqwlPde/Pzy/rbWvrRuRg9K5vHh8nmVq931/upj1WgREzgSI6qLZ5KSAy3cUnODGYweid3Qis5lJK7d08Nu6GkxeIyIbJJ8EJ4pwi2FEPxaWzwCLCxdZCiWWHczwcBga6MQ/PwTjdMYwNBk1iQRFdrBTBgHmYOJhmIXOJ60gZ0xAXzBOQ5nEtEnE92uKjVs6XiIIXgwI/LsBRffHgRwSek2khfsxeJxbMFN4Nr9NZItcY2kJj6YAbj/c7CjiPWnigJqnnjxiYb56RDRYTWaRcS4T99Z/cPxyiJxJCbJ8kRrpNUOTViGkYtr7ICSTnbOf4nRE+PLuGeEwMjqgXChKm48krkQybnwxu4ykiAG3iV2/M8BO05AwMHN/zhkT6Gck8vywLJ7pXshl4oFKJX41UHZnTRub3o0gIE3LVGL5kS6hWshEMIM9cc0GKAK/4J4PD4eCwe4vPyZQ9C4PcXeRoQ0a2HKSg/O7HYDq/9zjFWa0cwJaZazpgMAYuSSHM9DznFCjV+EVIP6Yx4z7aOXNjsLtTtFSjyjFXuIuwqw9jEsdhdzqoctmxCDMJwvPOPDb9OsW6dy8MYxRQ6w1FZJksZojMINLeVV3hzsSzeG/GBKLeWylLU9W6qcNqBU9MYMK96669MKFMS2Fi2ltjMEszltb7aean2/PpdO7N5vNpInaQmhbjruAi61Zr7d1us/CyzDC01qZlZmZiK8uspt9+89vzabo8nKfz9Kt//utf/eLna63LvPyP/+LXP3/3+P23P3z103+zd52n+e37N6eJm9qpnH73zY+fn58eH9/0XygaHj78XCZ5eP+wXC/g0vu+vtzarvMyCxUzX04TmYN5rTdTuMkyT+60rpsITcvcW22q737ywbt2ber9cr08Pj722p8+fb7fb6fTAkzMePP+bdt7Xe9dm7X9/vLS6m69TVPhUkhkWU7mfb0/121X7VNxookQWdpUu4hEiSRg1p7XsysJE5HB1b23ahkzv8B9R8REFGGeeSDvZOB+r3UzAA9D87yHTLDXqoWZ3RQcTBhkoZkT4Ly++auYnVwK56CZiIZW5wjSw8x8RNJ4a0zuMZ5FlsFDuoyYcY3CjJKOOYjt+Z+USoDRBTExJa3IHOyuIQ4bfm2gnE8Ee93dIczd+tHWHOU5vrCrxgCdCdmmYEjYDhlcPE8LEVYqyyj8ElSNiGJVW9b+jqPWpOHSfIwSRvdC6g5TJrFE2KNAj2ohKVMpvrOcIib6N2aKQR4NN7Mw8DlMw4BULYc8Aq9xGUhjGzDSYf7oJwZMA7Jo/ILbE3s+LZ4/jVog+8RkXw0FRjwm+iIfMJEeXCsKqO8YGjMzzEpgTLGfyOGmWoq46ugxcozDMtD55LZx7BJiLuFocfSOkXOYYRkxjWKP8KhEAloNvzePxsShrscCJov1mKCxGh4wMCPAncgbwzn9yIYpClQzovBrPI6ZxxfkHr4scZfEyRxBtkNIP3wMuimeDyLPD1ZZPMhY/RpVDwhqVqucicmEudfOtLPESCVaDnJWsPeuk0wgzOfT+vKiRLVt87K87DcFabd2v5d3b/d9J0LV7ijPT88yL0REhVyx9+67lpgU6/55er6c5tM0td7AYoq92ryUprbdbuv9Pp+WTrg+PDDoejnttZLBzXW7o5SHy6m1SnDr/X//d3/5X5eprpuDpZTrddlqW87Tw8PjaZ7J++ePL7X3z08vn79/JtL5Uq72QOoAKWi6nCCi1re6P99eUGLzJXXba/O61+k0UefuvYiAqbVKTOfl1L231t69fceODqxrvT3fLo9vRPi+bvu+9b024a23t6dZrd/uL+q9dtrXe2+ba3M3A9T6Mk/zadZe67b3ujOxjV5We9Q0wOhs3dTg5MqFjZIOoDAntFYjE4whZRZWUcj7YIOOhj6ViaHsF4pj7m5OBcIF1s2cHSA218Edx1E/ZekTxIYsARFkkMD8PZyjJI1LRgBPwCISTqg6afy7uYNs1J+gYdIe8S9fIuMYxhA4PyoNhARIttCRFnzkmeTFmo2hrmeRGxfCzdM8NdAN44GE0CslkTA+bRBXKak7mvE6Bg7CIV+KdzPcPH34uVEM6jOSuuUknQ+8PmrnWPuIxObcmTjlOPAxETgsFli4dO3Ib3/k9y/EdDzoN2mhxgwDcZDyEVyRUQbEf2TioFee6PBTCFI+aPhRZn4kjIYsn1NgPThyEZIj5JQc3IFXeD7gSJL4YiTr6cBBjiQ1qbkQFwy0MWEUohhRl6EvDxI95RR7DLIoP2psiFbVQ6vtnru0KEc5IzeMg0jJIkUUIJGZyYNCOqayHvYMfuSr4beXzKq0vjgkeRhfD+XPcLZRecdAcIU7FIMfFv9fKm08HyDyqALW3dKLi0hDxerGgGrz3kpJc5a6vsznR4KRd4bDqlGh9G6i3juIyrxYXffaJ57rvrrpet/my+Xx/KFw2fRupm3bt/PsDkKBV0WO6c1QynmvN69dqLAXn8raa3l+vl5+NknZ62ZUmuoyiWuvhvtt3esmQpvj8YGkTKS6zMs8iwuX6dSbdSog3u6tzHMz//zD8yTS1M6ncu/382nGbqYvuuv09Lmue3N7ftmNuCxT3SvIlbR1ZZFqQtve9nsFlsdrGHrDTM23fTd3VpWyzGBy8a7n64Xc1nVVwvn68HJ7cRAX7Ot9b9ub6V1r/Xm91XUHQVt3iKrdX1a4C0vda993qDIcIm4Gh8hkrd/W237femsUxr0iFvITV5BPIkTs2pystSrhMsZsTLFLMrptQ8fA0XOtqwE+THBGzB6X0GhcUR/9r4Pc4FCMtY027JUTZs/xXFBPsqvIhjWQmoODllUb3M0oFnDE2mtk4IBTCSHKALAR5thZUJn7F822M0kSZuDmNrH4mDTkxwIotosN1ChgLn+FjjOMmhtLtiBZaiW+mmqAI0vlEDg7nRglEjNbNvQcrLmEtiO4DI0C2MnZh7o6WjIHmGLpWTobmxqJZ0TOPJYB93jnSPAg5GgUn5RiRRWFwpTMlAe+RDm8d6IvoowbY4Sm+Lzkh4wZ4//SKRlH/+PhOhH8n/wGMA4EXmtWHpJrRDMYEYFZ1dIwNj8IA5RR13E8tHRCGq3ka5p1uLu5FZk8ALI44UzItZ2Zucg9BXV5DIc8xB1mrn50BeTmph4i7Dg7MVSh8QuJOI5pHMBBKUuEiF4hyUwDHMOkyCI+Cvh46l9QPA94BwgX06HLSKfcY4SczyJ5+/F6X0CiYbxubmHeMi6kD5XgeB03Fs4kTt7NjLmHONhJa0v80Z0LTLtrHxYipL25uRQuRbaX217vvVVnVm379sLm5+Xc9qbbPousn5632117RTf0Tk17VVcVghiRmm47gbZ937e6bfu+VylMItveACYSoIjw+XJ9fPOh9U4iTdte19bNnEqZiKlpW9u21X3v5izPL6t2nE7L5XpZzqdm5lTue1Pzrepe+9PLulbtxtOyTOel7a1pM9cONKgSnF0mevv+cZ6n82kpIt47F4lSdWYRiNU2SQFUhEx7NWu9k/N2u6l5a/r0/HLf79Np0W6fPn96+vSxbXdy7+rT6SwsxLKu9153uJuGIANcROEKkmVuvb28vPS9WusoYepcslAzoyhThEMkaqZMjohiFNuYmIKqqOYeFtGhGIUNkN+yxMsbGnmChyeJuWuQGHiE6MSs4UnqgFuyAGng0UMtlvHFslw8Dm944sb8TI2cRjFKA62PWH6YP0eBDEey6hHldYDzI3IRmDk8OzOoD4rdqNbG1G0Yv7hlKczHmwPIwxk7FMUZmlKegrEGfNz3TE+JeyC2uAR2YYB5ALYgMDhAGJjZ8IjMaHlgJgFRR4NAo3HPTHzMVl9DaWBsdIyyxydC4DIRslNDkB7gCTgQ0XC4o5Q5pLYjP4gD6mMV2PiwkbrjR4baI39JtgUZcZNNZEdJgajDR8k94MfUNgcMZ+Zuh+FPHp180cztiXZREsP8C7HY6/FyZ1O32K7GLCwH1jTSXeZxH1LC+LUsJClOO5TFcFUZzF/KRTfgnPYSCMK5dyhOepQ2PBq6yB/BAkJQ18y7jqThmUVysIe8nqbqpmZqeUKjBQ9egOaby2A9ntHBPg4hefKB4hlw3r6RRACCOUdJT0pw1+a9uXdTjSRmrQPcotYiaDez2Cvuan1bb+4gyLScZhYhavt9mqe6vfz447dQe7icYd17da3erW+71oZW0fb708fWqhh6623fvGnbe6/Wmt/X/dPTZ3WXMhm8TPO+tbks87SUaVqWqcyzOc4Pl3mZDWrue+976/etOggsZr6u1QzrWnv3qkaz8GkhFjd+ft57bcRi8XyEO+y+37ZWnQmTKBsm4YlMba9V5HS+XCF0Pk2TsPUuk0zzNE1Fe4PTVltP0160fX95ue/rjaS4m7Ot67reb+/evj3NZX3+TK27WW8dJKbWer+/vOhee2v77d5rb02diJmnUqYyOfD0+antre2NxEWkSBEu3sOpqYFchMycYWQqMHeFaTJkRtsuMvlYWZqXa1zEmAbn/bVBcQYc0KN7ztLRJeMRKIlvRCDJqA8mYTATW1onjFVLB4IcpxL5RjKeUyoR8l2MIJykC4/qh48//qLzGKU3Qc3U1NVM9YA7R0AnQmrK4iWD6Zkofmh+suIPsVtQXGLNdT4qZg4beutGIOHUE2VGhGckhntYKXqQzC0mZTS0aaBIa8eFjfmjJ2wVXZI7hSsqiIBx3UdUBg2AYSzaJRBBM2C5wdXc1ESEw/MmmzCCI9YKCEt2ga+AWdSeIRWMhIwv/PES6LI0/DOP7Yaj0o3fNtY/crAfo/eyAd/EA0uCvgiGSX+SlnJlENJhc+SPLEu+0LEeuCC/zkej1o0WNHQq4yuFkyuYS/rq0PCWGkPXiPIiOW2wqH8kP00Ak2Zh+J9Zl4a8IC7KSICEzA/seP0nBYoxhc5z7dlaDLQw8vbRGyJnCgp39iGjzxY4icRfvvB4KkQ4qGbIq+0B/tArAQCvLb9liyEGKjKRD4mZeW/VTRkohRFNAcy6We8EEuG+VQLV9SaFSEpfd9ubNq1b3e43CLn1prWb9dbVzLoSsVWzrq1ufa9EvCwLz8XcQGpm3Vwh69rc5c3j2+l0WVt9ub/85rf/+PnT50GFWIjYjZbLI0l5ua+3fa1dQewQNetNYSAXltJUtaqZFYDBwnKaTkJlXdf7fW3at7arKYjcTc2Zi7BMLEJFW3Wl5/UmZQJQe2vaD1VUN+cyN3MDd+1du5TFiYWpN11f7kJlfbn3fZtKEZbWujVT7UQSe1Fd1RGkryBetdaauiK8wZlOpzMTuWnvCmh07lImYdZWoydjTlvv4JUwW9TJrmHOTIKk2cQ0jLNcwmvDO0qhA8kdpdwoRYF4k9lWxkkPLJKSNZ8V1aiWS6BSIPP0MHEMvlFULAhyROjBGBQ9RGz2kDyghJjiwmP9UFZhli7KaqbIaWYOM2O06BZat+iG024RB3IKALnycLgS0JhVO1Pa/UbNFXmlm8Y9NbcwNDbr8XIIouBrC54bERIytiCJcJSAr+Izj1U1OeMYbyKuvY03FSKLAT+ZkWSBiFE0xzWHk+oRyIOAmqPaaPtb72q55cbMouaOBbwDMiYD1E3VQoUQ83SiHInCXnH3GPLTCOn5NiIVBioU8+oxf3VH792GtuE4LERjPQ4RhrA2h1IDq6SRFj1q9YDoKVCbaHGSY58FhoazoRdiuDpTIPTsRrkTJcI+vx5WxCCGDcn5dYSLCByRgogoTUlGwI+MpR4RP/tAIPsXIjpEv/ESREgBhUWyjKzzqlCP7z6Xfw0Ggh+CYR/qM4IHT2GAZrAYFkWtZx5erPmlxMmM6+SHUWLkaCZYgLkMZk/puTIy27nadt/is2jvXJK9BMCtgwXsZOrd1Kpin8sbxt73O4pszx/13U+m+cRceuswOy0ENwK13mt3dnWiiZepTHUjc9Peynxm9n2vxPT0/Hxqda/7fd/ruu/WpExmmE8zEYlMtdWmju4zU9ubO03TJEzbfe97PZ2vlA591tpO1pQLEXMpym61rlvdt40XeXjzZrk+PH37fauGKUISE2HdVoPPIQff1976tm/kYC7TJK7o8N5ab02KFCrLct5aW2uFlGWazdzUel8BnJYTg5+fP/dWrSsg5jZZF16IaN93VQWp9krcSUitcSmFi0zsvfXMCkDM9lkSZzBLIFyttcZuUEN3cBC5DsWhOazrDjiLxL1xd1PLTDCaYhrDTMLgGyKpOxQLWrMmNxAloyf4ja/j3igOnUkSQXZ3hI+EjB6aEo9loqPcH6d1eFUCTG4uRchoKJc581KkhECdwWH0H8wWDke0kLSTg4QH1IvxAsHIdB/e0bGhJLWyEXR8sCdGxCSmoHsetbIhfsNhUxRtkru5GkA2yuG4yIkhH4k2FJr5bykUsLRCy7LOkvYTEoTEkvxAn+JmRogOLkl2+5kvM9GNUjRShwcuPoJVZIwYCEbsijKcQO7R/sGRPEMzFxLiML7O75GYMjVqbsEa+Vgp0EdzOFiCyT9aMh9LREFEECmR1VyNJN7DMXiAI1zTPAcMFNvHhkkyExwJ24yMSo7iIZaJvMMeJPrXp+cETye+AKuCG2OmaY8SSxiG0C2Sd+Jl40rIUL65p2qDsjzIroR8COLJj9cmPjgDiWzmaMc8U15mVIrHH68dEGiOhgws8bpZldBYZISgeVH+wrEEI/4nI4YhnXvNk2EGAbEzU6/N58XCCgMG6r2ukMmsc5qVuLvxVMy6N8fJTTuL77c7s7gRoHD33kG+3j6eLm+5FK31VGZWNnLV2lpnYXfSfWtTFSlB9anV5tNE6lvdG7fnp22aFzOo9mWZSeV+X91ZVZdTIZYfP36elrnd99MyqVPrXSa/v3xet63wpEREZqST8DwVIQhPUmZ337oaEQmTFGI+naa6b1KkWaUednDFzJWInGSZp2lqe9vXPUSqba+wmRjqUDcHFZmY+b43s8bO58dL33bmctvuqnuZysObN8/32w8ffzDr5tYUqtt0OYPIXK3XIlTXrdfNHCzFrIkTnLq6aw1mvqXBoRFMilhnNQ2uHpfCEZMS+wCJmIcwy1DCzyMRAB/0AbLYeKX+JdfG81wR5e5qhzGKe8/ywwcNYMCzREFi5DB3dAKnVc4w7XJ3h/Cx44XokAxF5eIWnAQzZ8q1fx67U5wQjjcMQHrYiKSPucV1iO9lbNMIBswhAY5FNEbBwNGwQKcYjsGSQ+eDNvN6L49GiBAy42DEUkJo2UrHxQr+DEOiehRhJ8CySvNBcMxCO5mmeJW3xb/5Fy750bwgfSMzifCgbBFgzhIhKZNERH+Ww9QyvqlsSkI0TTTUuHAcpnjxpAKeOsKPDzjJjSCvYunXejLLg8zHlm/UY0EcMTGbaTocwzMMykFRee0zKSjM4PCGiyG8R+M4JOUDAvdRK8DhHCsMIibDh5ecq6NIYknjeSEEkvEIRtsb/2UI9jwEbKPYyabmeGTRNPNYfAdKg4z8N6fsNAkO84Dt4oy/dtrRN8QQacg9XnE/5lEO0YHXR+MQP3cAtdlcDvQt6SDRwObR9cxxKdWzER1ifSvIkDVb9MGlUAGs1eD8jNxm3hvMWGL2rSReChGUHMJUmL0rO4hIVbU3KWWe50IkRLrfy1wmkGlV27W+cJYUoHDO7b3XnYgnWWqv2/rS6woFOXnXum2fPn1S8/N8mrio+nK9kDDJxDIRXNW22+1++7yt931dGfT06WndVyZfpnmeiAEmXpaZCwde3U3XWvd9bz3wXXp4OD+8edTeFWSk5hCeltOZCjftZZ64sML3XvetCYRUiIqZEXgqU621qda6t173utXa1dwUMpV1fXGz9XntTanM6767mvfGcKgWOYePJjO7tt4rTAvxJMXNxImIpmXpdWvb1qtaV2YJ1UgEEyH23hkWzEiN1UtmBmMhMBdJcW6RwoBQBtMEk2Nomb5WoyfPHjNqkTh6JGOPbsDkBMrQNv5CIAOJsmSP7SA3i2AdlQv56z82oAgzpGVV3rvXa5tv1RyvPskxykLC30MOOlJRnvzEamIUDEpl2QFUpM9PgvYBYyeckvDXaLsHRBDRLBsVH7+OcuUhBVRAyPaIEnwJWAhqNsJWKA7i/EcUIwvMSi2HAvmN5GT0gMGQjKlBsoyQ5wDSyj5AIh52Q5G9PPJ/DAXgcrz8+CTjM0bCCbx/QEwj9kXUsdGdJGSd2eHLBzuCUUB5GAEIEAn9L0l0JY7RZOSZQe6xx/FPlATRqA1/awfAkk1VAGXZjQbQlsLzcYAd5YCbRDjZYLmV7EgjpD2tXGMWwRINEbJhggux5uNgkI6nm08wsMIoz49WGoP3OjhV8S6O3DvcAZMVxYkC0Uhtll4NREM4k9ch/N0w5Mpgin08PvhRiIU+sRaDmHlUYW5Du0EDrXRAxLUHUkXMxC5y7OQjlglUgntFhKq9MDGZscCN2F17uLoTswi7NddunbTXeZkNtG+3Vtvj2/f7/tJ1I6BWyFLm67nvq+vO09L3G795P59OfH/S3vb7fSonN+vsdm8yMznVbYWGxQFLKbVW8g7qta6lUFmKu7f7xouympu/ub4pRSZmm8oyixTZ97XX3rQbuJs2b0RUpEyznOZS7zcia+u2zHOverrMpczrXouU8HHZ99Z2Y2cCewcRddWpyP5yL1y2fS+nyXpz52VZ3H3faiGXUgjt4cPj48MbJtLetOssE09Xt7sUmYQn5vX52VqDdQqlKExrdRGolTK3ttX7ulyve+0OMirkbnDt3UwJ6t7gEwjqloxCZqOjyobHxgtORIIC03VKSMfg0NcpHR/tbK5vNaMY+I/JlgOuZqlm4rSvHHMsIyKoW65MH//joOEj58YyAuDohM2pBPnEDRBHbJ8ig3NatnhEsuxKhkNNoBqcLtOcfOskRuf1DEMVlkicUTmaIeRdAAVP8Ys+mzy26lDq//PVR1cxwKqIG0f/baYGBBaRmfU1WISpcookGH5MXDKG+RB8+UBxaYSBDKnuX9pRD1wh9dXA63+GfVmYvgU4l3V7eoeMrBbf9pFiaIBH4Trj+X0xy8hBkaSJsoSgg4sZqJUPFJE4Rd0RkSKzx0bheFwIojyic0zcn8xNVaT4+J5Gah9NTLwDDqMRCydRuB94eZpzDF+PUoR7i50Y8THZyX0gg24+NlweXaMP3uggOYE0xQKvD84HPxMGG3Pp0Z1kZnN3lrEOIm7cgPqiTcnW2i1h/lRTJo4YBFjPK5TpdFiXH1OjUJKMLyPKAUQPn18YscCUc1G4G4yz/ee4fdnfMJVYBeCAW+69AEdjMbH0BiYGmas62E2liAPadyfApbfO4bVoIKC3ZnBXPD/9+PDw03K67usN5G618AyHlBmurs3Jt/X5fH6zzOd77aaqvnMh3ep0nvfnZ5wupUxlkXVbn54+n68PFC6BrNbtv/vv//Wvf/3z//gf/ot++6O7PlwvISAJtvc0T2Uqt329r1vfW9v2qUzN1byXy2maTx/ePLx9/6Zp27b9clrKNFel03JStZnkuaK2/pOfvOt2c+sB/WlTVS3zxDApE8HLlS+n63a/9d5VmysI1GGqjQqmaZrPp7bvbd1UeyHq9S4wJnY1005uppUdiciDiIVFmKfa2r7fRUpvdYCRRFLg6NCuTc1BXMpEIjR4YSAmEQ6FpcV+3VH3MbOEbAZmgBgAT8ucmLFxSPMdwz6EyTWwoxSdBlCZlVDg/4Swmw1mjRFgyN9jqTjIyi4cr0BukMJBk2eSMW8clyv+GZ4AA5vKJ5Cl61i3O+JSFLGW/8rsYduL/BuRRFjEjwXCcYdDLJeaZR+lHQZjJ4kvr6GTclDiOACDscPg6BkcTBLBl/nwTHYfgEDW+F+wMaMFSrZIRFgmpNFXZtHQwEV3wJy0+nzZVzoJuTpJTjsTKPexIMFHMARA8ON7GS6wNuTNcC8sEayzTI/ezzHYQUeRmgBapmkYUVYGWW50DQ5iAkqRUi1TCEmot4MAGtMCzhYkHlggFmrEoSFMAz4ciUFyldj4kjwOUmndDpDKwq4eCIMHMk2siwZwFk5MmXPCxQLjSIT8L+vnSFqpIsvMiNf/Qsnz9zE/SXmWp/SLRtN1JHYWh1ESk8brIlNAvsRAAfHlKx5iY4QSz+FJCzsah+jncqWqtcZcADpa+ZHEaRZeop3InfUkIogpsYNYnMgUhThQbPJORGoa4yBzJyMpYk5OBnMwm9F+vz886LIs236DO4FjCY4T695ANBFr3ej8Jp5wb1t3nC9XIla1en8xN2Yu0/Lw1bv9eWv7DhCcH94swqfv/vjtP/+zr3/y7ic/fPcZoNoalRlO676pvoAhXNRp37u3XvfdesUyizBgl+vpl19/vZxOf/zuI2O5zHNt+7lMbV/Xva9bM9j1dD3JfNOX0zTf1puX2cnLIiL09Hx3lofLyXrf7i+X87l3vb28EElwO0CYSMSpbft+33qtZra72d5YJmsm01Srtb47qLc9QoupMaSUSWQyM9vvPJ9YZhSrVVn8en4Qor3uauogkHj0ft69W2o7aGKAvTMRzIjMvat1Fo7ykChHrMTsGmX5gUjkac2gq0k9CEFj1x7XNeRBxOTdDWBO/KHWCgTiTERwChf7FJEyMhJ5RngKsmAMkNKUF46xdBxH8Bk9xJcamgxenDYSGGKdww86f4F/sXKSACYbQvwIYAY2O2q5xHGGm3o2za/RIMVTOQrAUcWPFzQzYfliekg5UE0S5RfJhjO8IJXAUbUhhnxkySY66tH4xCKsdqzfouNVPCJCJmkFuRQx99zZM9LIKD8xFr1niES4fPuYE7A4QCyjDgDCejJSCOdvGfTCTFRRR+RG9IOcQwiHIApKaRqEwFzNrEjY2PawLckfjcSpCeWZuRSx1K5kXQLkVCNi6ZGrjiReiBFKYHKQECzsfRJOHJsfHAwzDdgQI9Uh1SLZYL0iXolMxlc5NtSMozMOTza40eK9/uuxSDpSLPILyeIr0sOAxUZflkOm0YPkYoc0+UuoKmAnznCfX7QNojAhbi9QWOIjBFxMUuDOTk7BeifvzbV3bYRKVMil6c69wZRoyruIdEB3ZjPrrXMpPM/WOzGZgamQcNsrz4vuW9u26+Vyv7+0dVuWi1oVmZllc0xEEK63p3U+Xx4uTWtd1wmEVo0cxCJTXXfTfn5Au21dW5SUTOX27Nra5x++/8Pvfst0IhFhqW3f108OV9W2bVU3IizTiaf5zcPj6XSZZj5fzwGLicv3P35u7Ycff3i63dbnp5fe67uffrhe3lzOp+VKb93vz7e212U+ezN6PNdWafbz9dprZZdx67zXXqVKkB7cwWTwMp/cGsCqptrLPBe1dr+BSU2neVF3Muu1QRu6MpWe8guGzOZmvbl17TZNSXeLUy1wUnWtFmt8xOOqRLEYVtuuSgJmJzagA0bpKwunQZ4zZ4CJmlq6zoIDAyEfa36jXDNlIncT4sCPfMipBuORsvHNcSgRi5qWMocYJyJhIDaclE0yz13kHss2nET42NeYn8eybs42H6FCynse/LeUmMFB0G4ymu8YDKsmBkIQizkJPBbC+OsAIvuZpEUkaBJ4UUK3njBAjgrij3x4aQAHRY4C4yBiT2p6Yguv04tYzDLofKPIQ9RSubPMAQazGJI1SwmMOxGlwineuocfcmaTGCBG4Xl0KzmNP+RqQc/3IyIF9gMKGbNbyN3jCVrAVhiYDBB1OhwOjWfiBDMdZEenMCQxC/TfTIHhXzuyZzztiFZShIiCS8bC8VYNqaeL5RRh0RFflWnG3jFjzlCYM1NmNyvxtXHiU8F4S+lzfLWZffPpeJzppNA4EpzKAj49/9wsO75ciBkP7ouYnU1LHA73Q7E9CMhDvR7HmSRU73SgZjlPGdA8jZAd8gR4qu8IY6kzY3D3s3xISmyM4p3ZkzCXjeTBy4hpGMyIaCo8x4prM4K5aZlPsszb7R6VWw5kHEQQmawrzewENSWhUqa9VXR1FjViV5lnAtx1v38+X76+zJfn+9q2tZzO0wRTm0W09apGBX17Pr/7+bLMbbu3upVCprx3fXy4OvWnp3Va1LXN87TXbTotba/NuriWMm/1pd83MJ8fHnrv2353B1xR+PHyRuaC3qXMj4/n83KaThOBGvHHT0/ff15//Lh+9913989P27bfXp6nwi/r7Xx+89Of/+JP/+xPP7/cXurnHz7+2Hdzs8vl0vZ9u28wFJlEzYjWl+cyzep2v9+XeXYKlm1ZlqKmXKYi0763+22tWycVMtLeqEy993ma0uLb4WouFkYjLNzWdT6fvamhCFy4tH4jslIgzN6UAG8NriFtIPLetRg5sauWaWb2QiYMmeDegE4CLuyU4mGEoNvhiXUDuc1RBlCRvDphJpHIByLC5uYmEnZj7FAiCpkV4UgYcdKC0MlmXaLp9C8AHPeg4BD84Je72xeIFRKcSTXagFlHtXUUWxiTXh6/WSJupi8xuRkkiygC8RgYj8TyJbaaiZSShogRJQYZECMzeCghRlmWKSLjS6AoA/kZUFS+2LDJHNAVmH0gIHmRAcSCrZyi59+NOQkzjClUNfxFe5JdjY8onYkq69J4Ocr5d85dj/r1QMB00IFYgvqp5LAY4cZb0ZHwEjgEAzAnxMgEDDI3Ao3VbhRlgqrGoeKErwMgMUlCI7k6h21kkHIHNHQk3nz+I1nGRwsvNQIQM6qh2SqmPmAy87CrlqPvGy1VfHxDSgHh0bxhbKzPoxFl9IEZYawlAx2sCtBo7Ub37J6fJ3nI+aUE0u5j0svEYT46xk3RGlhOcI5HfVweS9LuIBIJsYfrjxN7uGuPk+o53YpzzYN8pYZBWjDY5DYzT8QioiAiU92As5QiE6vuYBImmLhpAJHMoq2STGqVMIFMiqgZhQaxMIGhJoW3+tJ6Oy+XdXnRfde9KnEpc3cXYZmm1rZa761t83SSUuq6uZprm8u19y4TyVxMdd93NmNiU3t+uV0flqnMy2USPd3687rutFYin0+XaVlKKedlfvP+7cPDw/lyevn89PTpSRlWWzc837bv/vjHeVm2T8+fP/6wvbz4yxMm2d1e/vgNaP7xFz/fbp/P17fnmWl5eNFbq31/vvV1Xcq0vzy3MgkXLnI+n5r2dd+ulzMRMYt6h5uZtlrfvHmznKbtx3Vf71q71matR2lUzqfYUGXa+75PYfMS65IM8+k8yaR1dXdiNjXXSkJwdmd4721n8t4bc9iCQoi773AVFoJ7N18c3qWgCK+u8zRPRWIOnMLaQVxgIkdsFxqjSHMBhT4uQr9n+WPEbDruuZkbRIgAA6m6G8wiCkdL4WaBQRzgKoHASSo6SIFB44u2g8lNiAVMSe58DWKvy6CyEnQ/MFj344wTDbfnLNAI6YASvmFB7aejthszwHg/Yb6Wi/nMPWn2Yyox7nJA4YkCONHr0qgRBIhGsAFgw9jZyVwBJ5Ih7gl4HEScO8XDUcOGZO5Ie3zEiASuj6pufJ1h3A3XwHkIwbENjIExQL5D55TpPnuIMPMgAlxjpEcpMzywnES/8qfzCeeHHMhKlrf5yTDSauCBGRtHKEWMRnMY68QGNR0A9vjic7Tp8UaJDjQLpocUOHbOwMyZUWIYPZQSIyIXCXQvB0QMVyPitJZlTmWaJNYTL54FNuUbDaELJSMKjjgwB7aZKGHuzbGjOcknwdGQh/v5YAOPU5xwDYjGdr4E1XxULHH2hWDEsUqHDj8qxI5XH92If8ESGHBdjrtJmEk1Tm9hmichDvTWzMxNl2m6s7i2qVzU0NCZYpu4Rgq33rTV03kmd2utSIEnz9rdCrERu/WX5x8//OTrZb08bVVgDNLWyDuXyXuHkczl/vzx3c++fnlihLNxM211Ol2ZBQAJ9lqLs8yl3m6xixFCakws8/UNTdv1fLq+vfaO3pu5rVs/tV7X/vTpu/2+dafP3/5xu99rrfveb8/PrLo/PfE0+33nx/df/7Nfm9m63vbb/fO3f/zr56evfvEnD+/fLA8Prfa2bW3dnNTbuu13Aqbzw8LnqqRqMpXz+SxEpKybqXu7byTcapUi27reX+5FRFXVjEm4FHeDobfWahUhZhKauu29d54mJtFW922bFpnnpfc9VxROpL1qrTD3rqMChcO7driWGOXHqgG49p0LgwDTaZGpFIG7m2ouHvVho4jRgzpltwlXEnZ1VZdCzOyB5fQIZAYP1gGRs/kh88xbQkM6eSC2Uf2M+oeY2cnVe6Qfc2MnYcEYlo1pM4g5wABzRe6iIvJQBkQUCvkNMUuEJTVNZczAuAdEPZhA7kNVo3jlgYwoQblyPYAhUEipR8fgB439aEeOmpu/jNhEr3O/5KSOMQsxR7d+1Idj3QBlbWzuHnYUY35LCR0P57RgPBxNQ8x7B4R1BGIQmEa17WMZ1IjKGeXIj8rV4JTOY7E4OcYM49mMYIsD3IpPnvYEfgwJxisGwZ08wHhzDzvoRKLS3ykaJGJ311yAE01nisP5cEHOXiVD8+ujyYMVVFEAVtRcxoM+qoEQOWUzkXtWoutIH4wwz2EgE3IWC3Fm89URG4GP7370hiP1ExGHr2z2Gj267nE+iEBBhaPjODrFTDfnASPFAtlSZYHjg1/kILCreZAM3IzGX411yUPL4w4fPkXJfk5PiDF3cMfM5Vom4bsDREZFXFuZH4rMvVWP3kzEujopEphzN/XeyFwg3nuWUzBQcUUTQBVO+/0G9cv5+vL0hLx0TuZTKc3NVAHptbn59fqwr+u2rsTEvgtfYW2ei7W+b9vloZjztJzOl6ltO0hqj0jYpcxU5m2ztu211rqtdWvf/P4PhWTf18tpJqbnj9/f1tVqW96+9zLL5G+ul+vDu9Ny/tf/+s/+5M//+dPT7X7fdavf/ONv/vq//OUP3//9999guV5BQs3m81z37fnp0+nxWubL+eFtESnTVLWW06UUcbfeepnZuk3zRYRM2/Onj/u6JiWOxaTHASnTTFCQEZyZiL33DXCZC9y67m3f3DrTxCzmWmYJVoqqCoWrs7IIlcJE5E5moUspRUopZNVN6biKcIKVwszQiA6hm2Ic7Wy2p1mJxY10wKXIYVGVJYhbIsXkMla0R0EaPrYec0jOEz6UiWQZn9wCLHMtYQhjNpWJxn3iw7NEMAiAEbuJiVzjTmCQ0zECJYfpPIBcCfOFp3/Ua7k7kEBqgwKZpd7xM6+Irg+BRMLlSOzfLV3vkSz0RGnoqGmR2BoGHICBPFNqGkLudsBdAZfl3wj1VjgjffFeeAD6bg6hgLngUfvHl2Cx1xdxqA78xH0qk2ZNn+P0eDhGTggVnZtaDKkDHUu8IIttOsB2GmWlZbwBEbkasRBgsZ6TPKcQr4hd5AOEoJlYRlWQrQAFGwj5PbvbIZgd3ms8+MqOoCUMhKOUcqimPehwTCUs5TxmQQF7gt1yYHI89AHOBSrqwwbEM51G0XLM+vPjEGelEw8HI4nR8TOO3C7t5sxBa/gnUB1yakXO7goiTtUajZ/8gm/7WmeMFjC+19ijB9djyEPECa2mMjO7sHHz4jcYS1HVLFPcS8Fl5kJeey3zyQHHLngrpdS2a1cXBswYcGdYa5WYRVC37XrlEstAusvETmxO5uZKBJ2WxbR9+vjN2/dfT/PS9lu93yCFubTeIy1ba2Webp9++OpPfnV7edlvjQkudt9epmXhiXttxOS9d9VlEnbASTswsbsVKUS87/t2v68vL6ak5m/evT1Nbz999wewf/r43fr0US7Lh5//fD6dfvL1L959+PCT9+/+9Fe/OtP0/Y+fyfp3f/jmZd3u+3Zezo/vrn/6Z7++7zcz+uG7P/SXJ1CZ6kW9Xt48vP35z+blLFNZzmcnVtjMtK21tcYzC7MIg2Ka3tf7szdbpsnNSVybejw496rdahcp2neEzTyRuU3LyZWIwSKEaEkVbjItUogMujczJUq7fHfjBGwhBGaBqYxlhrNM5tpVi8i8zELUVCOMkeQ0LJvrnGcO+y1yGiT8eLlhk5CWjchlTIOSASNmeMdADPyoZoIsH7SWQAeYAsRIw0QQETiWVx6eX4PdmCOGMIIOvCgYzFFvBIs6/C8tqkhN2Ghg+sgJHA28BURsGrofziYiphQ2eudoK8bWMxpsy+MyRp07rnKG6kTHR9gbicuZSNVGuMwyOe4xY5T0sS8kSzM+OnVPz4nXtuOQqY6KL8QhOIRbQS9MIQiciDSW9zEPFwf4IIYivOdHqklQKA4AaIzQE+kKpmD+xIhnamEtz1nzJyIfFPeB8mWNjlgsSAkoRRyLtSV+HJiAoeP3Z1BFgiMxq4l/HUcXnnoFQ7ibOhFQ8kP4UUlHu+XuMShNdXKm7MCKJMhVR0B/XW55DIk8SwkZB/zQQeRf+aLlzBkFjluGUcgjHHjjgwFHYZIpMCd1FFjt4IERkampeyliPZQmSekTcEqhMcyY8qp4nuiEoHwQA4yZvCmJqGsRXGaZCAxnAZmrNbKtyCzc0y/erDC3Fk0dCgnMrO8wFVmIRLWTePDwQSSluMLVuu7b86e3H/7k+vD4ab837ewOAZfi5PNUGNRrd9Pa2nK6rLfPwtK2vVLjQuhEzAy0+52mMpW3l9OizbZ9UzV+7ZcVhjKfnGiBzNczIKXM9/15bRWXd//2//2//sVf/MX96VO9v7x7/+HPfvWL+c3l27//9uX56R/+/rfCsm23Hz9+2rd77/10vfB8+ernX5/ffXV7fhKXy+OlLLKcTjILQe63W+utd3V477s3cncRIuK67ywyL5PWal29K8yYZd+33p0mknkB3Kybdm/dzdICionLHD2pGXdTuJibqpubuLrPrt20xXG1zMhhWOxwjXBIxu4GBjvYlAjabC5zYbipuw/idBgkwHNMGeJvHK1ACsYAOGnC7TbCgJHH8M/CNMjcPfwo82QHrh0ZJFhrIGEzFZERvWWEhjioxiSjsc4tVm5p/Jn19QBYzDR3RQ1qeSytJEoSxyiZkiIRVy0H4EQOC4VqtPoD/ggU4QiCwQpB8jWT3Eu5/9LJ7RheHxjI+LhpDpvX1m1kFdDrRvQoewcuBTiNctIst21n0rWRlZJ4yQlVEAUf3cxlQO6W20ncoYMomM2OqrHwq2D44EdRAPSBtBFTLryK+tfzOZiZZ6RyPxJJtnevUAYcGASByD/5RaQJgw94ilL9BCLVgO9j4KJSilEifP5admdoHQ1bfHzNgWhoGoPlD3Lz4un0Y0jvT04jw9HIhhsOjcRuFE5PsFwgTO5OsEMkFtyeVx8398Ay7Vg6mo2FRG466gEaiN2I90f1n6VYNCX0umkhnlvm5IjnSYbKzEnIaTmlnUCgiUZmHowz97iteSwHrQ2eRQ5et+VZl2KnmU6Fn2vT1r1MRGa6yvQ4zcvaKpOzFPRWWNxNUNxURADd91tZrpAiMweMRkIQbq0yse0NIrXXjz9+8+btVx8/fdfbPgv33rgsPIl1U1OWQkQvn358+/7dy+fvrTdoX06nvm5lOU9lAfrebHbuXfe9EoFLoUHS29tdSukB8HEBS2v789PTbX2S4j/7s1/96le//u/+1T87T5NO863pb3/7+5enp+Xhzcvt+YfvPz/tTSZ8/PHl+fMNptMk+2oyy77uEOH5/Hh+ePv+TWtb77ub175uW1v3zczPp7NMCxdoa63V7W7L+Xw6L21f27Zqa24Nantt2jsil8WMpTZtyjCJphMkXEy7qhCLmhtcQdRUe4f3croyS/dqHhtijUBSmJlcLdS5PBU/ePbai0ju/nWb52kqJQ4UJYgHjDs6jLUCSBbPQSgQBrTRVRdOTF3d3EWi2LTRt48olhXuKANHWxxczzETG5VNRLzR7OK47cw+bvxRu2CsPo3gGYjT0KySWpdSzAaElXkka60jvjt8+JUljnRc7pE4fDA+M+8Q2DMas0VaBBGTWjAv8g+PlichHxofHsCgwIIQRNjDliYzMeVbeAXngdTLEgUNj3KLGA0EwFhEx0qbEVUiTeAIRxQOAomGBHDjaURKYBFT0wM1ckJO7FNOFXV7mnP4AUId5f8r/hEgFH3x8BN8SKgovwhOOXesQcw0QPl3CQxXSq7wgR/R4a52DKSj1geF3UH6syqnjS6LcGEa6zWCnh2XzI1YAuqJmj5PQPIkgwh0jHiCseVkCBJrnJTYoDhiusa8It5N0lcH79ID9MueiV5zz4EO5rNKukMQAU3z78SLgEiY1YwJoY9Iuj8NkPUQDVNmGx/gZgJteQ6diRWZXF8pyHDAzqdynfhjNQhTka5W9yc6ncoye6+91+IOU5iRu5Opxe4k773xbNM8bS+3UoRZom/nIjAjYYf1Wu/PPzy++/Dw+PaTtm4mBtMKXhwgmZxo23bgNn342cPbt59+/A7kIDJVYYHZ1ho591ZvH3+o+9mYpvm8ri+n5SRFyrI4iJ1Koa1VtP7p+fl+37/6+df/4l/++p//+a+XMt9fnv7ub//hj998d3/+dHl8C1pwe/r517/g04e1/+O333y/rlXKJLIIi2vvHR+//7jvu4sIlfZj7/XeWj2dz+Z6u21GtCwnJ2ra4HCrYqSqZlrvt/1+f3l6djfqLc41GMJcpnmaFyZbb0/WmxQZhRygSpOUeXKidttExIFuW9N1nos71Mx6N1NTBXyaCoWHFbXuOglZUocN7qqdF6HCsC7M8zJP82SJ+0WQQ6z4TEgInuIYdUuAyEe54GEJwFnp27i3EbxHUW6Dj58tycgAOXr0vOqJzGd5BxZ3D9hfzQYGRAFGpTsFIYjcwQbNuonIdHB7wkfajIjNdADLr9D80aHTCPNj4asncuIc1XQo3UJc6e4i4oMylMC0ZrkWP4ugTgUnyG2UaDkBp3zSDs/ts6oeuHviHz5EbqNU8yP58uvoLsNx5M54JoEdBChDBJAeGmeMBBB/P/BF0LF+k0BOUFf2HLQCrq9T5aNSzaQWTzuIwcELCpHa2PgW339IH0xYumn05UMHPogt8asQfqI22AR0FArRjkQXckCR8fmi+vYcP4dB4FCipKThwIzIzEs0KZx1BMUtI5ZBt48qPltCDPApTmWOu3JDhYOPbJozcWIaEpj87XHiaJDDQnlAyC310bEERefIbKNIyVwNAnF8lJzKjal/orwghOTHB3oYv+GfQHjZpiazLeGmPD2e5QyyvGARZ/Zuk/hlkrfL9Mf7qtaghbszuren+fInpe5tv0e14uympLpKeWNqnbDv+/wg83JubYMZkxKxE5MwiK13EVaf3PrL04+Xy/X5Vmjf3cz91E1jM4QQl+nsrs8fv3v79v39/tS0t21bHh6FqWtn4trrPE1try5S5kmIz8vJrLtT7305n8+PjyJ0v78w8GF5M8v09ic//fpnP6vP+z98/5vnT5/2desV95dGpPfFqj1/9+13W623271ujZnMi6mbdmZstdZWu5sQr+u27c5eWUrfOoSYaWZmId13CLPQvEhb1+V8Jm2d0NrW9y0AajaHuxAbxXYiXm/P3nuRXNwW3TUKiog5etsKw9Xgatqtd5omJoErXHtrCfUyOTkz227k6ubT5UQEUmcoC0iIi9jeCFSmqcxTNOgxnQo3Bc9pVGIYiOmdwzXMaCIacshT00dtgIqeUkkQUWxhyZqbYjabpcZoNKLApNHHk5kJM9nBHPMRZEve0FQzUtQyiSvFRSMOmXIOuQOTPFj2WYKPgnHQweNzMoeKg764iHT0DUQUOtuI5ICnaoHILFAad8BUmYiEzUdffjQR8SktCsiUUtP4GQkgXggOy7iZMSBRo0ERjGc4Cnuk+johOIdD3UaCibnuwZUkJNF8RGc3TrSDVJUzl6fRs5oNbImtK4Kz9Fp++zFDiUo0wycHRyZDXtTs8Z+UOYZeYYwomJkRKgEWVZXQnLDn0aEkvodjz/gdR/OS/U1WzjH3cRvlbzipvM4JSowtRksFEoJSAj9HmxapedTC7mO3euAuA9Hno1rJVE0+xhVhJ8JxK4brBzy1cwdolwF6HN8DnAmdOo12wAbjapTy2VkFmZoGckQASIBBEBoHKAbvfBzAmNfFh7SkX4GIGObGxN2MPTJCv8zLu6XM8N1UuDhVVq92X9gnmTvWENCEaSmwMBXiLgTVXevqTjIvdb1RawyVywM5wurIukpZ3Nt6+7Rc3szTste9EGlbSY1lhkxwMKCtb+vLm/dfXS+PH9c7O6DaW5Vp1ta07lYmM2PtvbvGujuRuE4iwkKnx9OHrz7ILA8f3k4obdv+8M03v/vN77755pvztLg1eJGp3Nf1/offnpZTa63rTpPMDyeDoKsrxNSszaXQtIhrJHNWNYNwASbtbSK5PD4S0Sa17TtLqXslhhSiZtu+trq7da02STGKS+2EQmp937Xu4eYWpzbJWszMrNqsdlebp0Kg2mthkflEBOsNFju5CLFpi5msC3k3l1KEmIkEaFqXKUbD6uhUaF4WEdLeDMrMRCV81IzgUAfMISgOd7Kj/uWsjZwJvXcGG6UWciDOEWhj6putRGA6I7P4uFx04Axho56GJKDjqGdiSzjBKYU0HHfQ8t6BIMRwxVRi6GSBXsQC3kBsoha3QF/GNR+aT4cfkklQhi3D4KDEfc36MxsmP4Bn61ok4SMWFlBXpUxLuWvMMAbHuUN4hBI30GHWn2/S/aBFZo2PwJowgIrMaREuEZqMrPMc4PDeCftHH/zXEScChxiLVnKVJ5MDwnIsATZ4rnVLHdQwWrBUOEQo0kNDzoxc1mnjMVEkz3xYx1vwbADidDBxSFtExExj9W887fDgI5B2JedDA4Ej9UczBESLlFPcgMKJMTRocBBTee2EEAMcTxFvMh1GxZ55wN3hiuCdH6c4P9L4Izq0JON3R7syDnnWKu5hOceZjX28Dc/SG+QcXUy2LfEj2Y5FDQIMmoc6MQdEkGVXbAZCJgobRcDxUbJE4wz9lAgUpbrFI+o7EZu7wL21eZoez1LIN+1sNoG7MMzr9jJPl02KRv/blUiIAG+j3cC23ubLA7MYpJCBxZoaEUtYqRffK5ap7fv99nR5fNjuT7WrkC+Xk1u3voEXISKS1up6f3n34cPL86d232utpnSazkL9dL1Y26PuECnbej9dLihite61ATcw4YX4Ufan9YePH8u0zCSfn59vt5dCZAb26X576daXR324Xm/7ZweuHx5rr89PL2bOJAKZl2Up0yKlttq6wd177+s9LFPj67ieLyKleyMgnF5MtRCKiDbtba/bhgizbsG2UzMuWE7nbbu3rZFbznyIy1wAnqdZZFZt6l1gbO7e3IyJhQQMa5aAvTlLia2+Rm5uUjhQaSFiIVRlYWayXrvWeaHzaeYwiAZMjSVLWhuoAXFwtYPhMwJSINdAohNRY3mWsVm1+AExJrGEKfy8koaQvJ2Mw9GKIhCn8Rez3iGO60bI2jQPKtLnB0ddBUfmfi6hev8ipvuA1L/EOZHQhyN/YEzZ8qbaUS3FQNQ1wwMbNJCryFtSeJSdYQVKx3wuhh/ZN4w7faALESthDkkjjZi7xih5eBYgk9xwvAh19AiwzGBjI+eMXoEkDeFsFqyUANuBDeMVsEs0KeEMeIxIUwKXxHMbySVZADG3Gh8awpLIEIZ8NcgDZiHcwesLxst58jct7S/DdxmRtJFKCAAwhMGwucX7g2Rs9qMjGYDYgDnCajUeUEZZgpfsKihM8Q3GHoaFjlDOjQxvGaPdwdEfZOkefheB54z8C1AsX0iv2FiLeUxfAr/zSER04Ht0fCOjP4QRUQb0V0w0S6AhnEgG0uDjJkgVCcPylZCoV+TJEGGHzRZl+4G4uBaSCDjMNJkABioghkGJ/O1lPk18611IiWmhsu/N9hc6z8tSWm0wRxF3TdEEi5TZtMEruXGRaZqtVYKbd5LZu4YZMZ9O5tqrbvfnr372q/P5zcv2vRC5NQ3+zDw5F8Ct1+fPP/z8618/vPnq0/4tM2QimJJTmc7P6+00n8CSz9ndtDVtbvrpj0/r6bMryiJgcpJ1rRPNPBGpL8vZ1HtvTkrl9NWH9131u29/9/b9ByG5XK/MRff9dn+Bl6ZozuS9m7cQfDgghQhdzYEyT7ua31f1RkLzNLV1n5dpKZPADdr3HVrDpcO0m0MkyxcnqDbTylCZTqbWtTOYWaZlgenW1r7vJQo9UwTczKKmvTWvDWpEwqAyzTGsZQI5ldTCq7tNMzMZF7Ju2uvDebmczqWQ7qFNSWlu6E6QZagngWcMqFJVCxCzWh+nCQi6RYKnlE5axETOuQrg6MU9m+kMl5QOkebMTnxMSUMAm0jJSDg+YFZ48vCjsJdkriBWonUY1I1z4EUZHZBeRmF8NOaXB1GTKPesZoKJCnKE9aiSORhSTGNTATJVZFNtSi6HkcBR5tpruxAx5PUpp12AucWrwFPjaUicyY9djvFBjLkwe5oBa+6oz6FqxoKDWZ5/NMRm2cUgM8qrbcTwZwoQKkYe8V0TwYkkYqRnbPMReUcPBIsadBBhXtH3FCQfSmN/lSeZKREn+RhkqscQM45ZUBdNBzSVaFsgbMMVJz8t+bBTyKFX8qkyqZr7YahEjtxOSZwadPcA0wcsEzAKy0Dr4qjF+0ym10HdNM2aiYKnFhrEQV72xOfzvdJIWiNQ0zgFw7B7qPqQjCTQwO6jQItEe2SHrH6SNTWQsVeaMzj4zOF3Yh5rMYjALERykAEomwYj5AL6ifCwlMdTIVW0JuSFqDCb3by9LGWWwsHKUygIhs+mm2lV031fW9+FOI2c3ABFiCDcIdRVPXZdtbpvT5eHh+CAmmoYxbpq31brxrzUWtf15f27n5TlzMTaW2sbTazWZJpdfHt5NlO4FsL5fDU1Zp6m0ve93u+3H55uH1/2pzsb6ra229b2fb+39eXldn/ee/+TX/7y53/6s7p+Xu/PT5++++43//Dy8YeJhadCxLXXve1ujeCAMpnAC0wYLXYCA6a63m7rywrFVBaoTcKn5bxMi6nVrTKksMCcIeLu2kwVbqfzQtC+37w1uKt2hXEhIje1Wmvr3WoLP2UDemvExNPsRtq7dWVy6x1wkAAxQAYRkTCP/XdEbr0TkWoPsfBpWU7nWXssMM7h1YGAZP0yANuxMTT5y2OARDTIEQnQRLSyCB9EUfhj+PqSey4Qh2of28qcGDRCtSWlhtQtFpOZabNuMUL0fGcj7/goBW0MITEcUGi0uVkCg4ILl1cx0OKYWEZ/wcgNZRb+aoZhcnegwVHkjQxl/oXtzoEIpRAg5GBH9vKxdD5Mb8Yjy+o+RsfDbRRH1FYzUzOLsYE7htjCe49BkKsl6hZYU/wgjryW0ZDyTfOIOMQcYSoAotFyRBdgIx/Gjx0GR0fnlKkkPzyRe3o9xVdzjMIPWllst4mMZyP3g9JlJLaRRiR0d1ONRYw+ctYo57MnjG87MvbxTvMh5BeRFn8RY4+fD8ScvMf0g+EgESIBMUBpPJK7FyJ3ZYV9rNYji4CXX3a+v1GIp0k4ucNVw0QDgf7nJvnMHKO4Mj+eGHPx0dLS640YBX4kvqBAxLR9KA2SAESIJTh2dAk+6LbHlwaoWbfuiRnGQjwa/QETIDJRtsnubtdl+nCaZyi0kiusTYXYzNvOhFlmniYQMYsTgCnMosidHaZqZks5lfnMPGlzV2UGmIwcU3FwAaP17fnj6eFhOT+quYPVFQR1jfVxIsV7uz/9eL5eHx7emHNX760G/lBKmeZlmovWbX15bq3W2i4P13W9E5P1WhiXyzwX1n0lbdNUpJR5mqfTROIE+pf/5i9+/S//2f3p88vHz8Uc+75+/vbp299/98d/PM3zT3/y9Zu3X10uF3LU20u/3ezl2bcbtR19b3U3dyJ18lJkWU7Wrd3v+9a66rRMqv3l5fnz08dtf2naTTUCgzCz8DSfWKj36nAiFwhciZ1YiFGKhOUREDqP7r12U7BMy8nUrKt3s+4sAiJjxC6XGMtES11K2p4LF0nJuAN6Ps/n8zmuCxcRLlFOvCLixFFnJ3oPJA8bThAf+p6g3tMoLs0s5qWeSlocFKAsMohG222EHC36GL9R7vg1prFwkAhhsxIFeoQAGtDDEUrh3RoIwdd2ILQFlt2Gj8LXwn9mhDvndEFCzjYitAeBST2gLVjsmEy3otG8m6k6DUdid/LYtUCS49pEARwj8fkRQxHleyrwI1LF8JwZsTPryHGJ9KRWLsYbKYX9Iv4EYh7dWL5UsLdGlAgQjBzkVIKPEQtlzH1Q+EUoQfuYWAc9NIbeAenkLx9paqQBymFPTDpdWKJZCWZvxlP4YJ3E15oQSWSJ9GIiGraA8TDZzJi8lIKxCiw6iQxiiDVzo/DlUch4MKM8epQI1eVII8n4AcHJKEuXyMXMpCHnoGi+YO4DsPShQEFUCinAi1CcBy9xtHFQQ3AbGkL46Duzx4lekBDdci6vH7KQjPh0tJF5Xl+LHpZXON9S9JWJKnxULVXE5DRM3nPZPRGrdXYezx/xhnJAEUsZXC/z9OFcFrK1N5RSqCwkjbFrtbqLzEwl1ykpHAUQaw4hJ67rjWQRAhX2LtNSCOiqVGaHd3N2zGVWbX3bt5eXt+/e1+2mpgyYgoi7uTBj341439bPT5+++tlP7+uTrUZEt6en5XJlmbRWJnJoKVzX25vr2R3EVFtTEtUutIBteXxb95V6R5m0VTZarm//7E9+eTkv/+Uv/+Pt6UfXNs+FqdResa5Cvt5+eLh+9dP3D+fr6fb0ctv29fZyf3rS/WbE3azwfDpNRVhV4R6aABEqTORe7xvMWt36XrXXAg5dLYMdJjwRMXO53z9537mwWg/FljHm+SQytVrhSqbaKrF1bQ6fphnMve7hIxNYg3aVaSYiIeJg5XkvpUhsf4MTUeFCbtZ7Eb5cTqdFrFfrnSCAOBGYXBU5afSgMkeRmI6SUeDkgclpZAynPAmLbm6RFDxLooM/6iSxYCBipeQv8FGTEh0okbM5SYmN5MdW7WlE1LwzDKds1gPlCH0UgYmCkRxh60CVXOGhC2O4GVMhJo8liQPuGcUTU25LPfr1NGYhomSJDKSBI7Ah/ibh2BdibmY0HK0jOgeulqBCzJnVwVnbxaezBFqchwWZ9yDpD1xhsL4pZgnBNkz0O8fkGAyoL3Olu5NBDygoIxYIh7Q3k3V6/Y1gPZCPEb5HI8UcJ3DwJBPgUw9kjAb7fWjBcgg0noQw5xPILw6uTiBKn06PiSYzg9mtR9dhNlCv/BQ5dxrvFAR2U5bi5sxiamPama0qMGoW8sxLLMEBhsPVu5urhaUVHEFQpdfaPBNvlCY0Ok0c2LsN0i8AzRWfR0AfS+BGEkqnC3iamXjadA94/3i3R/7KnjEMt/KLjN82ILkc7ifGlDKfMU7xEMiB2Md7j2rD3M0pIBS3fhJ8dZkeJ0Zv3hprL+YzC5vrtitsmpcyn50Lphk8xQMIywHtXevGLFNZqJTozEhYa3cFKeCorTuJmm/3z0WmN28+GKGbMYuUIiwAG0G47HV/efphWpa3738i06ThA9q7tuZAU229ErO1KoLpdPr661+cH67zcu3Gdattb+5eyrRt91rv7ri+/fDLP//z85vTP/7t3zx9+k5bneb5dD6D+OHxwyIn6fbp7//+27/962//8I+913/9b/7l/+N/+Yv/4X/473/y/v3j4+Pj2/8/V//aLEmWJAdiambHI+4jM+vVPTMAhrsUkPz/v4UiFAqFBHYXO5idAaa765F5741wP2bKD2rHI7EFSE93Vda9Ee7nmKmpqal9fn56+vLl5XIh9/dA1jxyv5ul+Jbr9ZkzxzbmkfM4OHPuh5bDWIR7zJrX67Oj6ji4pkYARsSITRW9u1kl8jDOMLCme4znZ2bNfTdyG4KFOcaAwFElqyrTWG7lrm2dFW7hziSJyxYvL5frdRCHu7jchfl0EpjU8KNyMFh5yiM0l9yhuKrOBaQ0FCqrMiuRDb31o61ghC8qAyDL+h+mgUZqMRA1/2j2oK2tsc3pL4czeK1bTDIirP+0/kRL9hYs61p7Gc4IDa+oCK7QUKZv2Jvimcha/7QD3+OPYxE8OIF2PWycT5pCCc6t2bD+UurPSTOqp1Kr2d7ktdma5UejYzO3INkd9X441XDT7Pze9XCLsZO5VsRRlg334Ri9bKtT+sL71sBZMj9vvzg7KWU7GSATFIbJ65ldpaS2B7Bz3kP50mzKSVtVm1SAciSU1ajDQ5PGZQCTOZPFsHBztk2+nXHPvnvC4bGSmK2DlgrKQ09HWLeTS0Pw3nwLM6maZKekD7FmxDTBIHVbh23Np62/sZYkADCGiWe3mVWVVka54FoDEu2rPI+vTvw5P9xXSs9X4EV/ppVzfpblqwYkCHiw6twUyv+hM9zZyU0YLYoESp9ZQz0s0ErfC+Gocp8/vG5fnuPfPo46Noe7zc23ze2ou+UF42WMyCPABCxnuhsqUTC3PO65XS22MbaZ1bSqGwyhZnkiOauwf9xv/vXp6dP77a3ud2vlISoz3IwVdt1v79/++PWnH356f/u6v02yMsscecx57OOy3T7uT8/bx7dvr58vv/zpl5fXz7/99tu3t6f71zcexz7vOWs8X3/8+U+//P0/fvn5y/3993/+z//rt6+/uZmNC6tmTWMGrkrDWfHt7esf79/evv7221/++9PzJxYmaY7r8/Xl8+fbx/txHIU8Pm6qjsMvedTTy/Pl6Xp7f7t/vaF43S5+fc77QdIsqo4EIzaDvb+957GL2Gs0LAyY3OeNLObkMd3BPMZ28fE07HKrNxiG8SjxwNV7dWvK2RuGbYSboeiAgKO0kEZczF+fnze3uSc53QNWzug6kj3+U6ZmLx2ICLQuxWvp5zVyvKKrtW2iu6gDyXN0OWrWcqGVKhGNoLvXV45IVqg1J+HJ6HPszf00cpRMEPDGWEtbInZKZXgVI6xQi81sY2WsfrKbyciip0FXDQGcRnXIRuHdHIe5UGbONOEcpmb1e/2ObBmXBEohWDlCqqcO/X3tz5Xxut4o0AqmJs7iwfpus2DaeQIB/UbW2e3WB9JrUSLXB2CXCCojao0si1VvgH6SWueIQk+wcp0EM2PLfvSJFzhdzQGx0y389xYWgpWpcue0T2UXDrCl7Pd+K2YBUK1Q9T+6TFKR109LUnZhWq5+NFne66nF/ChHMsEGEjR3GxbWpthEoxZUwLO4yCy6BZl4LJugtrm1uZ1ORFHpw/0U6/YQAh3RqZ4C8soDPQ/ojUjW7jA9z1WhmhfSzVfRedJkECcnDsfgRIac0Lk2H8Bk1bp+GFbJuQJKVZNrgHnALHTOqs0T+o8TaKvVYlWM/dPr9tPr0/br1zuP4eOY+7bZNfxg7cdNpOe4XOZxAOmDvYY402Pw/jHHdvl0HbaRxuPIPIjjcnnhpOUsM7eIy0Dm/ePb5fX15enT233P46BvZsFK+EYyRhy39z/+9m9P/+H10+vn3+7vx21Hcotxiet4vkagjJX57W+/OmN++fTLL3/38vnT+32/f9z2+x2EhV+356fL5jH+7V//j9//+m+33/8G8vr6Uvfj9naLkAfivTjNwgKREeTx/u2//n//PzacMTy2zy8/XAyz5n7c77fdiiOClEwiw7fLNoqswvy4edUYY953H5H7nlW+BcyfXl5oebt949zB1OVJ8rING6GbWvcdlYaW+CXx9HQleX9/H+Z73lEyEz33TFCA09XBqjQgqsB5uWzgUTkDuD5dny4Xq1lzutuyPuuFS6JjqgpuNTVfb2jdG08OvNl5ObYVbcmG+ix3aOEJqxUvjAAW5Y6HFuKkPUVV+zrH3TM1/WpnyQPTvluMARgcUZW92M66vbnQmGKN7H/RQJa9WVDYykxO+gx3sZ8mO66i6iqsQrkyRYjb6mF2qS58Bpj3Di/TZnOzQgUebUz9AWsmmo/bb43z2DDZmh6u9ajNSIZ5ViFa6Kn3jDMJGB533qB4QVZT3OtPiWGQnYP1wsZ+SgSsuqfqPSHbflD9NXXgmMvUjI2/O76Jc/fMZGc6tDFc25JqZA+rgOlP0OuU0cPD6tusKGiajnJrm3vhWj1MGX2vsA6DXpqMCAHtsUkaMNi7JZaSgPJyaJigR+dmZeKPhJp6+kCxXoZO7lYsP2VGSpSrJOkhOltSXJ1vrKOmKFytaUMzRsrX1Ypg9y7HV/XHBeMNkB1EgZR7HXsphC2fCLmk2nkoa4mgtWW1pV3LgGoNGdqp6RXtq99x5PPz058/vzxf/rjdc1aFe+YxcNmsJvc6Rlye0whM12bF5Q3gE7iOub/Z/RpRPS/uVmWY+3U87UwgYV5H0jHf35/uH9en529fnWRUITZUzeMeMXxyjMtx//j2+98+/fDp2/t27PMyNu4FL+kekXMex+U63r/98a//O98/9h9/+SmAH374fL38MudBxH67/fpvf/vjt9+PeRjn8M1Rlgny8rwxjypwzjE2gG7DthHu89g/bh/zft+u5c/bcdx/+8t+jndEDA+nxCrml+0C+n4/7h8fdcza949vXy3CY0jUXoQZ3f328VZzR6UHqlhFv2wWW3jc54Hagcm5AwW3nBjb8Lgcx64BEbAq0w1st1eYGxNgORm0bYFJOaJUacQ/X15enp4uIieYFeG1qlELQ2m4vV2vmncxLk4fa8WruHzFF4CrF7eCubccqZVCxVwl8lJzVkkkBpqF0UBb2k05tZgE3QUGmso0X/K7pl7dS/qYbtRB7TJ35Fw+l2bffyqjLYLWemKgmWoVx4vyB9guL+3EwPM2+0mDdwRTo7CyRLV20xUgGRaKSQvw1tnKaGL9oaayRxDsih/aamJnPujft4i7xT417QSDVXP7duaTMxWiaQZ+R4ZwwX/FIhaiSfluc+o54HwmGrRSOKNbdJxpVZWpCmvuBx4eFCJ0dxc+hEYNKssivNqLjGundX9OCQGoQUPvVKMq8+F5g1gFpSTH2SlKH5J9HB0AB1q6AJMRNEXnrC5HB9B+WhqVzq4zC2t7UGlvfR904kGPmbtlFrPKugtkBY++nF3s9UtZGXppLPRJu3Nt3SHBd2cMq/ggVr3Tne6+GFnVm+SWONeMj59i5xx4n8v+9GxFbDudg2qGSJ9XlQH76dPrT09Pvx1HkqwZpJmne1Yec067WcQYozhhG60qd48ols2sOc3f/HX4NrwIgzty3u5F82C4WxTALN+u9/vtusXnH3/++PYtj7o4yz0l23CvzKz59be/vX7+4Ycvv3y8/fPMY4yRc9/8UsXKmfux14xiJW+3j1//8t+vz59nHmNsgB3HftzuHx8fNemOEVbACIyxAXWx7f6+25AaI2hpESyUGxBPzy/w2J63MyHnTHOEjXCvmkAW+fL8fHm+opgf78ftxnlj1uXTa0O01vEfL59+ycy8v/N+A8v8yoL5oMnlvwjWPCxvomhmTfgYl2cCyQNM5F6VqPLLsBhuKj3LAwYZaYQmjXSetm0ABwzD7OXT6/XpmnOSkhL48jM8x/3XKWnG3RoD9j9l1w2AAARAa83GImUXp9yjTc1aFNaQbdvHYDXROgagWOYaOBAN1ZR0eE/ym6bV9JOhZfMKuN+1MNl7VVcSenybQp0FxhloFEvE+FT7onaIbODWokZNzApdMmVcofFQ12YR68e9iAsAhRw+pOhccaDOMKOntXYREPAeD1nDpgqsHS/6EQDgSVVxCSZXPFKK9RXy+zU15WELtC61/pJU4vxLVcKsciAzF3jsPx3Wmws7gpxDsYR1FcpC9XoE2GKM/FxVqyimQrVxcrH/eYe49XTaq5Ur9hfQdn+i/opZ8r5Eny4zUeu28usKde5DJ9XN2tbf7SxOdchcaN6W/vf0DOw+0UIkWE3/Jtd1Y1SCGcLDWFnuTUGpJmJfJK4Xs8idk35RVbJqGaxFdMTi0QwEtZDNtbrBTFPUBlisgoZKj+vomq1vjUcJotHfKmv3TL08bWS2dVy8wFH55Xn78+v4p6/7nRwxPG2zKviOSt6NwxkWXhXF5Jzlg0iwNDq5399sjM1ii0vivt8PM5RlHonhcI8Y9AKsZk7cnl6/3G8fc9/Jypz0KkTux+XpCZNz7r//9t9//PnPn7/8/Mdf/2qebhJu2RjhL8/z2Gvm5J6o28d7/gAYb/ltXC4fXz+ul8tmhutlOPf7R1XOYx7HEcMEfyGdFQ8dH3OASadZZOaxI8KSnPe6bE+DrLlnlTkvz89PL9v1+Tlnvn39/e3bG2ZZsWaFRxgSTKaHP73+cLlc3/742367u4fFRg7a7jGyqszryGO/BWBSVthBwkfEGMc8jtu3LeiFY043Gi4ycnFo2pzRqmbvEdo24GheNC72+nx9ftqKO5EeG5sCsAVIxbdADi8RXtlkTV8Zdsho2fMSTmqaE2yvMd3GnIk+xLpcbuf5Yl8anPo59gx7Fd1aXmK01oqw10HK9qBxlcCNeVYuc5sOwotmgsHaPZcik0V9hDqQXbpbjyvrZrcqscmEMGAZFDf4LnR6YhPwiBCZBotWpyggZlEVm7t3TrFHyHCDHqCKsEf5sWKPmmYL/vdQFR8K2h4Noywf1JE/Y3Q9HrUio/gazRKfCcEAd8+FQrXQJqItHKRSZ5UWeQqbmgFLMOPSrILLg9qljsLZ+SCwGpFVKdW53CNMfUp0D1UQXLvhcFr7db/ejGXuNWXyA65JMRBuoAcgDR148h7rOZokzo9HqVENdb2/C+QPCNGesdYDYw1q9Ku+ewynyB9WWaVzZxbRU1Ds3vr6vAmgRW99WxbR0wG+sb1+MawZGyx2zyQ00OCla223orU3m6lHHu4GSeiah1Xa5//pNwKkSZ5ehp4nKM6sOdOc+/7x6YI/f3p+DvXsK2sWZyAvhlEVeXDeLd1dvglhwvq618zwmvub1XGJGOPiMbLAhI8tD6ptpMu8v9/mvifz6foc25jz8Oswt8yjAnNOHaDb27fb/fby6cvl5dliO4r3j48AQB9jGx6A3T7e97d3h99+/zpvOybn++26jXm7H/fj2D9Uds55mJfV5L7f3r9K5htjg2lheh15ZCWBWZOsyn7Jmcnc5+1jv9+qZlWZ+eX6AmK/3W/vH3U/5pHHfZqH2dCwdE7Q4unlBVb78cGaWoOcPOCjALONEhJlIY+ah6FqJiwuT0+kVd4H6VYu34gSRTDEYsvghEgxhlU0sPLwQAwHKjmvl+35+RqOyl0qFO2lbOsCWFb15ciTrqXgQ2mY1nud4dLAdd9BWKla6anrch6yziDUbxQd1edRIdHOuMJzyGYByAbs/RvUhwjRrmsHwCJjebZ0++evwZiF87Qvt5mlc+RrcVotEjL36Guyph9WzSxAqLjR5kuhuSKAxt6U3uUSNffa6j6QShvrL8UN8hxrYH/kFbYA69+1KNoVUVctVmWnJ7bB4CTWtOi694pO1Rq9M3ChMSGLzLW1xVvbMvT1w0OtDusF63h8OnucgJNFqn7qJ3rGyWV1uIVMikIDT5ITSLlD1TM9NigyRka067ETKOjNiAdcJwT9K3szEammaRe1mrLSWFxbtmrLqQRTnSfX+0ZTnqeOygzWCzMWBOgo2fC9W1htG7I0n6jCnMR5uB4ltSrZIHV8rS0SCWj7UnXe4/lsO6XJcksHePE07vJbrkpoMUU3JNZvXNseYXJ5J1lYM3Qagutx7fYQpfyAdWpZx/D6+fPzj0/D62ASYcmquV+cT5s5d6u5Lg5j20w5wFqQjMw69v3+lvNmNJhHxMzK49jGhvKak8sR5Ljf79++KZhqgg4eaQpu0wMRI7O+/f7b5TqeXl8npg8b23i7vc+seUyYh4/t+ny5vsS46KS5W2zDHOMyLteRe877O1EeEWPQGz/qAGQlHUlm1ay8z4RZbFs8bTHaC2FzP/Z9zru2a3kMptWs+31/e/+W84Dj4+OfLIbB5v127LOqxja2cQ3z437L+20b4RY2KywIxuU6LpfjmHnfA4V9dwgUjjGul6dXC2fOrKPygIPhNGq/agHHTNCR5Y4x1CrNYeWWIE3rSHO/XsbT9WJOolzL/6pYTYcSaX1wCvKalSYEwDLhXXLRDkMdfeqEwkSRs3RPbE3MmGhyXYiWucm6fklN1s9S9EYjOLPepbo4g4ZHC6lJbc/UQdP4SyO6JlLZ+7+rR6XEap0/Qn++p1Kzvenbc+1BiNkZTLt67za4RY/UNmnE78KLZuU85P3KImOsKVOcQhoVXtbiKCz+7fH9ev5/sQZceYCFWqusThnQmR9ElGk6tD9v9wyUIjrzsp8GwGVBaIZ6dHSwUtIi2RwnnpRvp9S+Sx0g6Aoz6wmwx38qmfZ2GjixlEVr10uVOrQrkYO2zOo6wawsTEiiaUXZf3TiaIZf3JitWUIj3NylQxZiV4sb3bWwPkc9xrF8PjTq2FXaklh1XeCmSmvl+mqhUmP38yU12aexyU76zZHqVEIgZVGVS4vWnRzpy9q44jwR3aDvG5g5SZl62UrtfQT1qm19GXbpur7aCTtAc8CXpGB9cXVPcOw/vF7/7tPrBdBIBInh5jMH6gL6PObxFZjFySjfhkXANWxoOQ/kPj++zdqBul6eYmwxhgFWs3JH0gq3207GuFxzHu9vf1jE9ek5xhZxYRWKlaiCWWTWfr9/vL9/ev1yeXohE77BR5lnlvlGIHzABy1sDLRTjddMjwH359cndwuPET7nse/3OY9Czjz2fT/mUVVZM1mg9iIlK3NmZiopOzAiYnsCHRYWw9zf39++/f619kkQydfX/xg2DMVU4VAIPL2+HPf7/e2tjvYgnKS5+9jMXK8Axx51uNPMUYBtPq5FZt5rHkAVcNtvCEO4Rw9VVSZJGMMYAdXXlRkR7n7Mgyj3en7enp9Hzvelx8cD9DSBoF6oLBp6dvikZdfBrnW4FwzUaVtj92j1uEmfvtC9aY+hqFKDm4Wx94305UHrrRUdq7LTDLXkJhvDKn8sGNOXsCHsObC7Kl20+EiFRBPhVSHwTsPyqFjtiKbxT7IbS24Heb1xkfHKa4ZwC7VqqxYvJUK8DDxLJVWKnfWwjIi6eav/Z+tDCIk3YG0EvaglDfEAphYa4JX6aWJgFGnCenzaFdPa3ADfx72OkJ1uzxLKFrxvUL9mNoR3RTmYG+CEtVG2t52rWCbVOhqehv6JWQdONhTuHCi/sg5dsiAV1NAH8O6iw2R3obhZk2R44BwH06db1Vj/faOtnb4ONN5HFw5uFoBlls6ZnQkWCo9maHdvpdbqVn7XCyCgvnb/QFuarZOzkV5UHQVtbZU7ijK4MnQupQX81L+phupJPO+KR6/JrXq4rGst8bwiHB1ucj2Cn2m+vXnXVxNRiq4/unbqNMXZfUNzRJg5K6uOT0/xd1+eXwZxzFlFs6MqgsNrG7x4BpN5C0tzxgYP83Ca1QRgDs857x9fZ96NDBvuiLB2b3WYMczAOvZ9v99qZs3DAB9jXLcYW/goWlXO/UBx7sfbt99g+cOPn+PpCrcYG+c0+H6XXZvYUuYx55w+Io+dYM0bkJU7K1kpwczm8t0c0EIIknPKETnU75us3mup0tJg7u4FeER4cNZx32/vH/vHrea8xjNnWk0gMw+gZNg5xgXgx/v7/fZ2uYya+7zfUFmZTmex5vTcL1FGuoXBEO6XsW1PVch9H+4Acs64DkP5CLiX0YnhAaZhorQCVwS3k/AxirUf++WyvXy6Pj+PmffkdDcLp4OKawiZybhbLfqYEkJoOBVNVPZ1t/Wc3eDISrMuYRV83cXVnrSB+EcX8SFBDkocbkSHFoX1FlY3GGoFRLJKMyIdYeEwCZy6ktBFbX2nSZSJNVImxzxR4eccpmFhWMUQt7Clt5Her/G1P6rqVcjADFkp7KgE1p/F4CPQvXXtHHHCQCthiDVMB1jbufGsa/SZS0JUEcwP5L/ipzfQNOYjOQuDeoTGAQ3Gs+JZNjMrQBvw0NtyYf+Gvr341s6OrPVaQXTmWHmctL7AmlfC0nQ211DWrUnRfdJc9a/tNEtUMbPQMwSdfwx9yFoUii7BVuSFS9orIUPT5GKxVVzqvS8mXA6mBsm1CqlRRGXXdAOdCwKjlW1AVaK5pXrYGC13hxJELwKWLLhZhML6FIE6etR2HbASbyWbcr1QPzsCXfOYL1JvASF2U3ixYbpJ+owt2QTOylTFmLV+GesN2/mHqot8aUhhYFiP76IaFxarusfiWbv78acv11+exkCBTjNZmwbqytqYo/aREzW9phFju1gMj4tfNkeYVDO393n7OvMjRmzjyWIUM+eseQCaJJw5J7PG9uSOeb9vYxsjPC7Z9rHh5u4xtsg5P25v1+vT9fm1vOC+XS7bZRtPVw9j/+hsZxZmcd/vH8dxJ2rbfGxOy8zZQNEcxGbbiA3s4UOUSXwSkj40IkHmrJmZWUd3oary2O+cyXlU5e3jDwyHE0YfMS4R24jtEmNk5cwdksDSLAIW2/USl820krwFNOYWSc6y8Xy1iDzuuR+57wap9AwO38KHh1nlDDPmAaN74yi5+plBwwR5zBH49Pp8uTg5Q+5oSZN5CwgsWyENAC9QEsv2asUFVmHxplaiPaUDWs7Dsplqtxa1HA2iRxLZFPVafAiV/4IpCkWLnrDFbGi5mHzrorf3GrQzfWnC3ZvLFPJq00Nr6kP0cF+fUlXZ7/0s1GEgs0ihnPNarQTSTKcCnLwa3e0x/9ux9MGtLBpKegJb1MYZc41kDH9AcH14dDiGO409ewu0J49ZeKwaq7n85WMWTcScob6jJWyJaVTk6SUt0mSBWnVXxEsrh6oKNCyuroMsiexE931xAG9yy3AqkdDVwKKUm3sPTUeZh8cJ9s/ORbZav9YzUWsdCk8yYJqVWKbZWAWf4PIKeaSkKOLieixCJ20lPgu496EhmVUEs7ppI9UXRGc2LdORsxFD/4tVxcpcpysfVJyszxdBpgNZKKJNp+20DUGfjX73PafVZ0hMnMG1X1K1j5mPEdZGhsu19pwh7APQJ61pKzPQqtuFWo/cuQeLc1U9r5Yl4Kxy5ucX/P2XcfUDNY2IMYqwpDOvVpul123UZKYRDvNQw8TMg+VMWmHue+4fmRNdk/marc4x3GJs2xYex/vXeRzbdds/vm5je7k+b9erbaAxIVB1MeK43Qz88vqqRbjPz68+wqwYPi5XkFUJh8c4jmPOOSsL3EYswrfkIerA3G9z3+e+cxYLrl5e1cyZyX0elRMsZiqKqPEHUEB4HnPm0f5hrKzcRoyxoR0ZvfO8xGDzcDNmegRsRGw+ruZBZM57zqlLnpQCZfO4HjXnsfPYgcLM6EDkMYZtPuvuQfP0DWYpVTvFmiDDo3LmkWQ+X7fnp80xmVk1I3r9BgSQrKGGWD41TwTBoOkhnrx4nZfVzdGSzCY/VQfI2EDIrpb9pjjJQqG3o8AM2nNW2b+gO8MKNDAsladilqFJeUU2k/+bx3exVdCTHcHU8n0ALNHZru/4cLEWdWsulNegry3RmsEi2gJD10WBRkNGCpNuD1S8KGXNB6St1PP4j3XHuZza0Dewb261gNC+z1Duq3vbCqa+u2am1YjS8hk8IhZJjc7T1RgamjlSjOf6vY9PABiyUHOlTDYVHwhHdBGs2qXOhgq7hDu1LG7VxFUPZJibeuAEcqV6BVF3P00oQTwefJWtwGjuSiGrPvAzOnbyRtMXhgffxZXqnJmQfHgdUzMwWWLAl5EFVDasD3JCg5Mf7Gqo8QIX+Qes+UnvzoHCaK4HdFYf5KpqYy0YQI/Rqc8u5LTqoHM1jzgn1crUltclgxXskGiuA9PqB3XQX8MyDreKTrd93IqV392YdsGSFAtesP24fb7Y3/3w+uP14kcKscmRaAMvwSfzq0WANmfN3Rzhwy9bbCMiwiKwaTI093seN6DG2LbLM2KQE3XUcQtPMlGk+XE/aBWB29sf29N4ujxdrk+ZyeJRecyjivO+f/3jd8R4ef4C4z7v23Z1H7f7PS3HtiHL1h2sEs6N3GfOY845973mIfMaVh73e2XW7FvenEySTCc5M/cdIOeBqgDdbLjJNCmEt+fhYZzzsl3NI/MAMDOPYxaxPb8MG8fHjbcj73cWZtG3DTESmJU5d2AaKAlZVtolnj5/irBiampXEzRu4T4iNqdxllnRsmxGyAR6OBBG9YlihJsX05ifXq8vTyOPD0MFjNVKU0lN6CBKjsWqd/3UYBuaGFx4sVnEpVVfLJF4Dnr05FVE3zU0/kIKCyofGBEGa2MTgwVkGCaAax3kdYMWEm5cJ5Z0SUgNZ7NF08uoqpmVyeJqe9bZtBDM6uLgnDw4mZx2f4HGG5qJ9zV3u34E0X7y7u4p8ZOG9SNwNgV7QrijsSSt5y9S4K42AoIOq+Lgmj5j+7d0o7eJ+4bZalia+oJOakEAqzKzF14+Cp/11yqX3FY6aoblVPgAbtFoWgal5mHe7HAE1qPjmglXm2BWnvJKhbxumrq1w2njZVb2egaw7fkWR+FsHN8vnavOQefxToZiSqpqeSeoSGnn1UKbo5r6AoZh4bYUCO5Bk0dmWe/o1Yh2SxLdTvqMId+eldrceg5A5JTTzMNG3wN/LE4FjVViArp2QDudlDeukXS3lQPUVl4hIXOyIdUaldMHpYVX5Rr37WXWZ73cz7hHSzTXbp3P6nGGvCm4nlJWHybg3R0vlduaE3awNuePny4/v8avH8ft8IqwiCwE0ogwXjz2mTC4Bwh3z0q62bgYyo+q7OWKdX8frO3yeRujeMwCcg6PPHYfWxF5TL+M29u359fX+8cNNsb1MjJ2qRuTO+/bNsx9v9+3y+315bnm/vH+YbCxjUuO3O8jtnDDnFlGpgMebrR5TGLOY+ZMV7rtsXWyTocMyZKMqJagWw80Gsy8WOURkhVmJiAHPjUFNsBrFsjZfu582p4u1+f7+7fb+wfnYeGEx+XiPmieM1mp3lHA4SxMguN6DY/j2GUpYcZ5n3EdHqPbRUlwhnvmHJfNwIgR0jhmmZv7mFVeVcUReH15en65Zn0tZMCNhTJEiKTtvE4aAr1rpCUehEw9F5Iua2LZeqRd1P9SJRS6cmhKMYshRogidt3MsjJ8FKXfN5VIZ1js+rupo1WyN8XTemiU1dptK7JpjRm4wRT5yXQz7aVaxnVQGhCMJx+TL1wE0sJs31lK+EpHHfdWGHZULspGAG4R9uJRemy66G5rcoEkPby9iPu5nQYRtgJyX+WuLJSwQWuDm0U2aXDTbebUMH7pGxHFXI/DuuNvJpSsWFqrmlm/TDRFcIHuatNvg2Fmnly2yWV2GQ2QosRTz651uiblPslHZtUHU9EoaEB3sq1u1jBEKNj5Kk2qqDdvBndX8hBp0hlxwegCYKq9fNUuFeYAR38ZAGBVqh5XXLUirKsnnLQi+rkUl7ZB9KySwCocm8uAc80oc3WZ+BgiWOP1LR5ukM71RxtEQetMF/Ba2EZtssZhXJqidUZVqCxUs349AIpYVyuoV5KhSjqKTv4t4qarkSfyZyUwlObXUKzk/dPr9nc/vP4fv/71yGkxjsyAwX1zM1hVbYgjC1Z1TA6zCD0zbsNY7pZz+mZ1vCXm5bK5Pz9dnz/YKqiadEPEJatY0yKypod/fPv12X7ybbvwmXPOo/KYCCN4ZL7j9+fXH56u1+N+zzzcsMWgGcnhhirAjRhjmGE/dlbWzKrD5dCSXRsR7hFh2+St8khkmcGjDANqaKfKCcugOWnw6MhXAGtsw90B5kzASMs63GM8PW9PT8yqTK/ajzmetiLj8uzjQpL7bmao0IyHsWjlz9exbVlV84iCo1DpHkaPbQMV0OoSJkwgy2ZtGgiVLsQ2QtbgZnkZ9unTy+USxy7HQyncVSfTDJXnWKwpgcGcnAAfAbIB7YL96zgbV9hs12IC5c6cMmKQ6YIt0wMrMsLVdE+1QnSlC2HGELAswIs1NPGrSMFyhOSPXZIDMu7VQqauEORVZzQfCtUNs53aX1jrmJOrEF600qwcERbGrB4fU0zpP6Pw2iFYY1DS4wPaLz+TKQzXA0Q8c0lX4ay1iYY9zy+6XVrG1eHoedNutwPnYCsEXr1VRrBFSJn+EJaw0EG0NBxrvaVYKpZGsSgHVluGMz1ZFg9hJfvpuFtm9sIYnZ3G4wUHHlN1KrJWnvGlwtHP6UFea7ql1bBGloQVmSVbbxM86RPT/7paMybihCwsVQGxKpJkz0ibBDLWNL18pVYdKIZXuc/cKG654zGWCBNi51G2ILVpmP27P9n97rOEzIJ2y1RzLio2ezE7FgLRiyVzcVjslweuMqyKJeKaa65NjdlisVLydnSrZNVF+rGLzRP5ZaddhnlfDnSFRVFW7DeZRKbUFiSbKnPzmjz2/fnif/7h+efXLWpy5giDzA2KQG6OEbUFUJOcqjBAZHGCDFiYjTAxZcft/v7bnF9HmHa0zFQOKGZuYwsPFvf3D0PA8PH115zz6frksdFqbGFapm223/bb+7fb/UPODaQxxQO4Utecs8NCVp9k97DIfZ/3IyHtNrWSLHHAaB7hsUUYc4DhBqbej5mOTiUqmcex55EAI8YY0eNwSwESPuLyZOB2uWTm/eOdgF82j/H8+UdDEDUzLcLCpZWp+8wq+rherzFGzp3HgdxrHiDdXZvih8fw2IYP9yCHu7G28LVYWPIqIxxZDpvH7dPr05fPL+QspOa4VOr22dOE1ALDwlUOjNhCndJcgb9JMpxNqhM4YxXNXbET1pvbO5r04RRyRciTSh+hHRoXGdHURJURa7qs8WedQNxsmSHro1WJ+ygt22i6Jx42vRDvD7TsW3+oiMyq1Q9T7BP7IUGfroOCh62htr4y2V9fip31W9jRcDWl7aSzzFSIL9TfjM6ZVKN3FLIJkK4lOjk1pbvYFDOTNLk6XomXsjCPbi43GLTVJNVfXJRGR3MjltfTEtRr4Gglu8UPLzYa/bhNDMRqSGPVKs07LQqxQa9+DpsFInpB0hKPweEjes0Ze0BE2VFB0mydsDPa0c8lZi0YAwu96md9HphxiPVTuaG1zgtfLyCx6ju9D7QLx9knX9la51Ajy2wMXiw96gadtsa2Gv7o2FvPA64DoZPvvirNJk4JNRGKPkK/otb+d8XlfhJ9qEwl2EnsNAxYN61fNgzddW3rC3WECmW+LCt0u2x9SeWwVdcM8E+vT//uy+e/fPujbDXHjQcripctMlFWI8Y968DOsZnLgtUZ7kk77nmkmeM40j5Y5Xa5bi/TMxOVN49tzmOMjUZWmvux3+BeeeTt2+SnMbY597zdaWY5x2WY2dxvObNyemxHTiw5dhl77yh9HrsBlaQVs/U/0pLB3MXnAzkPUKEzWBzwgFY52np4nLJWrWL7FJGVBc7D6XQYS9ZM4Rp3wKiUKIk553Z9ur58xrD9tvN+IBBxqZm5725EFBjb87PHlpXk5Nwl5EZTl44COSNsM7eaZBoR12HIiAAIY9UxwsPjfjvgNOSnT0+fP7/M442ZFe5cUaUhkTebk1Ik0D2K5WtmxUyKly6LuSCpn/NcvY2LpnjvZCZWl6AvmYkjOoaPPXdzdw8288mmDeTLoLDFxWIu500xDpLKFdDG5pSNs7Mrg8VWoeVDXBwBk7BmgmQC1ww61SVycw+EgXOmfS+oEBfm5zQb3AypnqKtYLJ4IUM3IharFu3LYd4bdNt5AgufKkSO4VXJpR1cP4ul+ZtHEYDe97moLLBN3RfPg2aOm0FiaykfdUY1H0dFelsdEn3qSskU3c7ugsom9TDcLfVLu8ta3/1ertJGc73KIL3ruCWKZ8xWhVB0Q1at7iOzsiefLBpkNNAVNEYXrWTXdpDoSzoXOjwzu6IzVFUYhkcvvFxv0M6WKRownA+w3dm73DMrMiyq6LEquk6rKC5pgK1ieln4uPWUgHD4yvJqBz0eltptleVCNKA6H70mqUlCiiWtzv2tMugHvW7LKTJb2Q/r09Hak7YzXtGkzm1oYSdPheWqtHKhmMJE3vbnMf7ux6fPf/l6v2dG7OQkbJgdROVlDMBurL2mmbFixOaGg5TqwMbmpFeUW2nmdfsGYlxeX19f374difJwnWYPZrLqcB+k3T/e68jr66cRkSArnYbyZNEDZNYkE3TfRre3qshyY85jzuwcnNNZZOU8xrap3REeuiKuAAholtMQQoTW5GkQMNsq0+hmtAgdpGpfKZ85UVZZMGfW9fXlsj3fb++3t1tmxjbG5dnHuM8djuvz8/HxXvf3qmQd6WVbXPziMZKcx47cNSBA+rYND3cfZuawLXzZjnNsgZlxDR9GznDYUB6cMGTdt+APn18/PV/u91+zSs715j24yrKybLxqMvZxiR1nUlIxDZ5CrE2b9Kh9JdpAB9vB/p8orAnPtclO57VqROhAKp/ojaOv0GIRFl9MMkY0WFuz8GjULE6gWW+hHKUEEudVlRtPxCpL5FRaRVh45FIdDY+qjOHlLvlsZvm6hmsIaAFkNzOHlzVb3PoThZeGzQ/iGw/I2p987VFZAbMpXLHNyqK08+LLdMjNqh1fDFgTdgp5S7fCB2DjCjL9aHt9lKAkFN71T8sQXGwNSLZhczcGmqUwRITeY5W6fU6QB8qIpdlaVNn6z6zFYbEj4Xq7lWW957xJFA93va8TYijjW69+k3CSK/TJ0QiNQNBnSSejy2DhbcBsiIQX+VNgdIG02CIZ7PUbIUobCrqtgGKaOg9AD9L5CrzVKFlPpCceCuoxrOrMO8ewB87WCWFDC/hw1kI/1Q2hPDdeyvMX640Aj9lxdAkNPFq+3dIo+lJhsGm6Mpc3N9c/4HePTOd1jX2DpUOQyREz75fYfv60/fnT9bf3jwzXppdZNLcnt8y8GJJwlu0fJIrweHIeR5ZH0BJj1F7mm7zSjrffp98+/zzGeHn69MP9/sbMmYfxouzsNkzfJWfaPfcBG2O7zPtbETJtM3DbrmTO4zAvh2gSZFKS5pmZnEbO4/AIudmObQAU4D3qkLEjEmClLelAZU1WmyGGmbaqqHpIizBroUX4IJD7jqJZ+HDziMvTdtnInPvM2s3oHtvTM4BhhnHhfgvW/fZRNbUb8PX1xU0KhR152DyoaQaNPTnGiH4t4oJnmTMMoSVEc3ZMthxjKzICtefry/XHnz67V9UhfXwr3atMDqJYKXNFDTmsBVxj4iWPDF/D7M3z9AHWpJsKR4Pb4pCzVo28uE6B+0U0uZbMgDQPHTwjHeroLvMVsvupDyzVzTbxrhrZMvOmW2t2xU4YPcBUB9VgZrqk+hq1aJUiYWv5rZq1hEecFta1llM2JQKrykcbr3/IyXusOoRnABJYq7ZcPVvJVFbqRbi6vCKu3eM7gwnDQmL9wNvVB+1u/R3w6zGFRYCYWVbKfqeWF3V4KwHRZA6hDKASCYR1wVsP7kU9TSQKKpv1CtpnuPfMCAqgR5jLfPUY8N3vOz+u4lilR7vJZpvTeg+jKHIJ/D/wqymvhwjsQlgAa7GBcoOZRkfM295u9GuDpg+tpPLXgdKXBBAA1pRWqkck9XK/z3D/rsOsL2BhfvZo9T1F6bYR4ndtE8ElNLRZ2XnxSotic7Q6vgMzYB7Oak2Vh3L1uTSA699Zz9i4iKbHvZZxLlaEl86N1v13PWhfRYIwgyCFSpwgLVnH/vnp+R9+eP2X3973Khs2YuzHdN8SAXAeu4/LBTxqIm8wDeS4qcQOJySe26TwnXlkHm/f/vb0nJfra12e5nGbOYdN0WVjmPklczK24zjAj9iuETHGZc4dleZj7seI0aclE1HoGq7XNhQLSLK2YUB5DPRFNjoyDzfWnCa5iOJiVYTD2+ieScMEiJZZY2zDOGHRmMYN2ZeYZMAj4nK9htV9PyqP4TGenz1GYbLUWL/n7b3qyH0naNt4fn7ZxrWyknvNO3ggd5zBhLQInbBkWXmEytaWHUvyPpxWEzCPqGNmHqjj8+dPnz+/zPlRzGjawhXUhLUVd6wBhhCAtSyiL+26tQ8tfotYrBGJLcVhO8U8xBvdilNfJHhOn6hfp5va5EatgtUWRcHeJdA/yQRripKuuZA34WhhRZf5YvN7YDs7Mq/fcjbGCEBQXhnDHLLXPU0uuf6FE7NLPzKzBDSW0VuDdChVtX0C11VSQ4KLTG4wfApb2QzNSg5dGvS4Dpywtd2qGwtNiJx8nD9oMzY1pOUqy+gNqvx7X03/jVVClQg0ngFHT9IWN94MyZnZ+syAXEMWqzFu61mIMVtUynp6VmSscm9lO7Zy36qniEuQ2rIHx/SQUEVEK8G8Hw1O0onov7P66n2Q9LkHqwyNytHN725N6N1ooPGEwy2DbVjxfWSUUehi4xZuUk7WFEinFgXlNs9YQXrlQZW0XeeiJ3DdBxqqr5yk1osMuXreek1W9ydX8ln1DLoeKlT/rZXPz1c5s6yIWDdQlBLa3FsHQOBIRzbTK3wEKvfr9ekffnr989/evn7d0yzh29gyeTi24OYbk8OwOSoPic84Li0MV2G1+TymhuONFVbz/e19v19ff7Lt+bI9h4/j9raN53lkZg6/bM+v87gNYx53gMwBMIZEOBljZGaMEaOkUKycAVIBorJydo6smVkHj4htWACZc87cxwiRquZWM1eSFLUMp0aQ1tXTzZ7T3KsSNHUGaTRDFS/X4T483CqPmvvtLaueXp41oZ7HjvarPuq4Faf5tO31+uXLMKdZ5se8v1sd2vGmAUBzi+3iY3ApQtwtzDJ65C4ioqkZAJ5VNjX9M4fNLz98/vTpuh+/VtUYQ5dzhV1bAaF9QIiTJtWlEjUB89DzbUGNn/8aWDiNyU5a2rxlxG0ulJRtmhKHWY/2eARXcdzhSSIee1BCOGkWklj+aFg8QIMbroAL9ijDWSu1BydBWS8Ye+OVnJ+t0QOzpV4sPuh9rP4Bi9Kw9ha01bntWCbQfY7DriLkDEWr3hfPtsZtTspYuajxWP+XLkYeyRVNXHcYI/4HPI1ldr16IOvfMa2hMiN7W0lmLgVtytRYrBPaxUg/RRKbhptuIbaq98igizgskLCIHznV6megMvUPxK/oPXs4aZVZnGaWWdUqWYMm0gkz817T22arxEPM02lNKj+ljc7oK2rqofekAgbUli0pkRr0Nm9VapNrpEC/WrVVHwKScFI7Hbp8K6z6RK+kSf6eaTeLZed+vl7iPFEdSPopc908OTz0nMW6XZQnFtBpQFMw6KMOO5sv379xrBNlwJkUHADUYIETa8Mf+xL/n9KbAVKJuA1xWVY8oo5PL09//unpn99vb3PCtotf7pxHTf3RcA5iVBV8r8x5g1mMC92rpMEhhnGijhnbhXmU1Tze+WFx3Mf1NSIwnoq5bdvt44Y4LjOenj7t+0dFZFVlwunZfZfKMhiPDBjhNdPCcx4kKUaxoCiZZYRXpXuWoXJqbDtnEmnmc06AESuDK/6V6nAN3RQ1B5tzJhABmMEP7m4uz9aWaYXdb29ZxZwjtjyO5gLmfry/ewrCTg/Yy9Pl+XM8jbodx8e95t0yfZ3n2EYR27iYB6ssIHMFi8h5M4NFhFuEydpIuzXCh7XrVD5d48cfXi7DPz4OJ1aAK3UMziqT2eFe8sEeYeio6kUaElg772ASJoT7dwdyqcKb5zXKp7rP1xn/aCDMWW0j0+MubBJIbsy9HtXcz0FjOZ4pIXm3i4WQSjpLW1VCU7EMollUFWpoNKkTXiyx06Rux9IOqnvcvYu+Y+yv1svOtCWtf6IhlkvdeaG+G2BW76Gq8fHCgEQrYWE911rnUAKbobIzx/R97wrp7JZglRW2cssZLNXh7Szetakverf3EyjJmAsvmtt6C+fHxHKI0q5dqNWzGAIVBf1+O9gqL7EJxRNqCAmbS7lEc7PwnBnd6zwrFYZHVmYuEzjQ1gove/yYzo9uXo7zE2hb0ZmLlE44c3h/Va5T0HSaJHEwVKWudrcEV8GiD+W2qEFKW9KPH5TzYnTA6JDfYrV+aYsnPF2qFrGvz9nP0NVZWtmhxKShjYplvmrW21a6dF/ZpN8xVxbjd2XbWZkVAYwRmbIR7+K0C5n10ddr6AE0vXTQUAX3eewvl+3vf3j60+9Pt99y85icbplVSXrYMEPETCTTIZvSg7CiS35IuFvA0uSOG4M1Hc7jYx43zPeMa2wXx2VcxvN43W/7sd9o3C7Xgscx5353izx2cx8jyGJ2u0Mt8zpSnHgb/DKJIFEIMiOCmcecIUuAornJVKOKOc/Cjo2ubJEGeS9fOEo3TPFqGyOCRHi6DxGxt7ffiaqs55dnwq0y557HkccNqcED87HZtl1ePo3nwSMx93n/QE7XwMue7gHEtm0RDrcYmgrNcGfe3codA4gwZwVQWcNRrNgGc86Zx/H+0y8//PjlE+rOOc1ROcMJDOZUI3CVu+y+WbegzJZOpR6V9YmdvqvMdarDuNiGh3KDeUYnlcPWTiHRAbWvBw2NKwtMZFgUq4eS0NR876EVxlQct3U+25bHsHQpsTl7Og8mmVPxhIZoyVBHCXJxVQrpDnSPBF1HgqffJptUVBhZVIyQYA/5a5bQSlag2hVjeo6C/OKTH+BQcrWlqulnJz0+1rSmBvIa+Gt2qW+qLe6iu7tdHmnTWMPYlavWq/Gu8vt1LVbokZ+UXZqr6H+m22DJVCsFjzkBhfzW/rOqRVYerF4hJPHROQxCFdYGGsJCfLNKm1lzlaMY3XluUk53U/WKOJ9OLiuC8fynirlaLGM2sLbaa1huaUYpbyBfPKsspUxrxgw1090jVnbrNxdNUKoE04Y4PedoPK+v2OyM2uuqmdbT7MKiOaaFvDRS2lCi+/IzywyxJiqUBMO8pzdFM61T07/CFspA43q2ZotVYsbXmsuCPT4UsEDUkgdhpWYvMEhk5vz4+fPTv//509fbH7/PgjFhqfZRTliguA0/Duxzuo08jjL3uFQm4zBciqCDYZXUfNMIT86cc+63qne/XC7XT/tRI563y/Pt442TtPn0/AnjarDKCTdYAeEWxZmZBhQye9mQQ+1FZyVSPr2SjYNCCu5Bg0VtY4tLVFUdh49gVeak1A3LYsyr1zXaEs/BwHlwDNQh3ZLHVjMzjzwy5577fdu2/f3NYaycc1pJTFM+NvOI52ffLh6+3w/b7zwOzunsmaYYmqneYmwR8mOpqrk5at4c8MGLD/IoGyLBRxhYm1R9RdY+jD98+fTl8/O+f5vMDYgY382vGEBHT4GxycKFTuCGStaiE+rREmxowZpF0sJ6DHUS0lpnj25FPJbiCpJ1ZrGQbFkgsR17lgG1IppoaUkvl+atkR1rVdGE/oid5TARERBhwHQPmJ8nv1E0Cu0T2Xempykl9e3pL1hvVYKZlzaSwQySr5SG8JUgbWlOOpD09ff+yO6s9Ca3nOhmR2V5WJdBqHOESlmqqmxNSXQgXsW8biUW5BdQGREzp17Wmv5dLOJiMhaDUnD1/7zIS3hl8hHoVwY4U4G4CV/bHdrtbJ0Ldh2yPh49Qq0UfcKZUy9f4cg82ExXZzZzYxZgcmeK4bWkpStGFqxlnLbOnsJa1WNl8fmhT1KOq+wcEVbZlYmKHXZGg8Ey86wKWfTol6S35WZmPufR1Y+ShBm6Cc/FsKtB2GLN9RnP1Cl83ZBj/Y01zyapUWc4erg/VApkIZFo3pzuvih+nvvlbW0PRet3Fc2tSw/1vnFawi2GqIGZwMuD8dJN6+3DbTGNJA3pzqfnyz/+9PLr1/3tb/csY9YAj3ZQ4LZdOHkdXsRHklY173C4eeake/gGs2XqxyFvc4ZxwtzqXnfM/MO3+7S3GNftepHZ47y/ZdE8UAVWzgny6fKKYlayk2k1w2lDpl8Jejc2dTjc3MM8a+YxPcIczMmqY9/B2rZLxNZmLrWMqdBOr0aYhThBmnHfCUvCrk8g3DmPI/cPzmOYBwvJebwvX4xPx/3NPMxje/1i14s59/3O2m2/e5ZXt3yqyg2xXXwbFvSxGTKPOyVBhRlzIAxpbhbhCpfHfbtc3eM47hYGzk+v208/vF6Hf9xuAQOcKXMArRFWB8vdY9ENhUVVNtpyycm621ksZzOxJ3RYvv2AQbddM7EgZh5h8aCJulcJWi/nkjaGy3/F0cJwReOw6FjWuUMVRlsbNJJavTEmLZZOqYrMM3wCvbwFJjp+5ZLVGl5R9PG9fMUnwKqHP9g1Zt+T5rUdvpqv543yRYWd9sgwg4dBGpUmdqySZqdHRSdBtgULAChVdEvhTHHrEtuqyAgZZNrJYYiLX0+7Q3OneSHC8MU91yqMHoyEnoqcDEqRNCsiSoNy0fmyH/AJGM/HZYuJIsOGquW2PIKZ2cw0IGJAMiSArJCggXA4w6pmlRI/T/jfQVuWJ+u3RIzS2k4Shl69IIGMByuHCgA3SfUbfXeoM5o87pa7aWZF+PpaVkUP/VON3fKRdmHkQ++vY1spnaydHFbjCyyFRSw4v+I/F5EjQhAa+q2CIWJRDrKmkTYuQSzzE0BXR8miffUe79Jq6RMKNHUZWiq0ChOucbXOCugHrYpJyaDgEUVkcjv2H57iHz7Hv/2R+2EjPOl7zZ2FtHAzp01u27jnXjyAcIThUvPAcIuL0z0ii/SsSQv9qgDM4ilikDmPD/JeeRifzLbnl58Md9bc39/HNjxiHgdyHvt7a2vbH7gqkwmaJpHgFhFS2Ra6bWVZmTmPfY/LNvcPHamxhTHA6pJcHViY5KLQTc5D7DHNzAfJnMfMqYhWVXnsUWVjmDHvd7HLrMS45NwZcdku8fzJx6DlvB88PlC7HQkiPCxG5eERPkaMiKGgOauKOXvfBxixsaYN9xh9twGX84FzbHEcd+Px45fXn376YnawDgejO462mI6Fp6QHXYf69AWzrtN7FAikm2eWRywRZAehLInUabCqChm6uJNTxQgW/cJlmgIWEDBLFicZHDZo1eNsC8/1HUXfTZG8olPtMXwLcK1uR6+aEr0Ojc1j6QL1KbqfYMs8zla87umBFXA1OWwkkOURtdC0Dtx565efWVcmposjvdYZTdlqAsV7ieOw4nGRw1YnwNZ7Xs9hMVRo1qjJLpF1rrUEgrVQs5I9MYrqDYD9kbXbNaEAVeRpH7TwfP/JRR9o9tm6PGoSQq84usKQo6bbagBRr74fDlkoh8thUM2SqtLCWqXy1Zb4ridFQNUquV4clrwIvlrtqmZchKGbmcuDoZ+TQR/VzQa7UGygbtIjt/cZ3QaQ7l7FqoRai02VAGwvDT0b++4oNxfMBTSKMHiEPd5c165gv0FdpK4RDH76j7ZDSHdXzM2/c8nj4viipyea/jTrgb4HfVdF7XyHlyyLPaQA05vUZ7VWfT5oopNRO3P6Kq5azbVADvd9v0b8/Zenf/3b7evbXk/PJC5h96oDwHG/bNu2Ocqul8Ejq7KOuznDPG93XoJjZILmPi5IzpxuZraZASyDeWw5p7tx7qhkjNu8+fXJbFwuF0MV/bJts45MBdlakighLAFAteoKvpE1j53ulk3fmdv15Qkge4pSpiYEs/IAyCwzL9NNg5lDk8bKLCNQu8FqTvMx7+zzqO1lWShetu2Ycx7H5fIsGLM9v4ztybct815zWu52/9AIt5aXEhWXzQzb00UmZgYwkzOxJ8M0pA9QG04ac2qS3t0jjmNGAJZPYT//9MPnT0/7/ZvqF4YDSUS5VGAdwob7/6AYrIbXizJIruIUYJif6mHSzDyZvgpcGYhQLoqwY68IX+TqefpP+npdnd6tSKVqhBJEwQMn800avXpUD81DAQC8GxWPoMm2WdRqUV1HicCbdOC6EbpgDxLYlpnz+nwKGCwa6Wdw7KHBIsAsD5cNovQ9vnYZmkE3EU1eGwxM9dfMYD4CQGXXCtY6F80gS9u6fPFXbNQXYPtVrM+CU9HXbFgxw0eaWWdistNTJ5CqEpch1262906/JBUbaoFEGFb5jDkVJ5W0tPBDEdt7ZI2hMXLzyjSJiJwyVi2ujqh+U5Fgz/ZJjVnFog8Pk6Wugr0I22qcuk6nodfIrPmDWoVhf+WVTHxYmLM79ATN3Hpeu6sINVZQZaHlbSARWjbPbot1ArB2Qlg1wGoZuWyhhPM743GJAc5ycp18LKRgeEATMykRFtpS9/bEQ0a1VbGQV6efZpzWo1mEVKHMQmgiPBqD9K3P1b7zHitfKgp91lr81bpny/nEh1nmNvLHl/Effnr99V5/m0j4to253yc5wZEV5sm8OGewiDoqfY94KkPNO1EVmxsYqnUQYUwU08IkOYhtY5WNyONgzfS0SoNFDPPBrBhh5MyMjbIXMQ+Q7oEkV0JAMecBdxs25zSBOIRvMtFJC6+jgMr7JJk5DQTLfACVszRhqJVvTYOa98wf4RFg1p50WtHDC5VzbpdxZPrYNh9ZGLGNy9UvTwTncQMn8maV4UBOMmDGKvfw6G13VYBVZSGnZWnzV8RoX+gweOvYFMx6OlivGfP1dfv5h8/bsPfbTaaRfSDZHDQbC1lxMb98nB5VmLn8QRQHRfj2wLRbjJDzcLHGGDnXbtIuhUs8e8PHBih0686kaG6hMI+ti00zoxOM6PH7WFePjTkbldPq8ZGoqMeOawAhD4ZFwnTo03esE71pCH/dZElITlqIktkSTQQD5z9qJ3YYtFQks07iVeU4M0+DJbUNFEYX9lpumusXd047L7WfQLbnfOQkgaU8r9QDoJlHaN2Y5rdIMxaSaf2/O+VwKevNNL9ekPsmTgQo9rtxLuX2s4ZCGo0DZsg64foioboJrZanCeYbaTbOQ9WJa6HhNUSivgvcLLv1Lp2xC6h18bS0WI9BBq6oat2/OcuHE6rrPQ7Q6Jp9ooJC3wW04Uk/lFgDHfpteixnv8BWEaaI3RWw96ZPnQnU8sPuV2tYdasia3EF/gbYq+u7ppO5hK1xzgR4L3Dsj6D6CcvT1Cof/RHzwCJNPex8Vp16HAK2Qg79Ddd8vWongxGhg5WVvgoE7wErjHCrvI76hx+f//Lt+P1v75u93HJu4Zg5yb32i9klYlgUPesgMcHKW4wrmUxWJcbm5jEM5sk095xSAkQnPB8EPAJueaRb+RiGquMGEwxBDBcQKFbtu5nLES3C1fdWW9EBdx/OdR/BlFNcaW+QQyAxDfIjDGtPN4mEfO4H3Tzc1RVEWQEoUTxkouAeqDSzy9NzuFs4J+9zH2PbXp5HXIqZ8+DcDXerFMA3DzFFcfGxbfDGg1rJaQZPYFpEbOHmrOQID4cDVulu4WUyznUO9+J05E8/fvnhy+uRb1bTVntn0S/UzhYQRcaDUMZZcLLN2E1WaSfsIlpADRpZEQ4y5YxtbfX+IEQcLRy3/tcBFBBY46wFyAUIzaDakmPIsRJFSQzMl60xUFke7X4jLtVWY6uHYJUDTscF0xgNe0prDdsD8BGdDNqxbjU0lZ9IAtLzwFGzOhOKiHEHyqiG1hkB18ITUDtIDH3nV2XVsF3GlvxOWSbOXcWO9V7D/lQgEwx3mQ8DVixm0396cnIr8vCcucCPkmUTXKVFs4s61oFZd83OwowLrq6ni1QdUv1bqrrDERHH3FfierAJRRpp4YVyH6c9pq8ewImMV4JUwNKOIs+qydmwu1H06s08So1+ln2yVp5jycnqfB2EZKRMSFCMVfyK+xbVwRauupmtEUarlrS15w8ruYxS+8GA303QwUy+yrotbmjdqZ6vkHcD98dJOP8PWeePQZ8/yP9ZhP8qlQB3bTs0En3lzHqZwpI1wdoMlmCx5sw5ZwsAINSj37oapFbozeBdB1alNbzsI6jc6uHHkZbHz8/xDz9cX7aqvIO8WAy3sEpg4gCmGy6DG9OsjNOQqL3yTt7dOrqR8BEWRjt8C8IqD8mmCUK25uS2bQZjHmS6XlHN0LotVajNYFXOg0xqp1FVJbMwqzjTUS6LURJZbWU/RfFnl5rbZiN8RLGyhG8rq0xGxwrrOJvr3re2rcI05jvFMcu65/J0fXp+jois47i913EzHrXfmdkH3weGWZhvG4yQ1LcqjzuPxJFeFmPoitb92MKHmYNeR3BulttghJUx2itivjzFLz9//vxyOfYPpXgPIW+LBTsd7h4sENUTTn3CDGuwlKArP0gs2UJFVNXaP2Xwc6/6wxOdTDPKTBx9nNokqCEqAO2wNCyAD6BkNdPFKgs9WkEpQ4TCPUzs3lkeC/LYCgadp86h/eUmcNLAPK3eBI17tF4zqa7J1FgTSQqEy2IMBpOURUuKYDR3A22J61i1kDjPzCmJPdf8JourobM4F3RvEobqa0ecuBN6OpkzaevjdpRg2+W1aLWLIXaj7+Q6eg6g/+ZCyTgljt7b3Mx6a7GbhaZQ17dTBNcKNEKDWta7FlQOGcwQnX5WIjQjSwY8brDeY7iqshMcZ5mZh3cDtT/w6fzRiaqkAtO+J7XsqrJS4esUdynnKDUPIe7+TY3xH7i4D0zbeaznYlh6HfRikAb1DzS0oneJO9Ix0qx7uGdm/+T+N06ex6DphpXjDb0gu4wI1U2y7Ktzsrk/2lmrN+bR73Ol9XVmWiHKLGu/1158g7KQQeAamsejd6O0BTtr4LPJ149CqjybKV1ABt7+9MP2798+ff2X90kjscF9uxwzZyXJ7eIXs+frxtkbg7SEWf7GsJhFjy2PwyOk2WmFMRJHWXQXpNlu1pzpeZqqG8fQvgOjvldklrpb6u+ZaRihSY+1oLtMGzPK2iNpJr3czOSzyOpt22bj0ht+BQBQmrJeBwUAsy83ilM1MisPc+Pc3ce2bWbO45jHbd722IwslyKEGgApjzFGqIdnTC3btSSS4zI8PIYPp9MMsuKezmIevrlZWgvnQcCdOPLT6/WXH18dt8qblXXP9uQiCUOcpGRP8JLWo159uJtq6BmrDrJUX4WYdUBqLlH8PZTXpu/NgXcDWbs6DJoMOrt0vVEyuhKtJOQJpi9iZt6K0vUh2fyqTnm2TKXx+CrQF7oTfMQ629YivaZLqzUdJzMOsYb6w0JIhBfLLUhW5RiDq/Q5v0NlyhFGkp0HJd6tSpCmOXOq+NDnKQqXLjisL1Jqi3i7BnU2BrAKAnM3zv6SJGIEoI5uajK2+5dYuRxY1sU60j0mFwMxgmSM0RRC51dvxKBvYoxwRZpwq1rDa+pCUmEaC8Xb93hYg8nZfI6tCAaAiSnZYkNfMkZUUQWOARZudNVD/ZyXfBldsHL9OrUGzL5zZxUTuFKGjfP7m+v6Y932XPXcug6nAkyEikBGrUJpXQCQrpq0T+vSIqyAeRJ8hXaWVn/I285inR+2KAPWbj8odi5BkdoAjFJ7ckGFzkCy8e0qsUHpmiY+VQblHmZupbb2KYdYGbVVTFx+v2oqLBCoJ+MG6Ui7oaS156zcf/l0+cdfnv7y7X2+ZVrEuBztocYs2nFcxnZ1m4NVeaO09UPQkCTp1YRMeQxDlRfr4dOrTKmgaih3kKlFJOTMOU0ckRbzoGGsi0dkGr1Y5IRZZnJsnrlqygLMii6oKARWyUrQYO4R2yZDec2OaLmrJsSEdsASxijr7pQ5ymMUOefuHhbIiZmsKtSU1qZyxgg9ZteCvggpO/JI4/QiSAdsmAUYKCuGsWob7iw3GDOGWgSbCmRWgZOsQP7w6dPr62Xmm9hqWwCzkr5cK6w7YMqCASJBgIbQjPRZ9Z1QdOkXXSHGRHNUb4FvqNDn0VaoXBIgo5mzEB5y2XC6NS5GGVzTZOHW3pBuNG2YMfWoFMECVsJH1oYzdoK4U+pxnm9pb85Aqs+y2r8PgmVdih4mpLb9qe7QfIJRHMga1YEO4SkUQSm+hy06vz/Piqw4P0AHMNUcDrThGM5/j0t93tFF2FGbrEhAKFEHGTx993zNMdj6vgqd7K5vB0AtbCHMTIJyaC2WGdfuHbT3RoeA9WcEC9bWKIcDmUC3oDsv9ilQx6JTt9lZyqjIYAsx3Fo33wfStWXNVB7l+hRqGygmKJgqMGIBaH1byAbOG9haOGdVcfRD15tE2upa0IDvLIRsvSSBhHWKOw5S++yqq1XZaRWXFbOtN3XCECO4Rsq1CIJylIVUK46lwRKthaYgu4bu17CM+lb9YqZJNJxaBazoU932EaZQPjd1pNF1kwuqAvI7BAnrz3bKK/QSuRKEd5GmeYJJN5TDcw7zyv0fv7z89efPv358vRNlrJyX2ADPeVhxP+50f7JhcZlV95nwhJaOZppNlsncoIpAEm5elXPptpsE7Ar9LFQcRWQdVcWUNc7oKBVm7shDjuQwpxOCYDVbvkACFhESAdNarldgAiOsXWQIcooeZCVUJzvMCEMV6GUl+C8EqVswzTznHbjMzXHMM+IMkRIWLPMtzN1HhDtRYpqI4u2O2MJ7qZo7MTLG5u4hxp/GTDjNwiPcIpNgXXyAJPena/345WWLuR93EgXzhEU3c0WVEAmMXp7ao8DNcFQHJu9Gn8OEea1H4fgQYICUqAbm8sHGSchXrRbZ+SZBnkNXKsrSCfMY6ty6Dxboq9o2OaGIl/Uldu47aabRofXz9WhXa+1MW11E6vY19eGrRPkOinUwgTbpVZnFGqaFQr8vocfSOD70o34CZP3Pylqu/W726AIa+kEbvEWoKpREJLCN01hVbQgEbZDlOZEnUlowLmw1N5QzvLENBNi7cuJKOpq504UBmltTXWno+A7QHFxqK/RsLdYMKfFdmFKyYaJQpxO1vhIAhxdT833NlwhALHwpOsSJEpb3fq9rK84KlcrQbgAFed169KU/oRrF0NC12bLUxOpsDY8+cg6We83yICGLOa58qa7UqhlVccHtLPlN4PvE4Zrulli2S2zBVZMBt/LfKtwqJTAwoRLdvVZBlHr5XlxnpI3UTRAL68wqTuHB9qjpYcvfX1MOCXgf+eXGzkr3wW57yA3eVpmunTOgSuzq3cd6fqtKU/MuDEV4EbNokVfmp8j/6cv1f/u3t3/52POeLHPLy4gECrnPqty37em6XXcDrfbmgit8zJx0AlYw2BA0Ma2SyZJvgIhoIjUy3odMuJuaVAJIiwQ5qwlfPWXNQBiDIKuMazOsrdKzim0q1RFktcbByjzWdqJSlV201CkgvVu4Rqm/hNXYBvolL2urCTdrD0SaD7DSEHCzEcNpKKYa48ZiVcSmaoDm8NSbDONgWdGdQJqWlLm7WZJGcwpZ78T89MPLj1+ea95ZM8xMIIvmMhwoE8peGN3cLHuclURvHJQqEQ2DCHhxkQmiFBcDoBGZBi2lnoGYjDVbuGiBk1MyQwxVTAKHdMnUULBwtznTDeaxurULn+kD8cxDpp/f7dNarj7dWV3Xdv2XJcrGGnxsitxC+i5wGaqhUMXhkeZkiYhl9ohWf6K1DbDhZF/MBeTtxI/eZUffvB4oWuMQQr5m8Fx9jgb2q6mgv7JKMd1Mkw2EtccXutttZwReCe0sdRDR/qBYTHRmPp7MSYQVEGuutFFpI+EFC2rFPKhiUJ+mfwhPQsbQbhDVBLR5cSrGd89ft2JxM0ozYf2POnO1zF1+dickRM+X2Hd0BXucZX1nY9HDMWsogrl7zUQrz044YQQqM9xOfsbMKuWBV4+HSUifRO2cbqsj2OKYdSaqEU7XQsonJmtkC6xxCpxlA7Bgl/itKsLVbU8FO6MygCi65WQd1ndSDeZ1AtDLwcpY/QmlC2qudjG67Y49C2HrkPRHUMlZCyFVpVJqCzPWK6fZc9iL5y+f7T/+8vK3//p1N6ehaoaZ+ziIYdjnkT5982tYFZissskk4LFBxht99GCt1m53H1SSMBQTaSnMIHTg4MwU26WHUGRvbSoxRxFAzjRjk3lCiKsSrQUyWC00aezCEm9L0Fg6giy0HYKiGNL6Vq+OptkCcdNMYk2RM67KQ+4u2gwT5j7chJMrWXuEc1I7yrYtiPSABso2QxRdQqxKXb8iN/fw6KWFy7kWOH7+5Zen5+2YX033U58wVIW7wZmEicntVoRMuyRd7klJ9GloaSSLZC1TH2354KzCWlzebanGtVKXVynMNV60xXYK/4c3ylbHyQwyr5disiM9JCL3x8Ws9ZOkwzHEiEZt1qU8u+OxGh8qX21lhUUHsSPWyU/oGyjg0OCzenmAiCK9fegh6O9X2aLrhQhdI0lmbl62luwtkpkLuVWl9PLobITziXee7jAe/Xe1+KB9wGxtKoHX//CjGz5XoQfAzr1drKxwX+6cpDXrK5pIpLFGAiRdUQe71YlmVSZjO4dnTh9mj7EI685N9c9czxdmgFtmmWGJFYWaVkrssUPDGhPLFptSUXGRO0uD2p++RZjqDp4hFYuvIgXLuu8r1bZ0x7B1brqiWgm2zGahiOpZUMIK2sULQmG/OGeu/j26QQHrh9ZR3xb4X+tQyfYPNa/zVPYaKTSU1RCEPijXk+mD2fOHOohF8IRvj6Dd374kb1Bliu4tmy0ghceMMMz1Pvvb+3JB14fUmQGb9ehb009ZEgaY7/OYef/hCf/40/WXayGPg9NHJKuYETEsRgQna9Yw29zDCUwHWHDSCl6XdihtTF6ltgdwSqN8bfmtmjrZDUt0EAQHNPiGNrplJotuRknpdfC6NmRl+48XeueigOs6FDRjtwb0ENWE0F3WbagDOS3bG4WrnDe4nA7EmmYehVS9kZljG2MbPlxWMZmZx92qbE4JONyHGWI4jGHQsuGheywtVdFoRtf2RFfKplXOytvn5/jh9RpxzDwqD9XcerIyZy4pX6s3/WK175ztfYIWDPRgan+5PvCqiPpVKZwz+85h4QhVFfozejiZIrgEWiXqaTaBIM2SWS2ul1ZddM3CUj1+r+yjZm2/Tm1ZsXXIufScOOv0VVMDqwfLDi6LqG0yvS+GGZZ8j4CEYP0D+fiWpr0GJ9hUtLUuMu1sn6Af4BkrvuMPVt3QXqR9r1aUVBW7lIiG9cCwPs13H3iNyzV7jEaiWJ75+lFV5drBU8g5zx+bmVn5XUjspyE8SdkXfveNLIJg9ghFd+ZX6FJx1g3bM23rM8m7V8tb6uwXdPp5mD0vLEp7BGgRc7qSDST8u0/ZpxjmC2IrMZTcHtfz1lVfxuXWJtcAK/vtkmUnS2f9VAHKRIDtcmX90quPmfVhEFxaNR0RNjrcqHxAkmWasaH8zGqVAY07gGXZ8x33CZp6FdRkXW//7hcC91P1ZcYqyFN38VIrpJHi1w1GryYGirQqq3PcZnX5heBK+31WQSk/CQIF46DXrMqyYf/w6en/9vef//Zfv2npCImsGhgWqsoOkvO4ufnVPGLcs5cFztpl69bpvbf5dd4XvFATQp3DbkckYdBCQc58MHlhps0/q2yl8El73rKmgll14xQMGLSZqGFenXfLFn+gHNB3WydspdCzyO7a1thbkrqP2+LkCEVZxAjVDTn3AiF/7p634jU2eFplVV2eYhtB41TzdtbmmxBKjBiXIbFdFo7cL5sXZ/jx5z9/eX2JeXwzgHCdeJEkrNSs3FLrtSq6KllqKa1Y2R0/79DPJZQQWGAZLJv21aHTBDbUcJ5zttMPi1lt5iXGXMqucx2XhvWdi2MQceJ2BsGVWayFaHn2tHXDHd6DAnyErBWkldqk9u0dx+ymhmZlI/vrtjAEBtU77p5ZOpTg2rgi5a9HZTaT4/1kHilEWXPBULP2Y9DgWA8NwQiMiMxc7VBd/P51pmsOdLnbmRa9Qq0WZtcO9WLv+GoGGU2rkVUV6jupzHFzd1mfERoJElg/szWw6AxALUMUiCw38xHdwGF3DEX4SjHlS1vp5kChXdG6/Wkd1OoBu5XXlFeMhjAjcU5wdjnZiycUhH3B7pUq+qicuZArZnT46FJ19PTdIq96Cxup7o2Zt7uXNtOhqy2eIUhiWKPF2bsPmGUegshSuOkAmj6IkYVK3W9bcgjVLRqY7xPbekxSTh2d0hc6EKLpRN9gSKyQwI83g0b51nUxKKsaGCQJtfMdC5LJS8HkXpswzbIAocEr76erFVhVtQJADxTqJrDSbRteVVvx+VJ/92X75cX/9Wved+MYVO3htTGM3DkrK1EjhvkoYJ83xNUNld9QL+auhWeKCzLjs+ZXxKHnmrehVEmqq/qP6Guut0ZinUxJ7nqptztn5ilcUwxE53/T8hBlOjMsA0KdgtVe6DyhEVA+ylLASMuVlb+PaB5C6S7mu6o4WWkoI0YMCQ2HqwrLMMEWek6a7cysCgITXpqTH25qUxXI4SOsjMeX1+1Pv/z4NGI/ppu5e2WNkN6/3cu68LfvRD5mYHkvPIPB81RWmFHrogjrti+SRSrTCTRIaUsxDHX+FGDWmo/t4NjxxcX+mEgDpVFTOpd3g531dHMtovZIQrNNtpgXPpC+UJrNjuJYJp3ClNZe8isxrM9li21qCMFeBdT+cd1gVDdXPCObNvTwxTwIC7RaBosE786BriIhhxxNocoeRz+wuRY8siDsu/PTlf/6klx+X13BdPBjYzw5mPqZBnUfpFiBuKR+vKo3+5lUzyj0lICtr9RQePkmaQZVP6Tn29jB8HEvGnaeusEFbEHBuH63LUzqpdN0rvKGzVaY1vxiwbBV0BmWVTnw3UMgLGUB0swnPKwyAzZUTQjEw9ThLAA120POzRCuU472OhGd2clozQ7DSI+hjxIWPSVoi+vvAGXMXvVgbhGWxUKvH+pDYP59hauzutQCKttFDSqwtImpKQixJUktuzrnWKx7vFxZ8vxVi6fF2UzoyswBoyvCti9gV1790LpdoizgZq5gkWURg2CF5aw55k+f43/6+dOv377eKzM1hX8M87IaY2QxMqtQTA+7+vDwnbsh3K/6sFUKH7aKVkJjW0ha24uLklrcJRZF5TJl7GF3wNxDQvKVM7muSEgL0ZNifTQlYZNZJrVzTglBNRDVJC2p9s6uJrF6DjBqvYTR6JSfCMvgZpsbnebDDIVKMmlkHr3q9xTGOIAMt6HPPAuDSbJyjOFGY4L0GG7MtAMZ5lkMS4Jj2I8//fDp8xf4R4/bdsAtOiJMa0B0oiuBod40vQOMjtUCtC1ScPVC2rLR2jHmgQr6pAC01KBHpjmaugBlBKuYEBZr3FKGck6W+7AH6MGJn7HIjj7GrcspOJgdRwF6GHMV4VzWh11ANxR1qKyGi9iVJKwlvWiWQbXzit61loRn0fXLUolnFScGnHxND6/oV7RRpfDt4plAgHKaIZqFlouRnXblZ8z3FevBNRzWxIi+1JrNXDe6PVxX6FDSMmjZtYBzk52mXWAiwrSOTYXUkkFK13TWYD6ruHqSip1mev5KJOKbC1xaxzNZ0aCmDj0iShvOG8q2YtTMpDs/WQae295X/lr37LtQ/xAwm1kL7tezaz5q1Rjlqk7JYhF12tR1LLBV9WH5+WFRnFDFLhDLkxYCgBbYqxsF70m7lS7Do18A5NQrL0e9sEanbIq/WQ+V6TpOZtaycpFiBgNX4xJNNjXbyNLM6EI4yx6smWE9Lqd24uIUI54VVoMs/bpib0WppK4UYHBHLDjWBIFa6DGiYFWcWbfj4DxeN/+7z9efXjY3FtKHw5HIcNDm5naJiwrhzAq3raFvBTKMjtKHdsBRxgKrcuroiNjRbDMAs6AFETTtR3AFbrWMtVNQqhouNFiVpvWE5tVo4ztcKTnHyfqLtDPNfFrpXUDIy8nVVYIVoh2zM7V2IsEyIrox5ChLBoAis+Yxe2rRul9RBWb2s1Ws6AKC92MyJyoh0yGD2TSwKousaZA5Ranb4SOevr7dbnvBLuxMr+vtbSS7/ADcNbVOzWQKdstNWQIE3f8Fys4FEsycxTTXt+vW3+KBFpI5h4OoYNFd8mpwaIvrXaq2xlhEW+03Q6QDLEvuFu0o74c9giVchKumwvVOdVcLyRMjSvPWo15Kag3lO3A0tmzM3BOrAEoN5MVI2QIU6EUIEh+tQK92TI+hqlujmKWGna1+k0H0WOJRkBBaeMnzwz1sf9AlQCsgTG35xxIx6VNslbxq0mnYKaDJ26XpXecYfEh3NL28JCmw0ug1VDK6HpAqnl6b3Iek2NL21lLqSa7+imQL1QkG62Iu0Zi1lggevuaQWUVvM5gOqm1GsN6MYpybKZivckFZo/Nkc/ul/27Dw5kAKY8BfRbrXLHyhqowgjxftEBEwCxrhtgoA5fhURMS6I+OYrbc00St61kUiVUwVxfdavx0waXr4L3FB+5hNC247SK1viui+25wYcbVpdZ4Tuey1b4zmGsXpi9JtRncF5m0Ul3HSDn/mD1+EzXoQFalOqhGtawP364zgeIYyD0D8wX25ekab2VlWazMZvBIcx++XRGsfS9mHWaxmW1hexYtBZdc2iuTWZ/MHJ02yexnvdq+XDbFyv+a5ghbsrgOTD0KoKzgEboPc07z0ds5WJVzXUr0easG9GiERbMwlFt066CtIUNHwGiwcBhR5i7TroBhTtOQQQYhES4dHkTBwsIa1gm0dHulLDEpjoREuDukCbYR3gpmgM5CGdOswrdK+8tf3n/97eu/+4dP//DDdRsgD8kFzCVZNkhebLRWWESddpPG/gAKO7rt1XXwipNCGsJWkUyUJiIXiNa1NBIJikRu/ZD6TwuG6NSVfCTMZROAbvEBWPt3xZZ2W8ZMppKL07Eis9KXDjpWv2wtaRG29MWnY10+gSQ1HgunFg9n2BE0UyVNP6lwhV85tWiuF33Ewr2yDHHGINPnlRMyge+mD7EqUra34CqBOsy33C5TupueJTK5Q8pTmrQlZ4yW/7eTqMw75XBV2q5zRvZVXutFnpxTZ5v1J/v6c0lUO6dCXp0NK5oUMqAevnWaCOl/2gGkJ6Krk9biqvp1aNCvn/r6Aw2Yw9BqerZq2WiQp0j/2JXBFTZdQgMC2i1jw5lF1sBCeiw7hWItIbdHnaGOQUF8urdVUJMhSjuUfYCZV6aHwT0rKUmPAZqVrQozWoMscKKI/pfXqzbTrdBBMj/pTMPqIahS8AgY0cslTLfgO9RSZ3mQVUbCgqDwFgiTkh5ADNM8C9qhtuH8GhWwBfqaKNBiY32pJYrx6uZCkqh0+GVs4WOSbx9znzEudn3a9g9OeiZthJlsdQDnxbYsVB5GA2uYFxngMacMtwJOtyKdIkkbfYBol5gOvWHRo9BymFlP5Px/jyNenMK13Rwr9kz2ySYhVP5rdsK6sJNm1E6VJ5bIfD18GtMwDPKIMIRaCBmaGOq9SIoQNPSBM6l3Aa1kkx8CSCbYPSHAKigANQdGrLF4OaXs864k7tPCyorXyxXlv/1xzDE+LOjX//DLU/Bb5S2G6nYKdaIVI74uaVhxydk7oxKl16045QsLOaym1ir4QsfKxbagE4RPtWO4vReW+dpa11cjXImwSwKS7bjQy/+6nj9DMyH4vuZaVnTWuV1zDEVIf+KLgQH1ebr0QZaEb2ZuCMkquDI/u/oHmgbFKYDriK33Xo3iK1tUZFgbMcmIMXMK2FbBnctwF81boh7C7W4HKnbjRPfa0ds6SS0CQYeOJtsX8m1mrhGfEkxFDIl6uoHBRr2kib2u6ukqFWRKiaKc3RoJS3ini6Wr932ebDNBwNrbo1sp/UnQjFH1+GACqOz1AwAjQqDe9F7Y2T0rbQ0lLMJCv69ZqUd+bWIIeLBKYkzOYgm9WcwxtBehOSNSScZCX30BPx2lQltwdqlUlAkrkrRcn4hFba6oNgNYyh+Ri2ZsGzzJBpxZ7lLgqDekU+y5sEn/5YbSVH10/mwC06vr5aas+uFw1bX6v/0lCtYR7WSm1vSr8FOjsGWR3joOnRXCe/a301/2hLOblWRWcDWyM93nQJDz98P/5c3++QN/SfftecucR2YBCXnqQoQMD3cO2syZVTG26D4Pk0j56hTCRsnWP+9sza6bYXWUFPaTTfLq9BeNGuR00e/rUaPXC6LyqGWkUqy1bCdrJb0HBuln7bSzvSLOAmJuUZR9nnd5h9YmuYCFLkcZoOZq6KhFaxZVnaChE3OK/tt6sMcqRmtyGqweM8JdHfpMsGqWGSdA4OXpqehp4+1wbn/K4/nyRz0/5U+v2C6ReVOfIXqnnCIdV3jUl8PZEiZ7SpEgjD2KJ1JFczBgT022XnPtL5TV9LII1Xczd5zSHYPqOxLuyMrwUERTIFPtcHYmQXVfZQ6KVVKUmasDoSy+yKemUKw9JfUg0fREF/WQQKzx5APiKYIphCmUl8x6/MSmBLURFqzKNQgA87CVDNQs7Y27dlb8Z8UDN6sCF+f+3ahNJ5iGuk5CXfdV8AMQaoMakVgmq2s0w9zANI3DNLMnFqTHevR2Fd+qG/VdEikddWLttoPSYXI5YQMR0YLRFWHDo//Vao2/pEQ64C60smy6teuU6uIIcMuBStbWbgaM9QMNlHxZg9B9dgwgQwYugMbV+mnoc7qfbNHi9uHmw9X96TE/wMwpQCMvLv1b1Rm46yMVaBCQ6S/Wl9JF+7gHOGXU7VBrEbUGlGdODWtbC8irtdPCL8BjvzwJUqv91mDH8pkwq2Jb9OkSLm32OhAdncxWX6zoRvcuU3Xn18B4lTskSycI7TlvHLYOKh+IQtmxhaFdN7iZhKibOY038g+7/m/++v+68Y895iyOQtxtn2YFjizTulpW6sQIHFfuBC+4gBVM2XT2mg2nm3aWMOcONzrcRmnNC3pOW6l/iZZ1DNJMcFkzfDKSlEML55Hu7uHd+5UDqCh8FCkVQf/oMoPNZptEMbXCRYucCjUVo5zevSaF1CqeCLRJVINZITFL/Js5abnOKCEhsjmZhMVgc0pmOglhHihfClMDenwcJLEX99s8yt9mhP3MeP51vv/Tr3/EuP7dDyMcedxceFTRF2bm7pvkZN71dQ/XG5dkphGx9rM3zbOcUNbJs+7Jtgmcyn8lQHctP1lbvDsMC7KwEOegsSIpF8+u+M1Tp+p953oCyx8ASM9bZ9lavsmz083l6FGUSyabgvess/1g589ZSEzN8KXDlmUcNLTvXZkX4O0n2lYN6xJ112lZPXN1WWGdPBZtb+h2WrPtomCgjpW2KYRaLD1kulYho/dF0jR3RiVm0SDh2qliogyrZYXtYdMFEyDuq069TT+ANW3V8LTfDRRGsbhQ68/Q/Xou4dejIXGSp5DUR5J6bypJUczaV8MAJRK6WdJWhqtVCXQHJjPloCUJmj53V2gKhtDwl9iL9S8CVRxKeBFdgQgcajhQEceGkYYsC69UscTFzPFx1qwRCotqTjZ7o5u5bLIFFlXfAAuJduWyIJgtAyI3SpWuG3Ka6wqAaeyYCPfqJtapl1Ceh5HwqIYVxlwtrLb21EXtUyJMTHVgyKZEWja6virAx6JwFCkPYW31zKpo5sor4i/2/F/85/+cL/+ljnncn2zG3GmxhdcBcodFCk06CzNiIzHTrK7mKDCEeKwcNs0ni1lgstvfyUwrJw4LU0tG8KtkcVSAtFYC8ixMdK4FyBruNM6kjwg3tJt/JQ+ruY6cnksSFF9nPSHt7dyJggQoenSqs8yNdFhlhovoWLWx6397/3cSbYsm6VAQUSgjD5QsrKyObt1xQ7EmDeWwgLtrSdGcUyISkumwOefl6aWAor995M0uNmtszx/Ff/12f7rW5vNPn17Dppm2/7aWFW5EukTKrVlozoUrAi783GG/yUnlCLYGQ+dbe2jREVQVzioB+hZrEQytZxDkT+Xu4ebLWTq6NQ2VaIxG573rpnrV7ZpRXzoI19DNgtsWTV61wssezInrj/KRx05qZQX+zgT6vW5uscp1WyyIlZqUoWgLSjQsCkudCVSZL7kOTm2hKhWND5g5arX3ieUy26myO66rM0A5MXc6UdqAMtla9mKM7uIjMx/hRVzZMmCCgphMgHr77CMHcq2PFd5r56SGWQsUNVI0NnHQ//aibnqfMVdtxaWTxKqzyBbjYDlgmj/IGOrFm8Gkn+xEqzzHfhssckS0AYDiW2PWM/izq7rkWOXkWupGmtYaFIFkwdP9HIxtUElzCXULRNcv0DyS0UC0S0HLjdek8hJR+rIzW8dQX3S9Rahj4l0faGgcatyY5oNhWmCirhfbVVUZMzN93UmpfKyXapoHFhd60j6wNUVRZK+IF8clhCf4JZ6/h5JchgMwDBuSgaJArwGPMSxx4/ib//z//Hj5Z/vlj/Hl7fhrHX/YGPX29nzhJXwO1swig1F5hJs7qg6j+6DREihyFDcblRVOhLFQsEQZbM7dImiQvzwnLIYhZHmqUSMXKOkaB6gDEkGVGSxiwECme3Tru5JVNdOQrNk8mztoHuFD2khBKCNbMAgva7Reyj4qup1mqAgJwQmuiRgyaehtSFO9AbNoDSmNnHS0k1ZNR3RUW6hIpUaMETBUZZZfrFBuVkTN8ohxuU7Ak28f93t+OmLbf//9+uWH8XTZ8em///G1bnP8u8vPz5+B3eqgAUb5AOMEvcvZxxYxxDJ6Azqz00FWPKSz08BaFt8It/0mGzKwDxa6ndbiS/MlxBRGX+yEWiyaV2tvXWt+CmSusXgWJzMsNNSExs9N5sOskmGaYMMpkjHtgNTPs/Mbr0E+O+lU5fXmw7B+snWy6wHXrnVO9zQt0mKZWUrY7g6WcIW34wxWzWRmXj0SpGDJhbsXeOyQbCsdd3TOtUp3ybMBtNmkXgRRxuACq2xSATXXuk/lsBWPVgggNPmUXTlxWWN1MrN+BN9ZOTXlYD2f0Q/QXNYLSuGrPu1i2LGa31IHYi0nr7W7Tc4DZ5dfH8E9VsYqt2CVezxmtLja46sIBFcLatHj7jb0zht7sz2N1WHrzw3EY/BB4KBskZ8saFmouXqDquT6NdnjiwLdWVslip0aMZWfoRiMXofUA8kqfjVnRRgrTfsosOqGpgLEIxl7C2sDAVvDYqsJUTDLLKLMwXQN3mT/sGTzrIqGMMAtli6EWXDrHF0spyEKCQO3sAGoqH8bl79uf/+f/O//l9vlL+9ju34+6u3j9nv68fn6U+E3i22Myx33LIZhuNWcMiQEK3zEBrAyOUmHbbGVlyHNYweZhuLQDIdvRRanWxiGwSuzG1N6PbNoU4wsRNG6o8J8MKeISEchqw4jEzTvNRrNQWvuDDBLaYfTcimpRFq5YBZhQMIsHwxFB/0FeluYYaDgdi4GpMwspX3GvTitzKp5QoAuXUcRrOERhjC7uAU4Z8KrDlpgsifX3YMRFtejsFfcaB4bq2bebHuN3N7vz//tfT5FbX///MPLKLwbDzObrICRKMcIA5CpWeqQlF5Irb+MQhGwtDTqhiyvT9jyb1CB1AhaxIgOkcYhTapXOM698DDANfKiCW7TphCxT6IqNJNcsFDohFu/qW5CuHGB2dWZ695zgy2YnNtVLvQt4+LeHkyXiNJo5G6dh8xMDWrNwzWmkpi8xcLwHo9Qnb0Ka+Gz7lLh4eu2Ng/bmlNorHxqDTo3U01d64ZK/39rax1bpnsEkJVmqILmXtxdEhrrdQ6dTztYrHL3rA9cua0bH3jQVisvkvToUN41yll2UVWtSqoW1XQeaTLd+mVxpZzFh9S6Mil7AjWXTC79rRU1X9QINJcns9/v2jerytK5aluvVdTovw00SKc5hzmhpaNq6rZ0aWauWQAVyD3tsh5ToeU6If/3htTo9TwC0d9lVu3fVs7sJTseegii/IxgZUX3RFBZ7uZhWj3EPr5Yx0IknTQVLu9ZEadFDA2F1ZIx2AIR/eLZ0yzS5wFLHWb9q/XkgSpGz6Yb25q5rA6HufmgbY5jjL+O13+9/of/NP7hX26f/zDemO9f99vtMPf9mHdLlF3GKLsczJ33TH+NEXDNCiGM5GZb2WQdlbU7wmOzASQq3dzDEzQN+jPNwjGsw0ECFTCC5xWyRhqy2hCBm9Y4sAed1RgwkJXm5qTbBkNxghMCfDKtiVZEQoNLKFsG9KZCkSQPVbyPiWUu9tlqQc00qIUkoVAVUA6z9EzvHTBZxBiuQ0pUmHsTIhYRqKnV6iRnljncN5btExFb2tPbMd+rZtbFLS4x5/7y+g+v8fnb3379/Tb/y1++mtt//PfXH56M/Jj7W1wM27C+qMLcVic9Y6sjbMwmfKQEbqa3Iyf195dliNBkqxGi90yvCTKN4KwY4M3om0MedkK7bM9q9o4jMzu1HuxucPjyg0HlQsd11hkKprZaqjD1e4jlaLX+vAKZRA2L5j6TvTgTEELQsg471f7W2tYOC+2QWhoS1B5W68xAWjcfyrzlLidclJWD5Lka6myTC8GVlX8NMPeqlFLeYE1jceGfVaPANfMnokJMO1ui0xlO6FNfAcR6KQvCo9sArSwHFp2B1eES7MT6Xq3fMWec0b8e/djqGmLVPmyQKmcE0RtgWXcTUOLo1KI0p5u6NVzE5XnPVVUs+I/zIytpNe/SGKZq4IHZe6JHykSiIjbZojbbV4QKmU7fgFr/y/jn/PUPgA/Aum2vg0uWA2XFzDJzOdSI9cL5wLmKaPaytGFJhkWZuAI2+dhrqiTN6bnjekwYGGTM5I9cCd1ooqpaRV7EA8Ag9NQaagDGRLma3nKBL4y4MKdpoS4R7ozxB/nH5ad/ev6//C/x53+zP31skYPYZt7fYTFZlxEfNY9JiyDHiE+ZXjN34ik2Y2YlHJXHxGF+jcB0TvDGOWs4JwHLLLikUCEHWlOXW2vHHbKMMy8jGSJGtcgXi3J3EEiz1BGs3MWlQ1+vnI3TCR7E0Wo+nfZpmnSxYAidtfRI1X+Dhlhz96aaj0uh1UMubKjpcEQZndNhGgFgNMmi1GtJWKqBJZ1vFT08WUPsL4uEDw+3WTWLEaNmJF/ueS9L254Kz8e+X8t+f/9tfH6pS+wWf+zX//3fPp6v4b/Yp+s2xrZtg2bNnxrMou+XvkLRSGkI4lQZW19/qEikwc8xSdlue8cDaQbcNUUgkIRVoetmFSuGxhzKTlEpJCeSOqOv2qI8lR27f5VIs3DX4h/airm9U7YdugxGhydyapmPd5fOPRQazc7IuJQE639pT8+DITXLLKAtst2jZsKU6irCe5+JBC1hHTrbmBM8A3ELgTSahxXFZDlgtkoW6N8zM/PM8mgrIX04oP3CpJkHhI4pFqF6uVOfww5U4mCq42ZjSvTsvZzcxEuTXJ5JduJOiRybOlLlq578CrRdjlWvDi5mM85oK+j1QVY56dqU3NSa3m2XSihZ8Tb2dZcHsyFVdmRbWkM1SMuuujbpLwsh7v6nNtQztKaJVYDoL+km0ZdWkr+eUtbKRkZsEv57QLN34MotiqYiyxqqCHz0TrWWnyrJq9NlPL1t9ROWo4o2iyCL5gHTBJ2qbEmpitml+cPC4Ky5hPZhSJ1lAxxIJb5SNOkWuANWdqp3UdZT4hRIdp9VA0TdRqgfEzdefsfzH/bp1+2nfzp+/Jv/+z/8Wp9+mKPueCty1lQamcP98vr19p6GZ79cLLYROb8ma58YEWauqfCZRyV8bGPbKlnM5DSyam4eEzzyKNOKVgTMYOnUsirZdpAAFYAGoqjZK9K00nGNfIRaJSEgxkICCYRhY09TT2eZVx9yh3ECT0KJaaoGYEarRG+eWUGxJHjr+7IAXovugDKKEE6PkBqFoGvUwBbKVWEwYmhYTcZBqFabiXV0YybTJqWWHXtFTv/gkRjx9LofxvsRI+5v799+95++vOIy4rJ5xbf7/M//8lZH/E//fnx+epk1tbo8xsjM8L7FBXQb3PqMU9c8vJYruexHYGZYq0WItmBE96syZ5/pFVZttQqr4AFp4PQnaEZTPhADYUvdtuR9rXywJZv2blyaZCRgsewMkeh31e0K0rRsthcfKbj6+mANb9G2K+wGDxZ9ocHUVlVUUVIDSsIoRw4hAcse2QmvStUEI4YI2kUVFGBcphS5VIHqbai6rKUMXVyx6PgTKjRNoorSXRtE9JgXQidLTouKn9ZuGD1P280P1avV/vAr+9ujKXBi1f6HFGuABlAtJlpdlJYmWqtYq9jhyxsWF9JE/pw4dL0B/V4dQTlLnn+nl9oDrPJwMd0yajPtNKnEQxiJldFLANqaP8LIyWgo0l8t2tfJoZ6zW9fo0ih016hlX/LEaIeTSkDtnRXDmxUVYKlOwtUPbn0ZVqWZV7Hba1xcUT8IHTyrSml7QBrbb3HlnU6fMMjXu4cjmpdbijG4BKbrDxgxzOX+OshpJtqKpplpGvR6gFZ0GzjvY6uAFePmT//mn/8Ff/qLf/lr/vj7/PT2EXM4j/cD/Nhnzj3zqBj06wQsHeNlr7tbwK4+3CzzuBNlFTJ8dhtuhSyfBzHcg0mrw9yf4rLnHF4IKK+IfZOOLAhWOgJwmuVjgo6ih6h2PHcyRAfpiOlsmFyDKhzVKw4VoIqhSsnMyo2bVSh8uxlCW6oUQgZ6Rk74q2GIKgF5ohAlgbtpA+kqnPWuvcoSAJyhOszhNNss2gjIEMoBTIWl2SS7sZhG2IjtmjmO8v3ICquaNbfr66fc/8g88vP1j19/e/38U9L9csW8/fGR//m/Txv5j38eP32+EHc00dcAmEkE6gHDhIngOPuK7Hch5sgWK0BaBFOjR7Cir83OIl904yMGQHcL96pSMLXuoAxbDUhfK01bAs8io73drd0rVYzpzjRhrUjePnfF7jR0eA/3AluTvhxcsCgCN8sJFQGqi1oZDxqXrZoqmMU4e/ceGl2vDZrWhbg5KlUA9h+jVcvnG+PzbPS2dLU9q5ptFr3pzmU5p1BYoK25pFgOS8rBrYDSmzIt+2g51gMgLy+j1VY09efZcyG+ciLQ88/KNx0YuPgGQ0+LexPxDfzNYe4o9CCEd2WWLawE1q4UM9TsCt4fUds6pcGgtoFmgPocae66uzok2zG1pAJsJYJebsne0Zq3HO5LEgAzczPJp7Rxl01csUkiM4jb9IZfLYvJHuzUNVdTWZSewjUWHa3dHbFKBUdnPF0hGNcODVlVhJcGiKTy1Mnrqsv6WavJEL4y4nc8Idcr1zH1U6LHhlDoWA/ME1jZ0j6QPWdYqFYyzGNYXTZO4KP8Y7z8N/vhv+Tn/xY/f8ufdr7uOdKurz/98J4ftz++fbzvth95HJXz6YcfgQPHjccd87jdD1zj+enpgsv+/kcdH3dyxIXH3cNGjeSBLHg6jcYqZJUP28Z21BFuBSbT8MG8wC+AkUfYRQvSDSiTvFB2WtnMJkEbZCWcpKXRsod6G055EdYuWn0hdG/kpC0hSQjj6OUtAbWbA9olSbOU01klqCSqDZRnmJIwrQtOmRqJhaQUUWHiHgH66D0PVEkge6Pqy0+EM7MK8BGXp/TtduR9P9Jq5szbdYxr7n/wSB9G5v1+m/zbt7e3fPry7z7/3799/S+/fvz+//vnt/vh/4//+dOPz1Hc532akbHYyeroDywD6O5ecWnsVBkT4MR0j5VB+w5UtW9Br4fEqu5JqWXk2iD6wRAhsrgK5mEDLPeAVAkd0nVVYjKNQw4aADnpAyTCPdfGrs7WEs9RrVH97r5ZWR2pS59nbNXTPzhZ5aL0j/3qzzLPQEiYb40CVb9Vp6Ou6tkjiK0kWXj5YbMoklaFy/9oN1AaVrfzZ2FtW1NfBAgPgRiRG1hxHIviXm+i478kBQ+s7W2DLL5Oph3rX+rEsLijhgIkYj0GW9O0jRk6UeqbnUvF17fubIEWgJHa29PlLNd718GjygPPKrI8VLZ0htYmGYbcA5RAS14MrfVs9qfZHVk8C9UpRY/zZ5EspoGkFGc0agUe1kHV++g30DoAA9Z2X/FHrWo6T35XzAYylryzeaQl/Nd3tv5F1kMa3h3wOhIhmsjW91HvQ4fpzAiSXyxEq0tXlI4RyyalWiOg051lRE036fixGiHaQ9P2ogDNPSutMoLzyHq6vm+f/tf56Z/m51/zy9f5/LaXbenj9eX6dNnGOyKtJu7GLFaBl9fPNT/K6MflONKtduR1jEmbl9e5H1FT/PGcM2BwL1aQjlkEfcua1BLbGMW5bcMTlVHOyekYLAfvsDB71FJhNM6glWEFHueiWfs0qtCXcoHNNazygLDhMplIMyAI1CFDYLW7sGZ4tauGAiXwCcKRcvlgF8Wi0da16DwuEYpELmOMEww64B4xhjnM5SKaSfN2aCWQEREWu5jEy9Uuz/fEjZwDtLFtT5fXzz7x8fVvNjmur6y3l5//9PXtI16eCf/1+Ehccz5xr/xv7z7u/9e/3356ffK4ux7Fkt71sbcgMhszddjqHhMp++dicT0Zo5Q2BrOqJMojWKn/NNPOxfJFRIwYbha94pyjPcKUTUvkkikvKgS0cJkordBAbANVIVKIEs02HlRAamOt1dBWI9fZC43XlEgp3dqS+rhKeNHD3u9N2rp8WAMDsKo0g3/nXEGyV+bBxFeUKOCmjNqfgjini10HpE7JtsFA835ojZRbumQnf3G2QK0xYmcUO7ugXHmmWZ3FNhAalcpekc30qlLS5Yq5Iin71bc0TlaozeYDi+kUgaFd1p0vly2Eae7cWiRj3fmwc2GLIqwvKQ3WPFlVWQVOkhwnt7OqjIXVCS18RYlXWn2gxd/oDziLg9lNZMoNDgQq2nuRoLXQGWxuXGkNHSx0N6x3VXd3Zk3vdS2HcyGAAHX3UvQaVeHBVtu1w4E1CQRSydhbOdEJrEHDqRxaGbCTQourHrl3yauwSlWuJzgNE0TEyCpjkmYRaCNRjacVyG0g3Hnf7XL55p//0/z0//7209fLn97u47Dtt6/z5ZNdo+zIb3/961sdFrY9b/djTvPt6Wk/7gOghNsMGug+OQqTfqlxrZ3kDEdgqzwihtUUdhGb+f/n6k+bJcmSKzHwqOq95u7vxZKRWVmFQhc2NsG9hyIjlCGFX/hhPs6H+dfDkRGSI2R3owE0UNhqzy0i3uZm96qe+aB6/SUnBUigKl74Mzezq8s5R49q6/RZWQxGZ6OGNkaItHTVEZ7yNka1Z4lltYQHsrePGq6oDPB6LAgShgzjISZWpdNt/xctc3ACcXpL78k4JMXIoLNm7C3CUT7oLtoSM1k1HETqd6kKGOks1mAsvS9MxXJSM9eiRTBgSp9Bk64KadaakKIWVGvnyeZzhpdRJRDCYZeT7tj3FwnE8/jmm9//5E9+MZ/j8Q8fgWGCfrlTkYc5/+OvH/Zn+8s///CTd6eQiTnEJB3BfYaKsoY3QSBHHkvmmFMzS8dZfUGNyGa4LTDmlfysFx4UOkOCTRsJEQOJqokKEa3uoTbpuKnxtWLK6MhVzjJ71jy4NfRFsiy4tXwvM7jruhQpmVIuzgswYuLGBwBlaAwAKaAwWcYtmR9NNUh4WBr6IReYAGtkAK/AFGLNZP0ojOTN0JQOkque09IyOWH5uq057ywkueAdWYH4x290XnygRuC57EtEcAszpca5NWy14ZlViCQY6FCDiNykXrjxmhnvnKloyT+KhYTiRz9d80YBImZaXZnSq/rP8lbIoguQj2iFU0SphFL6UzKXyiPA2tRCAMyF0RXBBPUmikRQzSJ9P1XobItkSEQm/4IAmV4LR+Fa9lA6jwy2SanWOcvaImelwFsiroxYN4FLLE5FeKyXXDKlSJkOcamCkfWImklJrWURvK/Ue13wEqYv5amtj0kAh47Ip5F9VhZ1uij6lWpLjpZoIiNxaFGKeGQUwHZ+7pdfx93fXt991PcfP/HhZejdmdsldJuu87g+7c+7xv1Xb+ViGNFUub9cHx6aIo5jjqNpTjri4HVrerxc++lMlTmf3MfWWm59FzV3nxBC6aFNWjohCAmWbFRF1Vr6MeUbmaOz+RSQ0Vr81YfDc0lShhPEFLMCIjBVNmGWg6qkSgiZYzBWcLK/9u7QmVrnFF4lIC4iaIEcDktBdxVsAhNRpj1gEIbiCxJZVAOZzpFNFaIUJuQUHh4jsuUwgcfWuggcjAifAaoL5HQXdto9rvvwcdCFTU+XL47nlyCg23Yv2kRCt769eXv/h4+/l3NjiKl21SYSYz4d/MffP4t+kj99/9MPd8BLuEsa/AmhKp6eWDcbhGpKnTVKRoSKTuaG7bQmUNKBZSO8VAyvCDqyLpScHpCc0Tfc8JMIONNRZ1XlksUzUfYSr9zt+ithrVXg55qkLRkES/debETU7j3NeqteI6ThmwgiyVUEvdR/r2KLBWjkX2G5MYga1y/NslKsMdEhtVq5nrrhJP+qxpdFYVRYJymiZhrhrYIysCyH6zTnj+m6nEp4udp5Keql4I+VZlL4nnhXoTs5MlZY0xrK5epnzdZalkRSVga6fSyY3mBFsMstTq1TULmyjKZLpKJQLzfVlYgr+pfazem3vCIplFoD9XnPRZIHek2xqPsiGdkqE91aIC5bnAgF2koiVchldZ8/tLhacQTB110CyYHpSh2Q9fBloX3IQjz9eetXJymYi2kiq4O1mBhSzZdIkshJj4E1EJh4iIgJJGq7yFoIU2Rb/WbBYj1YuCyYxITUM5Oi3vLPRLikQZFG0xJzrQ8QEZqaOLpxXOOwtre7X8/7/xgfPvUvx3g7fdrdqW1vej9FyD7j5Xp9GUc/nXu7e/Pugp3P1x+cIebXx6c4doOLdQ/3EZRr62/a+aIyxPr+rDED4VKyZ4uQ6W6JhCTaBgglxBXqJcummZlZQtYRNTU0w5M5qAK+QANRlh+XiEBbNsskRZqmm1HybHID6+RW2KJULVDFqBOIVBS4h1rLHNvX5OAy5gUDynL8VCVU5pyhgly+ETnQDxXrtuwXSuVFRrhPhc0YnGKnS4CSqII0Spsi1vvp/t3LjJfj2MeIOVV7RMxjt3Z3PLxAqGZ26kF+9fOvOWYcw0eYtOs+29lOb7Zub1/QHp/4979+jIjWv/rqzbnZnH5tClrPMd6YsaCFSoaRzCIjuWims2FVl15uOxGlWxF4VblJRxW4rssdq0CDrGlxO2i317lmlpa2J1CQ7soPVc1WVGVuYa50Uwj8gs6LEMtCXqGaDr4l/dRV4EkJiLIcWpJWoCJywYXV8GePWo9/1aRlXu1g7ZgDSzWjr90Rqg6ra15RKO8EUtD0qjlPyKBCesloCkq4Ba9V2qzPz6BJhTDCynNcIl9Cd5WCoMJDzdxLGlQPU5RgMjEoiONHFe4aZ6sifPXN9diqvnSU9oQiyJ1Fikw6WUCvAr+G4jOB1xBK9sDFsEIgQg9KiXG51K55Z6MmhPP+ykK91GOiZEtCsDGTKamaJkSLLs99EutXsRyOQgChrbptnQBIOcVVGi/GJCd7sfwnS1iFQupRq8EZCpMk9hlMmdAtlFc+SfZZXud0c+gpf6UtJCovtH67rP6AzFxZ72EEbu4Er0mNgNBqzFFIs0w4Tg2n+2R73t7+ntvfxFe/9Q9H+3CdmGbS3oi9CcDp0/djuGm73J0fvvv4/OmJcwgIDozD5BAOk3TNsRgRgut+3F3uNt1wOqaPw48gu1qpcLSXB1VkQzrp3kyZpbhaPQw6JkUUKmZKTp8ugibSrPRxc0mbcwY8n2rc3k9FhAtYJBxIwBnVi0Y4mHyUIXEkLjulQHiWrpixbCiTdycJekA1k0S4M5Az0yZiqst7UZDarbS0iuWonpiFWVNhyGYaUicw0k1ZKE2o5/72vW8bxxiPT+FsvStsMub+KHFWOfl81q0JlWTX88+//tnLd5+++fiHh0O63kHa+YttzD30MnV8PuKffvdi9sN/+RdffPnubOZpPJUaMTHJrTwqa/I+IRqxnPpMUmsJqYX0yt2Ll8sKpYbkVpTLTWm1gFqYxv3p9pH/Zbo1c5WKQNyoZhENZ4RbaxkG1SyPgK6CusDp7CpKm5i8/SstWWcnFpDF+lfOXwQVLE6Dxc0mRZlTtVws6FLasPCpiIhSAcHdUe+c6Bq2Wnh9Ka9WLK0fSqoTRCB0ySULMwFWLZfpLqS8miECT2OJsmWuQVFZg55OgqFpm3NrxdLbDUtDVcpZiXBRS89XAMtdEkwyQAROpN+oAEB4Ykaslm5l5ayT61Sh3MK5sLS8vYAksURyYvn9x4KzyiVBgHRwShfYSrrrfiqSGGeSDyHSio3LbLUMlBpXKkVB6kZ6Nht5ybXtBAIswWmF16IGVveWPVH6VdSQZGqgFJHdJEEoC2apg7OC+Ro7EICT2uoVKQZDZFUlzM4oxSju/rqvNVNKvcQJpGb/nWkZImkaB1Et9Ug6+lSX9COsonaExcIqMdA+9Q9/v7/7lb7/HT5c9Z3iq+fn30g3WI9AuMxwmujZ2tkOHs/Xw2RvEAbi8Dld3XtDpkOl+IwxKdcr7u4i2LBRTyKbY/Z+53MIQZMYpS1xdxIGDRIzKNKargotJyYZ9ATMW1OfE4Ro28w8kIOfvvbWZhRZi2NQuCGZDHyWEFgoqmiacWpyjuVdxihrVVGIhFPVyqtGSwVRK6wigiHaFiYtIjTrRMqCIKJi6UjoueggROMYtnWiqcKkpYQ7wqewtc3dA7qP/Xy5t+3+YPfJp8fn6RQT000gGpxzjvhedWvWxJUz7u/vP/729+/PF4Ts+35cEVsLUfnIczN3385vAng4/Je/fSTmf/2v/+jLt2fxw2PP2eYIRi5sybKuginWwK0ndJAxrSB1qTY1K1mikmdF8iyZKzdo0QQSmnIqtSzSg9F6zxCd818RnsZ8CQpVRV8hHpKQ0s3cjcQiKkm+YgXVcCGC1ACQrt3IOYQVlFHtAlTENbFZXUd4lb5VqOYnMIcMMseEexp23eB1FLot6TAhhfjyVlhW9R7FLtStjtV+LZ135oF8BEHKGmW4rZMs2JfV/qgtfw4Q5bmfiU6IaFlsC3y6mMryyAsS4ahJAlvfOgX1C/lABa0qxrkm4G4SslW8Z3sTyPuR/jwhokF6uGnLQWuSKqprFy9/hOIQNDOIZmV7QxeLFck0HLlfgViW1zHdLD17akVoE/n/y/9IhenKVsVyaDNZ8UBQTu7VYsoSkr7WJtlhUiJUs7avFCcKU/FIw17JdwI534oE1XLf02v1UYooikALAlvlQE5T5O0vy5mE+yhxa4UXwsd6E2qZQ17/srKTHETK5wUBwjF8qjeT6NtHuf+Hef938bPfHvd7vwe/2D/P3aVvLdBAd3Rs/Xy32Tx04xdffvn9d58+/f47g8V1pE8FVFUVcyrIdNeg0WUcHhoDmAHKSSGHC8M8XJva3RvQ3OHHbpjOA7633oUL4YkgvVvrzZQ6wekIZbMtghGYRbnTxATSTUVywYDn2wRICsCkmg1yuvU6mVkGgBCGqFj63SbGt8q3rMIsq/dV+AWjtpcl3McEXqGvAgezbmo1qxkR0BaJGqu28yUcHpzBELTes9dwIpXKDEi782jhjKcdvVM2weQ8ho1T3wziQnDHpCPgm04VtTn5z7/8+8eHx+M4VPr1+en5+eUal63J3db6eZN+57E/Tv+7X19Nvv2v//VXX7w962SEm9jEYO6hDBFxsExSuTw2s3tbJUv6oAlW+F1HIcrvLOe50vVWJCLSYjVBuduhzNtbtW0ihJmapTwWJT3z0motOQCzcM8qyiSX7oSq+hrjQcF52by5oSpcReI0WRr86AxCRDA9iyldv74QbDBp1cX91nGuhGRmWRCslwZZZmQIzZLfq+29gUtZhif2e9N8Z4JZWSdva9R0oZThFXXtqyk0WQgyO7YMkrNWu8ZSHOltB0FE0CPJ0iyFMzpn7FtT0AlrVjsiUvNSvMlkq+m7zUQVxVJPszYJJR7I2jq5FLFMcoZL2JlDcHk8k6JLpXV1IAni5Jf9UapmhTNN6wQII8TUc91QZU+k6iDHKtNTIcAipCqar3FbrnKQdebXcwQK/AOASI14thJJiSxcIVu0cNTocIox6yqtklhyaNmfkmQx/lgt02sftN4CwY0JynpEAYdAApGSf5DhU2rpMUU0llcUCBeosOSfYGMVTqqY0J3twS7/6G/+zt//Qb/44dC+naYPD+n9chxTZRIqPbbzqW39GNeffP3Vf/8//Hf/8//7337+7Xfuhx+DQYGpdhWvDJXNXgCtHWM2IT3mEaAEVIgp6v7S+7u7L37uauob9yH+NK6fJfaYQyQwBuHEUDUnPTzACFeRkfsBoGTMyVzNARczLcV0ajJFZjhjqljEzGFXFW1NGA6GLrfh3EBvhCoiIqZb65ojIKpByciSR9C9BvjLq0O0HNWycQjIjfQAAQAASURBVMvJ43J1l+zCe2sj5rEfagoxNQlRSqHjoTJ7M2twnzHGcGvdeldthweuxxjhzU6X03Y6hw+fY5g13eCHiIoiyDFHmJ2lH9d43h9ennfuOdMWajoPn7vT2VWwHyc7N+vP+w//8VePveG//sufvdkugeE8GiTMogoLCEVUfFWZGWHKmSOrGoqIMJcRrTAJCm5VW/YApKVXW/lsF066Xm8sIP+15k1QMzzS8mEBu4Flhx5OXauBQYgakO7ZjKAamaORqAHPBQAiGeKMlVk4LTg+ByhviITc8NzcEoPKP5lZMnqxmeWAW7Y6utAF1F7f0FqDVX3AOtYJHxGMUBLxOtKL0sHUb850k0rZBcVnv1LlC+SGv2VyFhEgPUTB29DcjYmtoF9hJRZNns1NLDNribRgokBLn4jbtb8C+vhRdS01jXQLP5kziOSZbwMMkEnPvUBzzhWHV4Vd52wBR/Kj3xGUtHNBNiP16rAMIgFIwIWpHcgSg1y3NX3+li4qK/7CdurZVM7J7uq1RFjeAqnBFIhKzGpAko1IiK8+NKj1riXCk5uyJSJyFw0XXJVNMSKklHCx7nDmlHIA5oKBctwVa54MCnoAFFBXx3IzzEiPFs28UgkuoSVxn00Ftj3y9I/Hm7/xd9/J+yfvE9rbyV0ifHhcr+NybwLpp4uc9ODEefvFn/3pX/ynP/3//C/u48WnK7htp+GD1ystZnJNZk0EIqbCOQ5OTo/hZlQ7TZLuvd2byjGuh2y0s27bprKdu4Bz7hx7zMHpKqGKGDt9R9DjQIRZR2BCpElZXDDc3UJ0TJFcLCMSHiSHJ7RfNVpEtsmE0YNl82NmfborAG3SLVQoRkkNs5WVhIciVE1UwhGU7dTdMwpqGhbC1jvSWkwHA2JzemjjXYc0qu502br03lpnxAYdc58x9PzmbD4fX0I3ad1nzDEQTjWEzUFKE9tiHOM4QgNWIidR8Tl002N/HvtBj3mNbTtFQJre3/3x9fiEHnMfzxCbPpvg8ibEfzjw1//ycn/5/Jd/8sVpO819ItvMLBVfcYisudKIjyXDL3vnFWVyQeZrNC0ZAkSwnN9SQxuktSXlitRtvQJGq9Ijc67VyikiwrMWX5mVrdktZ7BqRqxgV4eP5TyTpg/rlKcT3Cq4yvJOBJL6cQDQG/pB4LZHJUMqa9FjMvsp2KQHUP5xElVBFoZeLWD1TLklJdLXI7V/S/SZxoNYF5B3o+ItElKxXEZYfEABX7dnIchZ7iSYULLO2vWM9K7NHWtkMrsse6Ll6rEm+G71dmFTFZu1RtJWnSqLZawksH5AqiyW135AAKbUOqvhNd1WxcBqjyQYq/oVEUmlbNzCONmsRTosVHPGyhmL1SWIoJi2yuMlNsACdgCQMyjLdBRpt2ZL9ZpRH5DUTMK5YLv843CxMmiOuk1VOmR9IFIuRB4hku4AWWBodolVkiQFks7zq3fEeoxtjXexckHdICZaAfJHllsFla6ijVAKTYGI5IUCojCTgDsc2toLL7/e3/zt9cOv25cvfDfF7NzEmnrsxxgTen9vrZnomDufJbr0c3fBr/7lD/vjU7bt/Xzn4xEe4ZhzhrtuvasFXdiDQ8K9pHCD6BzeL3cq+3h5OB73L886Bx6PT+1yJ5uZIOaAWli3fs4XaQIyBply55u1RqTAr/dzwGPsx3GYpL+eI9cTghEhPkEXeCYFTlizhajlgzazpgB8OsMsmmD6JMT6Jto8XKAxiQjbutwS8nQ/NTDgoe0EEWlpR6eAjAgaEeEmVJWm6I2Q6QETbjZDW79r2jmezfTxu0/n1nozmMzpen67z93LXiH6ZUs/IY8w6/RxHLtaVzvnHJQrNcbYn+c+I9q8uryVY3q/XD4//cZ0M9FwPoznroqDbrLpRaEfXz7+1S//cGnyJ3/yk95PPsfkhIiLC+DFnqmJpSdzxjmp/VNZymZQmqtQQ9buq7pKb4BlBJ0BNPS2giRBjxooEMhN6cKk/mjaZhr0yq0HyGmtRDbIiNT412FPLX1VysgoU0skVv0tImuMKFMVlvtCnsgUhlJUI+fcbgEZldsqZ8kKYVohOmPjKlw1B1+JWhEsr0ReFbXy2npkBCkOM+NSWj3cBIdyQwkodL76PWbcC0QkG6HByHMnaeFW0t6K47LSxtKt6AKV150DRHW6mzXUf1mMgopG+jCsS3ltbKrjWyRCgu0eapodEFmmZYu+kUxjy+ihWASm/X2WxUkme5TIbDHYN3BfbimrWkGSombISUOuPQaoj8trDjWNtQY3rzel+1hVjIkybsZslcCKEAZQy+XzvUxQzqv7EvC20KvsOOp2qNSIQCzH1LK1ut3Nghu5QKjCxfJeqoin1OVVWftaZaxcr1JwapEFa+5bhD59bDQ7tau03x797673v+f7x/384np6fy9gQHwOj2jny/2H9zF90K31ZF1ePj389f/+73719+ePP3xUaRBp24nzZRyHATGda3eQiKSQiaKayxr7hRBrlssmZox3Hf+Pf/OffH25+5//7nd//YfvXh44rKsZrbkfoabSoS2mHJMiTbbN7BRCbyUvkWC0kzaNceX+gqaDQT9660BgzoiAqLbWVBmDc8hxMG3CrGk7UXOF/RY85r4HhjS87I/W76X1g+j94vOZARqcopeLwI8xGZRNYM1M1frwIyJa6zBzn34MaVsomwnJWZ5WEGnSqF0DCMh+fT5oArfx+P509ydf/eTNzz781X/4m4chMsEpgS7MxZbpoK+kEqqn86YSg2KW9hFNGkTnMQJG0cvby2Aazfp+XI3WDK2ZSJ+MgPLluLtsd2JXv/7u48u//+X32+X8r35yJ4aY0z1AeOLLa8PRUnOwwj6goo4pddKlyDWpiip3RpYKVsigta5lJlqK9NUGJLWVHDmLulocbFXK2U6YImiWAr/gwo+xDpSarZpPfW2mTFOpRQxUi3L7Lkky1wLPhLVYaFalkjV3p1IBuzSIJQuSQI4jlbA36/ecFcjI5H6zN8xmRapJun3HG76NpQ5niC7oOJZndH6PDAmE1C6ayjtcnmbVD63h5wyslYdeU2MJipJTuUHsGZQzuJlYeEAii+NcupCbYaT0GSS54nuxr5nymcQ7qCIMhKR5Z04ssdoIEQ9/JfMzcBTeD09lBEOzXYgQFVNj0CNXLaEa0Pr7pdghAjkJnHbCWU6kTDtp+Jw8WulKVhuRLwSBok2K56k2ODUg+TPLVGQ90RyclDoqZSqdmXGNTqQggbmDIUgTqMBfC3/N3Y2yJjBQTZbIa4+1MlTVIbrwK7iHIHc7RQglF/0gkxM8oIrmDljodtXtd7P9e7/8k7z9xLezX+z+PEX95Xq9DjKk99Pd6fz28um7z/uxN8f5cqexH0/P3z18/HxSTG8GqI1xrQ5WRVrDdElg2KtTVYr2xuHabeweCK31mPNnX777Ly/6f/vjt/9dl//13P/nX/3wVz98fLJAa2jU0/3x8mLtHmHQuxD6jKYdYnMe0k9q+QI26qaXS9/uqIw5fBxizWIKUp9n2FpKHESm2BzHt9re6vmeKf/zlABv3Br0GPtn3d649SwofXJGs9ZEejB296aXQ4/AFLV2PiffH3RSjulwEupTTVRbm6kwxhxOn9Ny1OMsaqoMPj+a6gl+8Yf/vM//64ef/O+//Pd3Pnn/ZT99EbLjeJQxVURmDqTNbevHmOl6iNYplpa1betqtl93Ac3MfQbx5st3EGHjvB4xJ9HVDG3btu4uhwMeLS6zya++/+7y97+9237+5ReX5pMyZrB1daeJjQhfLDCyEGKWq1lIuSiXRwpAx02zgDXDX8OcLgrLZU+4ta1FGK6AEqrGVUZWFNY6h2tkN/leX9GzfqHK2lNfcRYLv0pt2PoteaDy4KekXQtvobOi5AqiRVUsirPiVvX8dSiFScyuhWvpCiNWURe0nFDNqynTZslPrnvGEkPV9s7Kt7c/UnqAzNWm6Q0HkNXGZBlKkYoBKPw5795K17dKdAlnpWCb6pHqodX+8gKXKsxmJEPec9xuHhcCtNgVZCUt65GsXmCl8aq98v3RnNrVAsLzDt98TiPLXitGI63frEzeb4VviWZu+LssNpdIEvjmEiu32pRBVYhEeLrAsrJ9+WGsR54yTTJq8YUUfkRGlGML1nRF3G7p+taSERyizGWZouqrrNdagx7A0pyHaxn8VQ9Q/6dyeu63QMBF+7zheKqaipRqzgO5GTK4Vlos5zzuSgnrV9t+PS7/x/Hml9cvHuSD62m7f+vtvI9J1ZhXmoi1fneG6OnNhY8hsPArybu7y7jCrAXndtnayebj0/Brsy7uVLGtBcKcjpgR1o3BWAFBFO5OazHnXbv8yYef6qfHT58+fan8f355/h/v/vR/++Hn/6/f//bvH7/5fn8ZY2q/EyUxTAaAoPhxFWu2YT9eYL2ZiXj4oWC7nNrd5o/Pcxz+9NQVm21sPbRHzPCZZxVoPP3E+h2axhgyJ8YAKdZy7BAhZio4ecxwd4Z0RdtE1Ee4x+5Xt4Z2VgFggNMPiLAKGYS7aCNziaIwjknSVJo5VV2aviGfNA6L63vw5yf9r97rL8Avn7//m5cnPpi+x5y7UpttcUyBxdhpLSL61vp2htDHIREzdutbu9xpNxC9MfZBHwPU7QSImJzO90qY2rwOcrTe66Vkk9Z84HAOv/zyNx+/fHt6c/nTvp3GHoIRDFGbEglFCLMIrk4cwOQ05OiQrE1tTAM2+mq2CavRv6x5c/BKhPCgtm39QUUZrFK7ouWK5qprsjJBsfLovyFBLPGELfU9MmNkEK+CXlM+KCaaoHn9NYEwYmUWkbQgzbNamENNoK/WAbKQnPrTIpw10t8FRRBmVZoY18KNuD4GwFJtirwC5St+LUlKwstaktyURjFQu3FEIIxUv8gyLsuPNIhXj4MyKK1EIAKmk1UJbCuPolYOJMWxeIVEsNJseI0cL1g7e6NERFYHWJIgLddokBCVpI7k1emTC6+uZ3TrXcAA2LQTrk0I+JgQdZ9qzX2q6BrxLna67l+haiIijGha/qiZATSbS1afATF1DwmaKF4RPhaMJGmaShHPYXcWfA+pobIa+kW2P6qp+rqRuZlRKgNKRXAAYhpgeLartXgjX1fk4EkVCLGqjBJHeWpDVW7dJcOZa3KBanFFCc+yoAgBGc1CDjbTZ5z/ZX/3b483f7u/feQX6G8oJ5G+NSNif+GMA2Jv3r7/4sOHfPXiOPZjGm3bWm3l82hbV1FVTAMjWteguo++3Yl7hJOipirKmBqiasl6tWa9b2MevdllcHO5fv7IOa/A+XL+n37x83/zs1/8u++//KuP3/wv3377m+fvX6Rba9auIER7O4td3oxj9G1D20gcPglIb2MGnq4mrZ/fgZ/x8uzpccEG95xBUW1iRmHoHedLTMpB9YGYwU1wkZgC9anCCb2E7DSVrR2kiEM7u7hPbnend/ccgwyOiSXhlvNJApyhUPgEqYipMWPANsSBKRqC65Tx+az7n9/Lf3WHf3OHr3y/s/bD3E+Uoey9Hx4+9tbtsIR0BXQEfbi0Hj5AFbHz5az9RFWBhTsIzhlzwAy9jeen7XzW1kw32rQuHhjTfd8Jodp+eINCN5M3H6/Xv/mnH96/e/sXf/KTru42I5LqFgjLjjKHDT1UzOECjVjtr2i4Q9OY1m91dL6/GdVMJSJaawnfmFlC3hn7bjRwSvogNUZUh4MVh5YLfI41Wp6+VFJILXCVCnZlGQ+gViUldJPFFQAqw/22hFW48o370tsU3rKErgRyy1658EYuU4KoigucnuU/EgJdvyizZlXLWeeW703ckt6qtPOyb/NguGW+IIWhUrzCYh/Xx6NshvO+rd+cW+l4iy9V7lfJrKmmL2Fn4jlFlaeKSYNuSR6/dk2FVEk9WawMsbT8RBLaUGpKyFSRG10WNlIZLlU5kSVyhbUlNLDaBUuEh6i5TxGVqAWOprYwmqxyF8GRBTFMVRuXcocFxmkBOE4IcwKQt5Kft+CfACK1bFBTM756t1xN4PlC5pLPV+w+ovB7rtSYzRFfX+9srSiqKznXbMz67QAUq+RaeTFl5tkoQkQk4AxhCh8jU3c+QkT6DlJAhAtGA5vp0P7N3P7q6Yu/me9/mKd+ufc47YN6LycTmQ4VWFM1NAn6Zhu0fdyPY9/Z+hdvviJHzLnv+8S43J0IjGOfMbU3xbbZ22ZGPcZ1mvZwh3t6aLox3LVtgpg+R8T9Zh9OpzuExDyeni533R+v+9MP58v9f39//m/u3/1Pf/qz/+3jp3/76dOvHx4fjsdJjfZGT+LTI+J4em4nk3amXk7v3213d6rw4/P14TPnaKLaGpziI2SXY4pMdI04Wd+IiHjifOHcFUMwBB5ziHhgAo1DAhHqermTdt7xIulIagwq2gWny4SKmrvMseMYELa7u1BjXAGYqekW+z4D0aS/v0AULwHFJeYZTxs//+U9/4ev3/5nTX7aMD8f/nT8/Cdv/vitnfb96eXZ5Tzn6NpVG8Tadh7Hi6pgzmSRcpOoz4B6xIjnOY/pI2QOADKV7o/Xdvc22vnejzj23RTny13f+j52FYlBoD1d5+Vymuhd3v7+08e//sdv7u/v/+jL++ljctIp1mK6REAoGoDVMhdPFQOn57KtkOxgkSLDFH0uzHm96apKuklP9Fa1aq9bsJVVUSZFyPLb0QosJEToIen6kd50K4zmlEY41WrsbyFLK06W/iLDf6B8Cm7znsKlzMmiu1T2C3MomkMVEpq0xG2hAbI/Eqkup4KhvILaLM55taIZRlOWqDkQygoAeTFYEFBGfFvjPozV5AC5dDgQCr1xiuDaYppfe7VHpbkSfQ00pWbF0r9kjSsovC6XDadclbdEASzDtNfZ5VyFsFqfYDm9pioUISzbBhZgVX3QSiCriidAsXRSopDhnvWxCGRrWzAiV6OTZRabt6OUsunXXWYtDUA2JloL4wMIIVXV3bX0TLJC7m3sodB2kpghpiK2LKQgJiQjkaQq/LNAAW/w/Gvdn1/bUTbatWMoHVRz9TpyxhoipRVLV4lXzUBJoLRGjRgOUUho7UQsrS2TomIuIcsOkJYP3dTRvp+nv365/N24e+C9bpujO0W2tKIEA2MOFfTzdnd///Ly/MP+sVGP63F9Hryz5+v13G1OPj1+1Ha6e38/Hl9eXq4WVOugQ3pRv6YBb32L2MGImKSIwbRTps9J8W2Tdye7WJxMpuJ6fbm7P43HXeahru+1/eTrn/zrP3v3fx8/+8cH+w+fHv/6u9/98vH588ffHbjTfr47v537y5x2+uKLtn0Q7bF/79cRewr2kCVzjKvs36m+E3WwyaZJC9MfebyYhjLUGiESMziIhn7RbmALqlgnVPUULHqbpGgzM4VOio+R636bdUHHfJ7zgMv0YMwYszX0k4ED12GHn4lfvG1fm359d/7vfv7mz092//zcxuyXy7Xh/Pb+w9PLl+/k6dFdryriY4o1VTtmQFvM0dsmEtMTEAzf90oMEfSwdCKDqKKZecT++fO47mpnH7EfQ6Vfzu/ceBwvlGhm2nSfQyO28/3h81ffPHzxT79//+bPT/1+vnwWMVPF8m7lipuxavNgJPNV00oZjQW5vmrBRQkxIFLFL0q6qgo0nNbyxGqhzVmCyutajgWfZKmUGhKtIe1VepVgEMU55/amjCaRPGQ1GYkIa/GlWQ5m0FRhBCGmq7JekC8WSM7UqyTsLygJ0AKECgAzI8EYGVs0qT8EEqsqMHzNaGZFFysKVvQvwJxrPWctxlk6yIrdhduQK+XcckI+jiaWgq7XOj2fntW8XjUOmo2IqOTmA4lgjnxXeAyK3bzq8PoiEGu/W0lpRBcZ/WMlaJkBCeNHb03R7+DyMqs0WuxHvgKpi2VuY1OVoDOWYxdvnMYi8zNKZpsYIchl4r6GyiBYIqdc41z57vaCZrm+aALWf5JalVmLMEmmvzhev0HasWZRkaVDXYzePqWasKRZvFo1olouU6PHKhe4nMBfe2iWyUCmwmpcAi41fITwdHtLQiKDf8B3AKfWqO2HuPzVcfqr69uPfm53fR56cFgvec+Yx5jHfryIxuX0YbvcHcf+7cdvm8fd9vby5tROpznHw/40ri8vT/Py9rTv+8vDp+vD57vLBhhF3KeL0MPpW2+MSY7g1NYgmRVGeIhKM9tkvN/07QbAm2KTxus8dWtmMYCG/ffftPPjz093f/ruj/7HD1//888+/PLT+HdPn//288N3x/682+6uaHh+2PddBJRd4oj9ZTudzXp09xkxaHdfZxsN+hxT5DlbNkJgG1SCoU18Dopov0d64FBO/cRynTa1E1O/7fARpEcX0DgFzy/W0XpHDJ1Xc269b5jcX3x8vkh/56d7a/dv24Xbe5H/9K3+q4t9ybufbNpfHrrHnCHKS98Q8fX57t2nx8sGwsY8nNHEiNyDINJyqjkR2CAPeJQNUo0Yq6j4cIhxFXAMkdZPW5cQn/70/El0+twngQ61rW9nOlwo7f6H68s//vbx668+/utffLG10+HXmJMIaVkKB5ABLiUTOb+iBQFHJKgt0HRolYyqJUNLm4GkwCTL+pUa8iRqhjwpEWAOJC8vzEWwArm0fU0h1S4ahCfoVCJ/JMOfzUqEWtlhZrWFVMxVo16qllXGZV+S5eDNILSmPdK1v0JDeQHmEGcWyKWmT6RhAeVkTm7WmjA43ZDKzlf4YIVuFNsQt7YpHXJ487t/jXmo/mbdG0FNKjuI6V50S22nK0wiIivYFaxvA1pAcTkQRpTLvZQ+Phg11KGG2tF9mxXAGlwlysAVRbFE2X3ddLKZiUoYzFu/JQk3rW4su6BchSWiKpEjtMmT6wro6xJX0ZHvREFeIm1JZBEUMVGTxd+oGhgjc0/uZ1+N3urMWOzILfUFlxwM+eIHWYS45AhAas5uaewG98mC6srrlasuAMjk40wNhIMoGcMSKSvC/ZYebzieiqBpeJCe8FT1wSYKFUQusmyqivaDv/nly91/2N/8fn6I8xu0++M4IkQDZqqmx3hxuisIuY5xHFcX6bbt16fLnZ1OzWNcn8f+/CTqb75627VdH59enh5hsp26NTme4SR8qnu33HXlEEAJCZi6D2uWdJi7b2Zfnc9fvbHrp/7y8NwFqhKEWg+fMXNj2IvEPD5+anf3f353+dnPfvrfnn7+MP+Lf3ic/9/fffs3n//lm3j5+PQius3VFWtv4iNiwKja7O5e2ETAOebcXUAxIGAXmIWwaRNIKEiF9cMZPs0M9ICbNB+Hw6Tn/BEAwuH7EzeLMREuJj72/YeXk+GMuN+2txNnzLvm777o/+rt3d3J3p76+3d3Zxx3Uz5YvFVv0/Xpiucj3LupUOnk1b/ctq+bfSPby5Q5p5j6PLSbGUCZEdoUado1M+Lj2A+o0l1bWqgGgfCJphANiLCLxfn+TdvGPnyGkxMCOou8GEczCchwFX3zh8fnv/6nb9+/Pf3k3WWMnUgJtUktfc33Ote8iMTND1FyvWFkd+5IZjU76QToNa3FNH1JVCCqxiqKQjVzwg05QSL7K04lVZtIQY6r3rZ/CEExSf4va1tBSu6yBtPsJAIgi7rIGF0wFESAUKzdnlzaCyaSyhTeZFzKCaxcgZlm8mWRUFYNOR1UMPnSSnLxm8mO0JlzZ3k/MoCsrfEVgrg8Y6WYbfKWLIF0tsgYI1UUyA2miiyVRW7+ZmbGSBgq971UbkrnEQFq2pm3qHqLWtXcpdFMXhcWH5BhDfLKkUhZ2hR4nvqxenK6AD7BjZzVG0RW2uLXJiCvhAUoic9sHVRe6feC+Fk7dipwIkiNduuKpMJx2qUu90wRr+gst230LFuIWN89Z2tZuuVqBatxSEAOUnqCrN8TgEERJK+5GQvqYqGN9dRWn5VfXsraP29PcE1jok4FSUoODCI1oNmSaq1tzW3GjKmS9jx9t/7rx9PfPN791t/69jbs3tGHeZPObtoalNZNOt7192PMiHl9fmZvbbsANhnqtu/X4/lFja21uzeX4/EZnGrt/v4tGpzD54hwc5oKxyHdROlzCqD9BJBioGrfDE7w7d39PY27m9r50uLpqqbatPeTCOc8TKOFXGJD1/3lIY4H/PDNaTuf37z9un/xn/0nX/3B/pt/HHd/+/Dwq6enj4MfH4/r8wMFLw87xsC7i95ttl3ievTeo2+Ue6hgTiTsw+A81qJnk34KtZgT4+pqIBET7uqhcwSbQAMTA0aKNISYqo8Xnc93p3Y/5s/O+md3p6+avdf25Xa+4Hofca9Pb86Xt2824Pn55XFzff/FGxXGOPbrk3hok35qTfvnzw+iT+/enb/qsv/wEZf3vffdXeFNCUooqRgxttYR0VQFiMHWTzNCmkk3BeGeArM5prTNVbqKaRPTfunDBwho287nZs1nzCmMCNUZorqdml53/9U3zz/79bfv3vzJ6XJ/ffncVI851VAE16pe6riAgloXlhaVZjZzQTwB3CxjhFXZJ1If2lt2zzfYI09ZnSD+6DyKlDGiVJdcZ+umJ9HqCdxzRL/OuNoNShVGaLMcIeTNA1xu5qxF90WKd9be9sR8knXL0CkimTkrLN4YhorYXj1EebJlbC3nmxtuQYEtdnfVsjV+vOQmlby4moCKOjlnsIIacr5doFCkTWFa22iyxyiYB0vbYit4Rqg25g7qZb2gIhHFxBRVUE6o1T5IeXKsOyN1o7JajQg1o5QkCUWwVuwr0CuxDTBflTWKW2xIsi9NexloZkWRPhappIfMmFbD/VLlgKxnCTKxHVUBWhl2FGiWowvCynqQ2tz32glitQHr1V0YWVHu6SVFSu6pKeBpxfBX+kpf55Lxf34/MqSXozgCgEYubynDonrcP3ashajUQHl+OxCvxumIkCWtCgIBFXawOaWZav94tF8+nv7xuH+5fMF+59imy3a+b3byMX2MOfbtzeWLL38y5rFfZ7gzSKeB7XzSwHHMuO4SYSaXyxbHdTw9UaM1qjMmOXeOaYyu1HAKVRlzCilmhIg2WZWKz4nwt5e7Uws+v3A/lOyXE2NKs+lHb29EP9txiIi/jNbvNIYde++dc8yHb1v77qv33/x5+/rf/Pwvv/3Fz7+Xn/3zk/3607f/8rtfffP8/PAOzyH6/u4FeHjafbp/esL7O5ya9hNxCEPEFELpEhERup2jnfKIwBvNMB0euTZmvjwJ3ls7G3weT/Pwfum9b3z6fhvjj99e/vSL+z/Gy5+f9U83exPz7Wl7f27H52e9hirvLOan7x9fDv303D686Zva1qIJu8QRkJMfQp1b71RcFB96O4mIIwLT2Q0xo21d3RDTIL7vJj0Rjn7uYwb3A1AGpXXrZ+57OgW7mvZNrY+I8fnhdLnbLuex761t/XxWtcOfxcyaMXyMgETv92rz8bj+028//uTDmz//+Rd9O1+vTypUMaerpZBO6dRyYM8GP5aeQkmqyVJC5FyOMCgtacBkS7nAarGmCo2q40DS1JZ8pzSgWWTm51W/V314VdXZjUMA0eUTkMZv4in1qZVOAoapCSTKepKaXg4qJUD9cVSu+MhbtMGqkxk13pVIQJBY+59xg52LOaiNpQVUxIJqVBjOJJaBrAtz7j8jDyjuuS0LNdMKNrEMYtpaUcQp9YlVhsuP5PBSkTiX0/3I1EEL1Ulv3JROZVCGLh51IdCVjiMfLihF+iTQvfAYqVh6q40DgFjljLVXpRB8QcWrupj84UxxdELCQ6zq7UqflQ8qXWXWKL6kfl31A5nCGmvvmqydFinTgeTMHgDQFmW8MmzQCRNGlMEeAJXUe9X4eF3HapPEwJxX9rixOEWP585YkZsOuN6f1J4WcnTzvKsG64YP1YABmfsU8itL9aNZS4najWcwUERMsgoxtfbs+puH7ZePp4/tnfMMucDOGIdql96EwcPHOLiLqLx9+8UcP8Qx5phhIyVGMff5/DxfHCCF83mf42CMtD+f+1UY8F3pKpQgw4Phh4ua9QYKPIJwirZLxJhjbMp35+2syjl8n/Bop42TTmrTEFp0gSvU2tbljHbIoXTpp7NjxMtVx0e1l9PLx3eny5/e/dFf3P3p/ovz9x/+6Fm34/6yn+6eNvn9w9NvPr389pvjD0/7x+vnRz/GMXQcCoF2MVFITO/bOawZzGWqwIV2/075lp//o41rbDTfPJ7jSo6B/bhvl3vu7/jy7l5+9uHtf/vTL/71JX4ud1+etV+vuL7MlwdcozP6+dwUrbUD2tDi/hIhcd2Pl6eAtN51s+mgio/ZNrEuYx7nY7+I+TF729DufO7zuFpvgJxOb8bzgwok2bDeAFWV1ntKtkUataPzdDqLdhexfjqfz2Pfr8f18WH+0b/6hc8xZzRTj2iqI5xqqg3hMW0GVFrg/pvPj//8209fvr18+e7NOHaRSIPfak4lAYHQKqIT9ZAFUkQGO4aDdJVNtzzbWjB0mOG20CMDV1rq3zxUavQ/3aEJj8V+Ldf4H0lJFuagCs0RARBRIsKICnYFIhegISvWp19kggxZdcmy4gGR29xYunCSaf4LaLKRa/Zooew3kBo/QlkWAF8CkzzhXuARRVlSxrQ+JkJKUZNXWs2RUmpbL+E1bJzogq417SLw5QbtCAA5SBVY9XsaY4tM92Y91rdJvI5RFEkpFUWkYJVsGyR1hzVXV/HoZsyQGpTqEla2g0BMV+VLMqJpY7adlWOFtRZXJHsdea3nF7lQEIsqRCwYzSx84kYFEeWXtwigCLYSHVWJTY+wnKQtvdcaqMtGRdPbejEv1bjlq5EPLqGhJHBzMl1z6xuo4a6LWJaVAGVprPh/+uSK3AUesl66RcOAS9mQdFi+F1w+F/kg66lAEtzEUoaphgrEjNAZ7ftn/bun/i3eDL0XnEIamS67ZEzRcPFJj+M4rrtPHmM8fX7yYzezbbsLi7E/j8cXU85jknq9Xt0HGN2UZByTc2/wltN9MXPhpIghKNYzdzNKoZV+Mqduffr9/Z0ej9g2C1FrIerjwDimX8+997sLJwke/llJ9E2gCDN1Q2NQfMTn72Pf5/n39/f/3Nv2pjf96uvz3U/b25/OU/v07vT5z9581He/f5w/HOOj9N/8/tu/+g//4Ve/+t2xPwAABop56TF+IAQ+sIHHw3j52PcHzueY3BRveov991306y/Ov/jq3V98qT9p8yctvjL8YuuXef3QNxWN/XmHC6fTYdrUtEHbeetbP/kYvr+8HPsVQqKFpiAGqY6JSWKHnr+675cfosP07u3xtM/B1uD7Ls1AUTX4dM6EdNUE9FRxiRih47iScbKtNxsikDidz721/TgAfPr+49sv3iliv+5vP3xxubv7+OnBY7qjqUJlTBfnuZ2fffzLHz59/e787u0vLqe75+NpzpnDjKXOydE3InILyqpiUsI35xApMwOjRbhZI9OUX1QsA2eVQVnBFElWzXgd24qdi7Mk0+BhTSSt8hYQldsmXo+ZszEVIPPg2BpXJpe0tIhMhdTSxJw1KwyqOhiRqpFzD2RGxvyQ24/pOt2mnVXvZ1qKNTuF1+yS2H21F1I7NCvMUgQSSE+LjN4FySyZf+IpJOkQ5cooGZYSkcvIkHKjDCsFjGeVHRGrck/kQnINSaLclREqelc8lcIcEGkxhMr3NWh1K9KDa9tWfnRCF7lHq6iSfBrLiPlVOalLN1rNhOZ00w0wyd0J4u619UR0ugtKUoUSYnE1T2zAEvEwQJguq+iSfIVBi+SJoIokha03KKiCMqu8koUM8padCSBKR1Uva21eS7GB1CBh4k+pWJCbelkKtscrRSCr2xFBINP9KiVALqfoqo8CJEOpQEsZrE9A0VT6+Yej/+0uv/TLY3/nZnTOl0M6TO14uV7u78Y4Arz/8P7u/u79Fx+en56P5+fj6YEMtR6hHmMcVx6H9tZNGIMcfn3qTRiNERKDY2gnp0s401dBVM3oDGX4bP1OKAINn91M2oaHcWlnjabatXU4dNtkZ25aT4Nalz7Fwd0YJl2bErHPx5ZzKc4xdp3R+tY8xtMnJQKOz9/o97+a1ifj7ny+nM5v7fJB7/XrP5a3f6Q//6N/+oD/4x9++ruP119//vT7x6ePnz8/P37fLLbempm29vVP/+h+e3cy0evW4tqmy/70rvN+tj++3P/i3ZsPl/51nx+sN8TLN9/Y83ML8LQNNHCc3vZ+eXt9fqLPPa4N2zF3mGyqATLYdIMlRx/unnyRbqINc3/ZLvdf3725iyeNEWJqG/ES4QPXrV0QodbdXSHCCOcRNBEVGWPSZgRnRKg65+l8Zz6v1+ePP3x7//bthw9fff/xh/36Ip8k5oDgST9f3r/bet/3ffoErKsGaK1ROOXy7dP+62+ef/r189cf7nVeES6tmU/PmQAphU9ioDcKkgyGL3AGKrlgJHd2V2GV4HCaNi/vM5gukCUyUC7PUY9bzx6JZ/MGw1SMSmI2dzrVGmEIUhJB5ELWqhVFctMPSNPmwXWacpY15fSL/13rbuv0EWVQdwu5qFpfbogRyvIhY06wmpX8kOXjXwjSageqPF813A38ZTEAK2SCNbkmkEgH6cUwZF64LZWUdRFFlROJKqRYCSFrBVuuh/SiOAt/QH3pKkkLhYusMm/RXRUpusXSw2AxpAXXcOVLFLK+AHMVQbrkSTVSObmaz6IsGdYgCFHPFO5ZqTM3zNFFK6ZL7lxMuZaojzDVlo8oGPWK5a6/2lRBWxjODbVKxmDREqV3aLoSfZYKResQyWwrXs2MADVxhkRS8JaIXqUSVtlQ9ToDahHUbLX01kW+TtOoANAAA17ZqKDC1C3kggyu9eNUoZGYIXK58vKr0f7D9fQHfDH7WfrlGGinU+KNvW+aI52q77/84sP7L5y6X/fxso9jCmNwUg/3XQSaU5lBqId7y+E8PyCgH73BGDEnEIwhit5OnvXMnLq11hpE5z7IIMzHvJP29fu3erzE9UWDKtr6z47rP5qQMDaJkBBjh3oTGWgN2ugDOnRTbR0TqipjV+v3X/7s4eljO67x8CjnrQnZ7CwxPkZOplrE6eOXYvej3f1fttN/8eX767vTD/71r18+fPs4Pl+PQ4acNh6fVfnh3t5t/uH+dPa3+sI+eceG5x/eub6/6JtTPH/+9YnR5k7I8fAwPn209++u3tOqiJez6l3r1+vx3M531k8zIGMeM9Kp0bYLO6DhCH98aCaSjBbZCCPfdvuiSTw926W3Ka01H0dLCskE0Ch/2VzGRRQMZDyu0lwIEaPPl5enIMecx3Xn5Fc/+deX8/7p06fWmgQDcX16Cej57m763J9fFDIBn3H/ZlNqTHe5/+0PLz//w8ev3v/x1s6THsfkWgTGmzBllYhIAYOschYEYdaDHuHWtDTLecprs4qsKCrZ70eZxwmwVo5kLvnRwgwp8Hdh8lVCIugqmiWfilULX/gO0tMlI9QayYIg3eTlVn3nFGbFUBavICucZU25YmO2KVX8J36QSh4uF6C8srh9jFTqWvjWK1ewcIL6za/qTAHKXKdiWN7zHAdMc2y5xajSQ6KcNWsh4Ppn7UpDrSHT+tYlZ12Xi+JuE2ArBdP61lJgSxbcslJw1ftVOscNlqlMYMXSc5FA68vevtgSld4awNU2UW4YemqfTGGactU0A0n3IamRDuEC2lpZSa15hfXr8g0EapDjVmBXVb/q/3rca00bcGsMFp+poolKha4xPw3x6sO4uqC8R/V6JWaXm5WCIrkiOaR2KUKtBlIIKGNxRcWysc5aTtchj0flE5FWloq9tf7NaH/3YP/yso3TGdojuN3dhYh4wNS2MuEQ1WY2x/j46eHTD99zjq0L9+Dh1OHTrTdtwpjpd2gSAcccQIhYa2oIHK6IOQcYvfdckxYOwIAWTBeV3JAIH/Pc7avL+Z5PeHmJlys3zuN7UcRkve4idrmb189eAt4WZiEDrmG2vfnPOX4/4xv3JtvlZbpu7XiYd+/ehtrd/btjv95d3j3u3yswDVelPDxZi8f9u63xi9PJ6X9s8pfne/z0fqrN6TvVX/bYn/U67lzfuMbjA2Zsd282ocuT+NTHnU96eno831/25/04dhy7dfM5rbcIuR67jqF4AQ6ExvDW7qZ/9uEKq3ORy+Q4mKI8Pala+CEBl3RAlTecNnzMEXQ1iykMj7HbdgIQIj7HqXU4fU7tLWbkjLuSZurk/vj5rF8EEDP8uh/YPn/+PYVNNzhatzkjxojr9WW6tLadTlIEIOZ0VRF2UX58efn1t59//rP3P/3i7eHXAP3mbJ8Rr1D1CmuVE1RKqwouukpui5oEEDVRzSXGJRKSGz6/IiVEKJ5j9JIQvCbcXpO6uorwotUWi1eDuiRz7klUchUEJIWIxTAzKVfRWtiSdahanrmFLVfIKCg3FzppLQq+He+EzJmBRiolaJp0JX+QPRtkOYwV8kVActd3Ng4r/hQao7WVLWNT5hv7cSaM2kMggLwqdooSj4rYaXDt9V3WmxJMyD/nO+SWirDwnwUz46aOAck0NLPFBDgXQifIEJw0uEgZv5Z75eoMkiuPmUKsUu6Wv/Et/fDWE1R3qZL+ZrLmBXH7TBLuDjMm8pwMkAqDLQcJyptWbn+r0MP8TxlVb6mu/r1ucb6GlTlA0TRiqoJFchftkpAVGQSi9lzexrFZydmyfc2zltM9WJw1AARkutds7k0S0GolVpLZRGIkuBFTopkhvWnA0ZrO0N88xi8f9Alb4ERpIbQm4h4kXXxSEIHgxPPjy/Onx+sxx/UwFW1terjPgMKatMaYIREecODYOXafu23dmtHDfWyGebhySDOVhFNBuJgBSrHwwSBdaAKP06ndiWwynXM7yT6m6wOnq7Vs2LKXS87OQZnX3t8CLYJEi/gYchw+T+/e6Ic/PvbvG10b43LX22nayWNcHweDMN7dXTSuU2W7+7KdP8+Xjw1x5jyuV9u/G4G780XE9n0czztCt8t2PHt83O8AhQY95jQVdYO0OXdC/ejCobLnZCgmEd2UIe4zGMMaIMCM68v3EZ4bDyMknMd4khDn0TQkN86gAkezHpRLsw9mTY6Ab82G1AR97jPWpuZ2HEfuEE9JgjWZxyHgdLSKETyeX9rpvmNrF53HfHn+pNgQgcnpHnOntug0cfeo8hghYtOHBGROQkza7z49/eb333/5xV9c+v3Dy8fgbM3mjFJYVMImBUihQE7Ypp9uAPQEZFIralRJv8IFi2e5uhqGmp9a8Y6MUGuSukxbBxG5AdFX7blMHPM6aucPQpJ0UC6hPwlQUVPz5lEKJRENzgpFgRsqy9f6NCU25e0Vdbk1GYACnYLyo7+4AN1lPiGlbEkjM11x9mb0sqJdeMhSzLOK6JSTSCIwoha+4COVDHCy2oqsbPnKS6//XGA6b1x38u3pn5rKWVlMd9QkbKx5tDKWXAuAqxQNOllqxMx6+XdNLbU5uubUYnkrL+IBtUlHasFnQmCL1oHkjEgm7XWrioxXeBpE5JYtQa7kzP5mqUyhglb5uGQ+sjqauHUo6/MlKaG84WWQhB/TU5UW4vbziOJSoOHpwMAlWk5EMl6VVGR5TWTZEEhrCiVEbL1UWjolInTBU0Sy8zc/iEr1VVWsk5IzjXQ4zUy1fef8uwd+c7TRTjHIhuiafusRQyaPcGgIEMMfv/80xxGw8XJsRhCny2WnEN5cIejns0fMeMLxJGMX98z4aVHWm8EPBVyUdNIITg+oqLbpgumQNsZL693aNgV323bXFQcFNa7i81DpecpmHHq6i+NJ6R4BkkLpvvXT7pDWnS/H/ry9ebN98U76tu+H7w+nN2d994GCOa+u+/78gJBtuyf3kKmNx/zt+d1XpmfZryqqbnPOZp0H2rZ1VcfU3uFsEfA5tZ9Ol5j0ORmxnS7WdYZP4xifwnf6pEYzE8lFeq6C6R4Ba6JkzAMxglTr/dxzi/vwmVgHuqoJZnB6E8accrI5ju1Nf9u0a8wIbZup2XbmeAmGz9nOZ9Gw3pjGIQ3u02zrfRtPV2tCOCMirDX3fdd+FmmiU8WaKYfHPEhnHKo+nuNQbaeTqMHUFNBOHiIaPrUb7fLwsv/2u8ef/vD5jz5cuj3FzDOfIvDcDGtJIfotihdCn3NXkJZh7keC7lXD/fifipLlshjVBKTCWW5UY9VTwVpYxtJnCiMzDZ25eg9L4k13JuG8sJyKmFmAMujioriFm6wPVz1YARDZmCukhphvoEmeVFaQrfMuK8Ppqlhlkclce6R/rNeUSqaouFh4COuf1ZRkzFl+PXXnaumI3PCMNRibKEwxLgVhrUvJQm1mK2W5vaOuYoFdQI1nZnCWjMkeufGPhIkmjlMx1Lkq4VxYBpbdf8IdEMiazFB3F4h7NLOatVo8d93vV4X9evD5B1GFOxBmtWM6W6hAiTOTklYiU2uaUdTrRS5hVWbUbKFuTzkh+xW48/lmps8vUMrSck0VEUVNJqNI2tdxj5whuT3e269IaChxNF9pMR9zapMKO8vnt64igPJSR+QP5PAkTESVTYGg6rbz9E/P7R9e9BEXtvsBQTNt2xzhEwz4HONlH8Pd0fo2xwAFc4jPMUaQdnrbtq5tC4U0czNyxpj+/BTjkdhVw0wsh9M9rTAdRg+EqKflEhTSmjWh0L33cxPzOQXy7u7+rm0Qt95FG6fzGGqAsG2GiKYy9uc5hzRVU4o6J6j9dIbK9J0BO92JbjE+zZeri3nreu7o7v4Sx5Mfz5EuAaAIwnH39n7bPsCa+wyyt9O29b51Q/crjO3u/nS5nJsYxyHpJtO6bkJMCMPUJZwx4eO4ejhUmjUVNVPxgQhVad1sk6bQoCpFpooLIzjUcgK+7MG31pvliK8HoadNms5xRYy3m52ccb0qtPUzrFMs++s5vHUTayRas2aWVYKJtXNnhB/u4wgO9wmE6YoJMaDRmgRHzCPCGS6ggj52BLvm1QQDM6CnE/REOQ1uf/h0/d03Pxwjtu0seb01Uy9CE8AEZMjywkqKqtDrxOlrT2rWLLr093lmpMrpcguov4wa0QKTTtUURqwBHSHXJl5ZmG/+qWRcB0CqiqmqaVMTlL0NIRFgRFlAKymOIoFzRLjCLvL4ZhGCokflNSAVAnSLl7nAg4JVGIsAEblKq0gEhixR+PotJXBfbEiFKZDw8JKl/KhHWvqjvIyaxasR7ZoI4PJM/RFMheUXiZz8K8WqquScWqWbvOe5XbngpYSPqmHJRcG3wbScVc7d2tWSlOanrs3dK5TVI49bDs4L4Pqz7Ks8sSnebtlCn+rvV5OQN5G1RqDQM4Jkykic6RknIuH1LNc8FwpUFwV1daIrm4uCImliLrcyIHsyS5jULPVFAkHQM+yTJWSuT64mJ4G3UknjlXJQEU13i8WhVJ9jprnULVGwaipDkrivhJLdTJJZDIImgIdYD23fTPuPD/b7vaPde4h0qxF8eMyrj0NEW1OFGOBzH/vIM9xa8wiaThwOp7KduzTz+TSvn/z6EPtDzKvELkJTSIx1tydjCmjNxFpIOhcJkagcwkOhaiaEKc6ib0Qt0C8XkqbRe8+dpWbWTpuC1hrQzu/fnr74wt7cBfRpf57icx7zuIaKB+bD4/Xz9wwnmov4HJAmk7HTcGraMVXZJfp2fgu9O65PY8RMB4fTptYY89ifj3kEwrRbO2tvIoiJyM0z2mS7iG2AHsOBrtQIaKhJ17aJNIYyaGIqTRWtZVIkiGbb1rZT35qLOg0KB3LLlqK1hmZB4RRQwgNg43xrei/AHBAh4dQIjVB3xhiAmrQ0cViOiXvZNgLAFIX4wfESMQkXhEhUmwWYCBAm6p47dxVzSgzEDJ+EBzAjBjkgz7vvPH1+xjc/PP7w6bG1S2+bKlVEmxXyHoWGpM5hFW15cmxB+iK5+fF2ejXlP9Ww5/CRoKyhgYWYc8kNncGY6Tlacbz0R7gxrKtrcIb765/XToFFFeSQfEYNFc0ps2CtmU0gpaIEktjMBZKrGF21aOH169ckSMyQak5ubMAa8ko2D2SQXiEXskRQC0IuLFy4BIcrNqxPW9Gc6a5GVJn72lPFwoRi2WEzxUhS9p/ItJRLBcJxC4q10isJF6mOLVWt1YhUPA7W1BOBdCpV06ruIU1bVh2v1MEqCWJpdcCAQAvrqNYrbjQ1gaQB8goy0N9SZnVLuCUDMtJpKu9XfmxjgJE8at21iqerB2W6W6y7zNenpIySj9bbRqzXJtIoNVbSWt1ZVgWy6vt60on/kCU9q1crF0yvQYTbNgJZdH9Edn0aASt5mqLWy4tIQCDp9QQhnDNERFob0X/7LP/01A9cRC8Mtd4gFjHdd6HJnFQ3M3pETEyX6U7IjMmpanPO5+9+r5B2OiHIObA/+dMPegz6FRqQszZTUxlURPjIUKRmhAVBR2stFvdDImZoF4a4D87xpp/eNPLlxY9jjl0Qalnhco4hqtMnFGzS3r/fLpenzx+vnx/n44Nc77a7TYMzJueLnk+9bTMiRIIY8xDZBK3ZGeIRnONom9rljqHT9Xj+NF+ODgHpccygHw4KWkSMEdJEa7pEVax5EBRKlyYzBwNN46hBTe0d7guWRE5QZkHVT+ojt35rE9G2zZgK6Zu5tOEDh8RJcou8C9zJGZCBcPejb9h0wK+EIxOz9TndmjEAT8vfrGdNOMW99S26zxmckYM8PndrV8Ubab2fzim6Tic7NQXCTBOqjfDwOUajAS0PmUI4gwiOwN77tz88f/PNw1fv33fdxtyBEhFUiQI1QSQB92MMIXlJhGgLluq6bhiJWzgB1Cr0l2Inp3Ol9PJr1P21l78V5AVUswal1o5IIUItd3IFc5dW7ujOCgpLLZ/1r6Qrfe4LucEvGZ3VsShSyVagWhnePAJKQb9G5Oosv/YJWQPJwkdyjJSZmxa6dRsGKJQKQi+n1apfF+SQYZrrhxOEyCQRTlmrdKG1LzjKjmIVmWuqNkcfiDTb0ZIsEvjRpMKPUfkU2xRMIkttI7Lq3UWfijojbWIlcgw3CupiBdkC4LCaP5XVUiwFF7J8MNTcd0Xu0mxSbqsRV4+w+knWOJpC0w46G5f8GMktEqtmkbooXde+lECRMqHUOpPVqS+tZ9R+UuaNu91YJBIlwhBIrPdTy6sjL5c1RbBQoor0t1cly4YS/OQWzchdM2unTNkdBhiaJIJTI6ybaf/uaH//aN8fp5C3dCLSfHQEMTxOZqIhHmCID/cZdCXESUYzpeCYB3xK20TFoL4ffn3E/kjfVVK/TgXoQ8jYDyhy3E6sm21OBaPZ5qvboodY2maFqlrDfUMbz/TrfNo5Ru/ZzM2ICWfr25zHRODc1UItpGnEqKUSj3sauIm27e7OFeznOUbQ5xjnyxu9vx9BvuyIAc7pQ9liTt0fY8xuXf2Ah+DcuoY7qAKPQYqaDLjDQ03NWiBnrOgMBNFtzh30mFNPXaTBwBhC9Qi4C0xEI2ZA1MzDOec0MR4UmWOK2SZnnQqIyaYic7xwovWNjKb9kAhy6+3+ZIopmKLh4VBTtJgTzcYxT3enOfYZ86RdTZUzYqhI61u26tCQmPR97g+N96anOYaP4dOh1qyHczttaZujKYOJCIp1UyVETSXm0bSZaoAfH/c//PD8s4eHn7w/m79wzoTPJX0usZzybkdxmZmvQB/QlrOueaoze6haqmO5dncXIFQatQIsxOxVvJl+DVE8nqYvAhZYXBoNvFJ3ZLOWF5exVSA3BX4UuCNmCmoWtXKrK8GFilNqWqp+TZS6e+HNqGIeNTmQxqh5RQVmqIlSg56dR443ZYlY17OIvRzKpcA9Xb+EQoWQEowlCkSFiIzCGasrEyRKXFPDkFu2RXLZIG05WEjNcy0WQtboUYFLMFUyoKJJg2TeWbdcIS5cfwCIhDsEZiogdcX914q99sBk1XSTlBWYxqoJZCmDosqFm1UGktzhLbxkb5DfzmlNV6bhjzweSAojhcqBhIAoiIrjaRDB9QInpVv5rwB6vO5YSLWr5EdW5C66qdI1ovRNohCVtLRQS0ANYq9LMJLuJxdBkPVA5nUpmiunzCivyJtTScuRCYRqLmI6H+y/feGvXuRJerQ+pweHH0fMPWIKp+oQC0oo0tNlqLhIMKbAm4jRJaiiprJB7Njl+szrC8cz/EXVBaZoJobwsQ8oMwqINgkxNQSIRmkQo2cxSYEex1BVayb0N002cWEgRlPhzEEPrB7O29ayM79+//3Ldw/j4SrHVDPSFR5jGkyF7i69be/enN+/bb3BZT4/KzsmFKpUOMPnfHkWTMW4u7wxqEwqBezb6X27nLVZg2kOLHEXDxETaQx0u0sFw9yPcKc76FpLrGpGKDQXJgImYkI4PIJBFVFDSS+EEX5MnxEeEQhVjzjmOCIC0dqZxDimSjPo1tp9NxlTGNZMm9mpS7OlpRSS1jQ3ZzVrNfa08G4ScCoMiLE/hV/neC5dGQOcPnc1zSlS98lgBGMyDkfQCAM4ZkzG8AidUwa3bz+/fPvdA8WaWgYsy5X0BW2TpRIFfLlaaq5C15oIXQxYIdSy0P4q5CKP/K2BIFdxx7ihzBn48tAXxnCD1EvJKqgNhRltBMyJqlLaJOyElWEKc66wJku1aAuMWbkkpxmRlmZgASwouCvWtP8K90U5rEFfgfj07H6S50BqO7g6mtVkZKl5wzriFeZWSsnHw5lRLFH7ct1ZVMjC/Vc4WiGlcKhiFyouFsIWNbZG0GtPduby7EZkVeL1SVrjvvQ8uWVgHMLQhCzCecuPrzUyXv/fzB+KhbgV8pWfw3pP6t/pGvJjuOVHnySZhwTMPlJX6dGgmrI0EPLaBiWTTEYkK4JYe2P0NbMlO0UBgpEDNxz53FNmR3K1fdX31kOqaF5WIGtKeCl2BaiZkXypCFHRkpqmSgircMgdz8y5bd70W2V/hVVaiU+ImfUfDvmHT/jDvrGdRRvnVBM2E6Cpks1EDkyZfpWrEj21ehSkSHgc4R5zpKBvvOyNky8PHI+IF1WqdgFaN8RMEEBAVcaICKTrm8+DGfCSlIqjqal2Y7S2wffTSe+7KHOFK5va3KnZxga15SgpVBXO/fPjcR0xnHMI0LYtiyEh537E9enuzbmdu4jSt/Hw5BMv+5MfzwhXaRCFWGunfncnamqn8ek7a306xY+ODaKQncLWezCG79OnmMz9OMaz9YZwjmGaZvCyh4MOQ4QHQ00UhoLALQNNQEkoMFdxqdIgrtaW8wd8OArLZts2sQ3xLKOGWZpt963JHsJQMW3qB2kWPiUoKvMIFXVgjtm2bfoec6hqjiUyl20b4IMax/Wxy53oKTzIPEkSHtMP6BZesybBaemOLaKNh7tHIILWR8gu9ulp/8P3n//4j97dn86m031SQJPyqF+TMRkIFcr0A1/YZlY7S8ZdUzJkDdNkbKqppTSES9iFWAZtpabPxCCaPl1Z82d0r8Sh1fXfAIF0PYule1whoJw3AVJMVDUSrPUq/wMuUKeDNegpWaPXzoNbAIqce0HpWReUk2IhUEQZDlWwlsguuwWQMNwcJioQRqpUS7+gpGgpairmSI2XrmYkHUwXUlR70m7VKwDSzDJ6Ews3izBLorPE5RWDWVsTbth6pHwrVjqGWM0qVTMhy08bIlXQogxTqwjmK81Qe7eqj1AuoGwxuRnRkq4AIq2NOEGQae50++nMHSsO39qMQJSSs+UrgDSZKFM4pNATiNV6qNRAgpIQ1ZBYCgEREWlrF0+No6uAKJuK/P41n4sbGqSIWnqQA2KFgAmSYl+sz3oVMw+L1KqwzDaZk4KuqoBB4lZc5NeEWm5ToaO1Huy//9z++QEvuKPewzbCCVdlaUdNAFfQYyZEnshWFoBJ5Ls7OK2bBlVjPj7M/ZPgWTBVm2mI6NbMmsw5nAcioGaqFDXrElSSlqQT4PTpOG0K6a2BMcc8I+77tqldBxERJq5lFbt6fmhrzRQxj/1Ajo8az2+/2Hp7fnyIwxsiAHt6mW+eN8v2KwVpwbmLzOm7tpO1Cw5p91vKVObzd4apfXMc8/kZnGBYhJiqND9efAyFmtoI5xhzfxKn5siT3eYci8ynzwLnUB1ajEk4Is0dZRU3ghkAurXDnXTPKVEPo7V2xhyBSVCaQAGfvfU3XZvEEYf0TaZSHEQzE8N0VxO4khEcQjUz4QDFDDOoXSO8+FQG4vDdhDNERRpCZTOhzsNtq/F0Oiiu2hEDYjKFPumEyhxOJ6Av0r77fP3uu89v/vTrPg8wRkwRwlYJLwnNRPoxpFtBljnh0XqXKvklyjFtzV4mwHlDrzyW52hO9uRn1yGKHLpSUVEYAgFozCkmAuTil0onvhB/opD9H9WPyyRn4R3Lwh4G98gwlGViejsvhxiCVErc8B/UXBdkARoC5D7MZTu/rgoJcRG1VxkiSXtGlgyrSL4BIbnEuOrRiDLi16R5UfFXtAJNOitwTUkTUIkZoppPokJHQkBmVVMKyoqu9orVTuZ8q2WZaLBGtgSoZBCEgKq2auY8GLmstoSMq2AvfgRJulRTcvu6ApZCKzEnkfU4MrWyAKpCYKJ+l4hguWYy7/eCrfIhtEzZa4JWQrKzi3LHk8TiiwOOyhb1ofl18/GJcM7AzS8QnrLRqE4zqxVBjuxU90qv8TpB6r6d9XXAmt9INR3SpzAbhzwKcTtSCiWleICVz9Mzg+5ABMK0ifXPIf/wKN/4dujddIMYoTGGdM+wiFwtL7De1QVwCQbdPVXsg6GAtpyqiOn7C2IKA+EQwB0y2zk1RRRSUcvNRJpqG/vugexNmyo4AzcPpRDtJKdPE31/urRwmSP2QZO2bZAkxRVq6rRuYxzW9Hx3ns5QDUF7e9/v3ijncTxqSDPh4fG8Rz+PuF6fH3U/nN6ztEuyVN2aIcZ8PgCZVz81NTspeQzvQ495wIeeLr1v0XaNRgqd3TYFdIZQrfV9HAgKLS0MQsSsp4jQI7JwiPCkZkAnZc2gaEY0GCHQXstQhJRwbUqf9HEEY87waQ3Pz0/t7u6+b+pXidlMp2FS6DRTDxfQjwNSCdtb6+0SwPX6rK2rNechOZ1ADwbHTja1Uh1A2KxFQBWRm+tVmS99cL7srW9uxpHDf3NaV1FRO3z88Hj95oenn/38K8k9ZRCf3jdjvsZKRG1IKRsTLeH1GgrORSs5fkVTK5hDlnFWBVJgLRnPBjoFFwINDzFRlbSOSfKAjFUuLxQZvJEKKhpM6VXGxZsLf6HjoqmkirWmMRWAEJE0nssvIuu7AF7IVXXzXM14MikrHGZqyFxAEVWmFg4CEdji/ZJlqpYg6l9Z3C9JTNb8RSZHNkUCheXmtQVslBNS2ekUXZorKhG3G4ukzW8cci750fJpguRTlFxqn1eRYk2pBQMLOkuwhFXIV2orVU2xqjS1zHyWbQpvlTok12dFoDxEykJDFhHg4euzbuH+1ikUayvLCQFErqTLHxOVCpNSmbOw9FoTUbCLLD9AQdZLagjJ1ZVSaKCkpjab5OKvCV9jkAVUrf/N5CZULD5HyFTGrkdyw7tYvUBdxWIFViOEhCbXT6WQVMpKXUUsbU/EBdpdtj+86D+/yCfcRb+T051DVc20iXscOwg/rpwREamViIKuaxSGpESIBHzGMVU7PGJOxIAP1PhDEzVROBiUFIm4cqrANgfcd2niKTELt9wWC4byOK6ZZ5rhfGrzuPoY5tMkhCLSQFVREwlwzkmhKPWmYRWLmP1yaZe7IDSiifW+xeBxvXoQ7hx7E9m21gQmgulxeOLD8fSo89roSjO0U+tnbQhRWLyMGC5znGSrOe85myohkcbkamb5irtJq5dFCOPwOTnYoE1V81krNI0HQlShhiI2Cr3l4nkiPHx4jAiffuSCjnGMGENFtt6NjDFjDjJgIr3FGqZxMghtm+cOFgGoahunq6qeW6ElhFoL1fxdEAhDW6dI6xusuVOtW2sps/E5xhjHfsx9IEJA9+djzOnhboe3H57iu0/j88M4n94YTCGtm/N2slcNqSn3rHde0q2oJBckb/u2cGMvpKZ563AShXzniSo5v4qZJWSUycOnZzRaDbWw1r3UjGfmtsoypSECXllEIZladtOWk2JmlssMYu3mAyivuy0ToI+q9IVYQw+VCTIYST0BKYIg3weo2fRwd1VNiiiS740iIm98pt4MH1PWmckCzJpjNRnBwpxulLUCZY13YxQyma3PlcXJ1u9acvcbi3uDyOsZEtkSJWN8s3DK4Kpgyb8EEKwov2iQW67lyu28EQlEDV1l4iTDPfn5RXQGEWUNCIHAS/hacNwtXmKNjBWNisrZjbEyUsW5StNEqKo6IGUEWNPBicRQBEpEGdaJCMTyYQZVUGY/LJ6lJvKjSJ8CJonk7SGwGtlff0ist0cS2stXLfn5bH1TchVYPdDr01+vSJRECWyip+epv/osv9m3a39DPYsaJklwDlVE7HCnWkQ0a8CcEYT7HNZUlGYSJGfAnaqAxHihj7m/CF0l1xA2syaiNWUCh4aZOiTSlkyoUNUu1knkgW69IZ0gm1lvEXPbttO2HdePVa1QhhMMVcJDEGoizXzOiNA0SvA0vezS+nb35nz/rC8vILWRMo+nT7o1C4rK1k8xjqZGWL656lOu03zQXQiOffqw3iNszqFmog3HhB0OKLWftsF9+GEqPkNMYw5Q6e77lUNEm8/RWgNcg4D1viEs2c+IKM9hT60YVDXEcz53OAdnzqVYt4jgMWKGSG+KRgm21k6gnKCbisTIRYrpiriAUZ1pbE4FZBy7iZzbKejDHR7i9Ajt2uwEEz8SlaKA0lrN4IiIWNvMREQQmPQQAYerMDikNRKi90703qCNsDH020/7d989/eSLi2nf51VMIqshQpThpapQFVFEAepkslmIVe8og5EmOIoIf9WelDQQyT5CYKYgFvottUmqunOoSB5yLI1KbdoqCLuq46jB++IHwfSOX2RvelcsEJIFOrByE1GDrEX0ZdBzUHIULZ0yU51EBaNKy/AyS04LMxHxmKkeTJVU1dEVJwpKyOCctr/5Ayu4i4lF3ZpsFVKZkkJYENAyuA8AnuBVherCneQ228CSoMaKTbiNuGWhomnAwxVjUchMYX11r1KWsO7sCs1Spkm+5moz4eRftQI8UNbGef8hImIq6dUBliegQCDZBArTcmPBSpmRYrEUqJuXDIeLalNF8pQgVRoUEbPsmBbfRPdkAPDKR1Bqkjz/utRSCEBUArWSprhfva3Gye4jq57bXl+RxPvMWJr9fHQLgHrNXpUp1zdMVFBuZkv5XFb2zISbG83UY/v+Rf/lMR54mnIxO4VIzGEMgNNDdSKNeYOiEi4RMefQAlfERI/wiTCICiYlfA9MEZFc6yoqImZ22hoQkTg2KZoDQdT01tYGNSWCHhFmXXiBUSRO/UK6hG+tnc9tfr6Cw7qECAohdZOQpW8XyJyRm+MWbBKg9nbqpzt/eQmQc6oEfMQRJur7PgLqrr13U5AxKQzMMIVA4ojgmPuwt19Jf28t+PDU7WJb+HXvb97TR4SIdplzjkgfOpIaghDfZ0QAs5mWn1/iGZOtC4A5GBHNDBS6E14BK0tbE8wwynSnaeoExbG1EyeNehwvbDopaH1rrUnEnHmmxFRNp0cKXKREfo2wiOnD9dzNe5in1TkITpfzRne1RjJHcSk2I+DUZmYyY1K0waIxOCRoXSMmBAolcmW0BOjinC6hn6/+h4+Pf/LyxXk76XwReijDc3y0XnBJcWchLqkyJCI9RANIUaYk05kHWLNLyJI8d/Pi1fmZWQRQatqIGSBQJWUIwZxpzaK2RPhZjmdHwnR8rxlQRu14T3v7CCJcNOWmSN841JYSVF2fYlAVyQlNLi/KJQJZ3ENhLFkGkiGpuCNVLX1x1OxW/yYCIaCKskQ9oVUdBnIPb6VGesyCbqTSh4eDt3UwC0C4wdDZMmpNNq/EImv5cf3oYpVziKVCG7LeryVYpN5IyNL8rJvz6uS2gKtbGlu37vWfV0E8Fp4tEJWy7C4eiLcWY+V5vDYumn7OhZJEJcjgsueQpG0EqE6ZAugymC4wbckJwjPbJHGdxYrcOpZ81siZ89TmZty+tZI3KkNEspbKFFQfnzdsqQ2yEMo6f1U6OV1WnqCrNVit2OpqqmOthCGJKk4JJ6DWd7XfPPM3h171HDDPc7eZnTZrXbWmH1pvqUsEAkFxCgNOYfg43N1jENS2qYpLkG4tpX5TVXo/9b61fhIwJdnTCbUak4hJJncUAkjedwZahNPC+vZWIT5nN7P9BeOqgJxUty3YVKRFcIaqQQzWpFm+5Iipym4yn5/358+q7CfR3iLCx8FxNFhj1ynG1hgtHMNT4iENpqLh2ENchYYIdxn3X13/1Z8/f/0Xj/df2pv350sXgZn64Yg2nE49vXvb39zL2WAips2MTk13gdZVSDg4fc44RgKF2aJOzoD/SNQNlKF3NFMCJtrMktxTUzOzbhSXbkL342Ueh4Hq0499juHEDHdh/Y9St4aCA3vMMWO8vDxu/aRqIExbU8OkX68cjIlJg5iH+HCEZDGeE6BzjuEHCgXOklrcOfYr6AJq6wRDSGttu3va/bvPTz/88KLtUi2/R6FiUspOlPSlgonV1P4aMV1IZr7KCjE1qbHbqoWldH8FOLCkyZaHoYpArhHcJZuUlPODBD1PVCxMIFI2jQUeS82vaO0CS1ePDMB58mpa+BaoiPQUyDhkalq1tPL2Q7x5HEQO+4qU5hOg00s1u36ygsmKJAuqqCI7C3SAHpEhaPVJOS5XHwpWmrSVQQHchrKrb8qgVcLxis0FpK9bB8m52yjseZlpA2K1lwvpbxqkmC6mJwGBStNZRkcwGfiEqXOR8npYrAxRCt0KoKavXHdSFGTZOgBI7+OlGqrPQcEoFXGzj0r+GTXpJvLKkayADoGISmgaB93uv6qoUhQiN7vVxcxLwUulwOWtTwpJ4jZ3QuSWPLIeV76AiYPHeomQtEm1RCqrJxCs4fiq+W7ftBa13VqoErNRGYS5tk+T//yC77z5toWabJsqNLszESqcItokHSYg4UDAzBCcR0ofD6Jga0YwBj2IoUJiV2VrvTfbuplSyQgnIM3yZVYaIkRvtUksKa4yopmaNffhpHB2QZ/QfSJE0NgaVOGTETBNXrzm0VVJT8m9NcNwjKP3S2/n3s4wUYGqmVqTxkmlCkW1MT1gNaRJ0MPTcGm2uzvd2vb+i8+Xuz/89Bd//9Vf/O7n//n18rU+u/WTz8OtH6QKELB2Od1/QDAiPDwjUyT21nouwVhD8CYMTBcwfITP4EDJQT3ojMlwCiEuuRSNoeEkAxE6pEcYPRwtfUMPwzAJ5wiO4GQ4EGpICYGorasSMXMfYx5OT2sgs97NVBHjQLiImHbYBuuyndrpjpTrvvs4zMxUSc8tb5LcBlTVVIzOmBMhonYc01rXts3QHz5fv/vhIdxMGyMEMFuEGKO29GYYrUCXIVfIH4XTWGdMJJHoPDJiNzIsZzAKyq74kjVmyt5Z6tI68MFI1zdJDdKaCMu0U6JP4nZRWSEzBMwJfyIXBkBQiz8WplHVmNSM2KJ8C2ypD1ugBFZ5CSxP/wA9PHOD5hcUAnEbOMp25ebym3Xz7Q7dYmO2f0kJhlPI3kxFVUyr0uDaBJAXmuDcYoaxGpbbx/F2fzTTJohbikhsINOzQoVaQTwCuXw8icobsP86QlfZAmt7WpXfWRQn6yvAbcPlSlTFoVROwiq1o3JPPn3U7MlC/KWuHwneSKZqQTbvaiQFnLXi0rhGiFWr5xJIlBBtdU8QkGIS1ZMuZlgrIQpL4YL6jQSQy9SxAMt10/MDYy2SkLq5Ba6t1mz50uZfz11I4Oo2KKomHkplwIwa0IC0dqD94YX/9IwnOUs7S5ioxpzBmDFhqs1ULMfQwhkScx6KEIk5p0hMn6Km0N66RHi8THcEEHPOZ4kDcWhrXfWk4nGdMZNAltYoJGca9QKi1tPmKOjWNoiGKF2gFmOQozXpiPHxM8cAOWeEEphgrpK3AFShzY7rVTlNlSYA/JhhEfs+r0dw7sdOentzaqIizfehAVUTn6Tr1tREQguuzClcsX46t+5H69/u45/9/pvzn8T9v3oY9//lww/6/P142u/f/3TGy/E03H0egXjgdM4J1VSicXoAPl6stzGmQLSwpqB7NnMqQLgL4bV5I995lRY+oWVr00BpRks9UC7EEoMO983s3E/nbnKNGEO2E9PMwOESql1AtW2OJyXVtjieorWAm5lF+DiSNhCC4qTrdlHraK317BEH5yDImCkminA07XZSQ0LgrW3hAxDOARGx5mFNIP3u4fnjx8f96WVuaeAH+pyiGnRZJ7NgCyb0oUDaZOiqxkLVMsDXeVg7exPYXVTfj9p/KbwlIioCyxqLhQTC1Ors+BTRQhFu4GqWT8UmoGyZU92RlXWiwcvNiwv7XoCE5pD/gm4rqxG05P6CqsKoHFB/u6r0hfsypMZTq91nuhcXSoDX2hgL5+drh1CjRUQBBoKScioiXNdXTSFhMkav3DJoupZo3hAFWVKrH5El+RtZrZikUUEKVYMut9UzXicrBVqmdivGFwtBvXU4GeHk5rlTcwyBMKlbnNxxXtyCUnJpLlWkFkylElgISKpZqqtZ08QQAZYRKdlSHxREIqHO0AUEidQu9fLTqWydL6uW/Gcl8Cq9610ofCxLjHw6+ftCWfk1kVAxrLc7HTy0/E7zT1GQFSAKd2C9O4k+RoTAcrWeLDupArRAK7CuPU/8+tF/GKejv4de1DbEBN0jhJTWpG/pbjTnCMYcR/hhIH2Wy+AsSCxnR30MOuiQORWDDLN22vrJtEtKvOHSRLUo/AA9t6r2CKGuLaXSQzXmPN3fk819kAT8LNpvzuOouVWEM3cyK8KhDGsWV6epQMUQY4C2Pz7FjDheWud8GcpOUrc3dlJR4RicCAoD23lLF2BHGMw2se005976FtYm+7dy/7f65fNJ/3A5ffOV/PnpH76Mz8prm2jn82bWuyFc5sg30gyzXlbGPug0t1XOxBzZq6an/zRFhFuzXB0euQQSZrnRQFyCSFmkGUWgEs52bjKjaQewmXZtjJH76xQSFFJT7C5g0NVajEOhYm36fHl+vr+7bz01/ocg0pOaYhE7Wk/QHSIiJ1PxY9cIW3AKAyFTrasoq5nT9KCGiUpHxCQxVcS++/T88eP+85/eYTyGDzVNMDo857ckgfY6z7BczShUMUFN1BToUWYqK9QGQ5KvEohU+iyhOhEZZ0GI3KZhxUR8hcX1mVxYLlYRIJLGKJ5oUlKPAqiVJJN5JayMs/5+Actr6njNIxSjmW8ECuLQtGgpCjT7EihflwCzgOH8aqvOXXGxikUpHaggSfU0xK2UkBmron3Ci3W1QuK2ZZcFTfOmxYw1MJXe/DmEYwogbtV0ZV2KCmJZ5iQvkk0/kueW0s6kz09WrRXLKpFktZ7XUzuBak5wsTpCABFhqpWsbp5O6zYzu8AbTgSRDAcANXIhUZ7OFJJCSw+Q19AC5CK+yTIn0bxlFiScizCKNG9I/Rx1WUEg927XK7C+ZD3I1fehTP5UCElRFyCw3FUmq/ktaE8inFIfsLJ2thEL/kG9TyJSixRQlIeILGE+RZtb++EZv3nhk55o5wiDMjxyTjWaqhi0rVbNWd7RRj+ARKjUGrQ1EdLpk0gaPp2PZQrYFB1xUumQw6eHBWBiMNC9hrBS+TKdvQkhKtK7UgSwto3Que+9twBPyk1dRbBtJjo81COVfSoWwmDMfeR6v4Bs3UQdkwCFY16BMUFqUz/2cGi7k64IgUOsMWZalIZ7ZNNmdr67D60yu4uc3l4eBn+N55fzzz5tf/r57vzNV3/xRw//8vOP//QL/9T4afjLhgaByjZ4WIioWO/WNW0hYoRp249DjJqmalVxUWFUIptDKhTiQYGH01SIFjFmUMBcb62SutoQ6qVBIC0QosIGRPaKkaV8vgsOETVTyBEuIqqneTxTm5NCarPObYydpGiLOYDHSYJnbKfz23e99+v16fgcjDl9piDR55icAlrfErWgBN0Z5BS9mMimotJP9PPHx/3bj5+//upLRYsgDAByEgIitayORaXlMKSKRripMRi6lmctRHjp+qp8rVi0DEFLqELm3FdCbiRzwEoWRpPus1xK/EJSVptfyGuU/FQhMCDB+jUUlmGkzmzF+XVKpcI3UTrz5LRZApJCoyq8Q7IPpoAM05x9J1W4oHCQr5e2gopUGxECzWyXNZbevo5g0RAMZ9kPJZ6xeMTIBecQCE1bIhuSSD6R80n1iXxttcq+qUAHjbLAy5CVwx0UEUX6/mXykVvNnl62q1yuQb9C7iLW3vb8tEyQica/DkXLcttZiB/KWm0hWrc9Pbgl3Vx/tMwjBCFYehlIg4kQTm9ioKitsRRTn665YRVUtcSzikynUpNWCrGVycrHnfVjK27XQDJKJJYp3bWaGizuN5/i6gZK1FXg/48aXSQuWTYmt1HJHEal5Iu7ypGt9UPat8/zm2tEu6A1mo0jWrVHEDOFpudfONyhDSrmMrO8CdLWwALLApkqffqL4AAHY28Sm7UzcLe13tvjzItuoqaQ4V6gnSlMOSkRAgM10hBGt0j5opmIw/1sLfdLa7YBJuEuTdU60UxADqEYhJgKBKf4pIa1sx8zODBDharwY2zbxTadc7pTKGimLnpuemqiGhHiUGvRup5PNlXGLv1eT2+fHU/S9/M5Nhsdn/CT05uf/PzDn/3lx3/58+u3X+7fb9ePF93bNiB+vnt7HM+UMJziOK7XJ2qP00W7xTwQIWoJb3g4nNpgakiVo8pMMZMEXSiIGSYa7ro1v5UhuSrEZPpQP3o7mxIxj5dn6gViUKgqh2t5qSV9qSBFGqAR03323mM/Wm+9b/7yQoVQOSLk4AHbnhVfWOvbdt7bs++7BtVs1WqYc0pT1Vw4Ixm/VUCfYwqArXfh5ekYf/jh8c+u77uV3jMR3OrcJRJNRqIyVmVpFsIZ8pKJQ9ZeCSCs4dOadwWQ4vqSeWQsfoVB0kKi5CyrrweoZuEuZqu3r6oZJR6RG8hUaAZTpHqDp8CSmsQNelogt9QgUYEZZepABSgp0KwzK5WuQEBR4D7yUKPKXynFTCEnxU1AVBE1x5TfaP2yvICEQG59lgA5BLXQIyIRmwq3ab4QNM2COvkwigprJJ0rlCFT23qpctosLyybOaqq3EyCAKzdO5kAay0kIWK3h1XTf5lK9HUkTURzmyIBIJIEeu1rFqCU/1+qyGjJ9K6smQ8SN7K6ujdVzZ3xbWjrElCbMxQGh6oGk8EApLzxCNwagRudcevWqplaBcGCgWQlfQJk0MMlh07yi9a7l99w1f/13Yo3qJFjoqz7ZO2qXF8nE0a+JcWjS1U2po1qj1P+5coH9rBzBMSQDJM2JU1NTVudmeRdZmSr78GMSWk+oDl2FSMTsPqBeAF3hbeml1M/Wz+penggEB2uoQ1CqKksWksacqeXak4dAwwY0SO899NxfQTlYidjcI45hgq0bfj/cfWvTZIdSZYgdo6q2XWPyEwAhaquqn5Mz3CEuztc4f//D/xIoZCywuHK9rCnq6vwSCAzI8L9XjPVww9q1xNDSEsXkI8Ij/tQUz0vdY442LbEUaQSGU6ZE0rjogh7Mx233IdfLr31DG3bE/3S+hNwTEzJ3Jy9tWtno7xrHvPtFkI3mffY7zMwk/uM+ObpSA0PbRa53eaxPf3+x/7035++/7vbpz//+pf/eP3bP85fv7u+XO4/Jfn0zr/8/KPm2xzHHEALVuonSda69spByBhTSZlTkwB6NxA0JUJnggCIOp2hccxmDjc6IqdtDR1Ov7bGeedqHEosb2wpJczLfwgx01ITaDPHse/uzVt3s+jbOEYU5cxGUZnab+P2ul0/0L1vl/3YpaNIWMcGDRoUEwTRKsdNc7gBc06lb5eZBC6Y/vHT66fP9z//4cnjlnnULsoi/qA83exn/E9hVmIpxM+Z+mx7qNKqWkFcC2lb/WlNAGRpbwlU1hGwloBbKU0fze8qgjiRhlN2BJIlv15VrSAK0SxD9K+yOz0OhgfMs+Z+nj6fhQSHRMDpK0wVZAUEFMB3zvmP+WK99mf/uMb98sysE2JlOJ8pw4+yUxnyJY/C2Uwvnf7yQFfqnJGCF7PAImyB03Nwdo88FY31A5HnlCM+jquzxV+T3Kr0Whc2E6xU+eqRDcgFJlWaMtc1RJ239QVt2X7L1MLHVa7Trn7KUIW8nr9Vp0hKiEX5RIS7I1n+wbMok7As/DpFY3vTdYtDMS59i4TJYsxq1eg6QS8Uo6GMdRxlUVN1OCsEalnUsED6UrOeswkkBG0tsTOzdT1wHsHr+q1LxMcKsTUE0MCUwQ0ZKGchZLYkuFxHSVkn5CuZygX/ZdePu93xXty8bVNpLFU13ayMu1jbXsO0lshJkkyAmY0xemuQJwSZIMZEHoa7GKZsgCPePW0OvO7jfmSiW7uUoM3IxpKxTmEInvEGfmtWQTDGBpEx1K+9hILvL1fP6VDZYQspNvNkarzRnDClSQETrdWRZN7c+7XvxyHlQFQ/4tZ7jsnezS1nsPVEjH2vgmo0aRiV4x554BaGnnG8Hfuvt5fp1+7fGacsZezPndftr6/947vv/vf+zf/99g9/jJd/tts/zY//fPvpD1/+2t4rP/3QLK+/+2aOYb2ZMvZg7+6tQryohM2IpLcpuWAR3po5K3BpUt3sGBMs9XrvvRbJMGN3Wnt+Cuf18tT7tnCfwlXgKtpFaXCzpgiaR4RJtB4xRoxtHs0sY26+3extzgE0+Gqr5xy31099e26Xy3Z50nHMCOWsNpGAQYqYSLOsQ4uQQuY9IxKatTUF/fPt9vMvL3/8wx/ctowjZ1TRXgPuOaac/Rzczsygk6BFLStfIUBcgBC+rjrE2cAD5R4q3fdyC6PU36eeqKJyzRiRpZSpunACQqe6utQVdua9V9drWJ1gYV+nYxur1T4xaOpM6F1NHRc4LJyw7Vdc9yyrp0CKOKd6nUFkCyUvynD9VxXBKi18FP/qmfH1G9lal4JTuLKaT/D80PUj06iMpZ3PcrqDhD2sdwQqHq3mhioEJCrlYjXs6zBXYLWhWr13/SCniEsUYvkbTocdxCJE1wdAERaZueaXTHtkaD5OwTNTQ1KJXpfMX8yM0o+6WQnHHvRsLgtMvYhq/zY/dKjF/ZvGDXJZY4gjcjSmYTHJVt03mJGWTCwBZY1CprWEcs19S94GWWXrVC9RhEmeetCvAGTm45Go2RDnN33c7pKqLlmQFqOjYuvW3q81IiymmjC3fkv78Q2/jC3sOaOBaYiKCkmDuSuQTGRkRKYKXcTD0VDRLL0JFjFCMweEg3koj4ydiNZoc2y8PvcG8j7nzJaAu0UqxmGdtWd4ztm9Uhc2N2vbZcYQDbKAZB5SYr6/XN5fLsAOZcbsVhoKg5nDp+RgekMKU1Iyg/1S166s+P3pOvY0w4xg8apjN3si2JoDjHHMcfdptA5Ea82dpBh7o5nGLXdJx57tfV50y9npzYkxjufr+2j3z/udz9uXpz/8Et/9Ffx/vn36zx/GP+J/++f819/PRn1Gi+0Jvrn2tyNvBqv0i5iZ3cULMmFbSsTMhFuHGTQFbL0fGZgjc2aobebefPN5jAZExHV7ylLyk5TTQQczzQQ4TBp1C2dUFTIDnEhTy9gjD7u8Y4TIZjaJyLScMBfIUL6+vvCn7fJOtBRIizHNxEw6FVk1IP0kpmrLMKa30hSaG6HL2/7285fjdvDivdCImLESq1YV5wI8lTQ+ysp6zAvx5AklnJCFtBLVtAivtVR99ZbiAssKez0B8Ty3GxaSJOn80svidBbb1X7r63KP1VYbLZRnhk2Nxjwn/vOfpclZbypLcVN+4BOdMrOs1H6AZGRW8v5XbnYtSzGck8bXFtZWnT+PjDUWAOJy5mJFrfKs+VDZOqpfJrjiFR5EtoAzMcnZBKxckwWkLF2PUjSX6nSHmdU67roRKxyjBho9DsU6yVb1XZ16ypbkFlCZs+FLWb/wDi5oBDrZkpq6HoayzHNqMjgrFKMsx4yZ7q2cH3nOZkWn1yVl4UpGk9r/4/V9vzx9aMcfdHxrx3Pzd5feYufxMnVveTOkwayhlvIBUE4uyWUd3QjFAm3qiV4FXCc0pzM7DjVv2oO1yIJ26ggsG3w1BlZH8+OcKH3CV9IJ6+ev6Is6HOikzC05aXDJ3w7+9a5Xbbm9T9tkFjMQyFIAOikoI6LY34obTDGJpAKVa5El5ArFAIQIajLvzB0IWl56f7psW/MjYohHgs0SUV1MrWEDAaZbMzal0djtMhVwKLDvh6k3sZI1ny9Xy19ouGyOKL0H4Q5Ob520tvWMzNhTCVoZOCWNcfOYvl3oVjLZAJkzxjDVbsMOV7weyLy/7tvGzXO7XFyzPW3HpzdkM2/NkRR938ydYHPRkXq7zzFf4khEjin0Pvv20q4fh36iXd/bf8jv/tfnv//j5d8+xK9/6tH0Zqmnby5vr2/7vKv83eQs4RrDMmUZBnJgFnBhUIeQQEQAaEazliFIub/2Sxsvn/p3f+zene5OpVw02yIPc+aUSurUHLnWandzuhtHEPM4cH0GjZq9+XEgXWAUl+TuGTP2tyC9P7PW+lGkvLuRkVJWsnUQ7m4VXEGlxIi4XDeDQX3M/uU2P7/En35/9eEzprsVKlV1a5n8M0qAyLUb5KFqs0d5XaCyPRrXVRpOIX/tR1qZBZUCZKuKrapUVah0QTrx9AI3SisC/AaIOdt7ruOTmXKnmedi7qSUVsjx+olWaV7ARFUcVWWIWNvJI1OWUmFTJCrwLlflOEtleaNPD1n1/4sXWX/sUWtV9bq6xaWJqtyadWQKOL1EPEcInRELKUWUs6F66iSQke4t13KRU6xZhFkdcPmIoLAsDPaRsrmmNJ3lPmnMEvMQBoOtoIQV9GK/GWJO7uUxIhUrvvh2YPkTKyQDuUS75ymfSqPXdU4gFUyuM//EjrBkvciCgP77/gT8+dnHz/H5j+3+h6394/vv7eJ4+4XzNfdfPd40XmMMN8TatGczwiTznnPqMSuVcLWewQWGSQs1KdhmWRoeezpFQlwqjvUh65atUIivbDBZi1VqwfL5pK27WuNbZYmtoclasP1wjx9vdtOlFmavDVzdfRGGHRmZQqYVpc+1waaU1AlTVspVZMwaG6ipeaOGeQVH5mb+7nptzX+930aSdsk0eJHlpW71PI66ASSbNWYex1ti0jfrfd6zbb117Yr3z+/ePV3sLUmhGaTETDiScJh3KWEyWNTlNEAxF98iWcui/UOwBjBjxpyRMDfvF8TBTAOjLkq9JKnm3yU+GR1PH47L07hs1/ffXPuG1qe3fTLnmBHIO4BpUAOcB0N+v/f2JTL780/f/tNf9ad/eP7Hf9Cn/6Rf//Trf3u684MPv+TVXMTIOG63QnZZOV0oXKXiIKpeRaH3pBVdWbt9ALZ2JUK3u31Ti3oYiRSi/NooVnmoNvAABqZ58w6mYiQsxpiy+35/f3mC5dbb7jaOA0YlyXcJsJE6NFupDQCtLMbmEuvKIgVNti6lu586PGZECs0N1lLb57f5y+e3v/vuyc1mZR/VJqvVJJYZkkCt0Ckx3uqk1wR8TvBc1OACT0sVsYL8ThoAtEdC24Mw5ElpPibqBaOXpcloy76EXLTEMlJWHEvhSJl5Ag2Fy1m9o7mC7JfME49SdfKg1e65+1nfqdBZtYDa6IdHk4tzAjrJvNUVL8Snjr7lCNLXRrhZK2S8SO9aq96sVYTFWY0WNESwsh0fdZclhiGa+/IUl5swz2+Zae6nQU2qfpyUqgE/ixkeMwXzPBJXV7/C99aapFI/r0lQhZvj61BWZxXXOXyeqli5DWdymhkrDYzrtLPzx6neYJ2FUV/qdFjXmUCapHbP6/G6faF9ac+/mF7782Z/93ff/O7y7cxx0/0Fb7/04+e8/4j5WXyNTCIHzFPNjRW2CjPWAgSeloOzO9FK+0GW173uaH0crvyJ9QQsUornkk9WzSswiV6jMkHQU2uZTvUCVUIcoNISdGdre/LnGz/ulryQG6w7BCSdYYAtMffag5YpydBQQg2TPM0cMyGOGAu3SikOzVfDQUOM49rx7np5t3UjbiNmOFpndsiFYUZlhgMuJmMcacLm3Uw2TbLm3tpabAC17t9++/7b56t9Oqo8WGPErPuGOF+YkSo4Iqs5m7C2ADRrgpkzx6yAphyHQdCQnP0576O7yDbn4TXqZ7ibxq31ZwRobaKPpw/bN/+YL7A02y7CMYQ7hua8tCZT613kccRxRIxBAN3fePn4/P7leP+T/dN/zy+/99//vf9//+Hth2+Ov3zTXMe+WcvebaS7+XqyJyHN0djTGLAjRt2dhrJU574fPFpzmTsYwhRGxi5McQYI85gSPZGQUx0numqnGbbUx2ybEMe8z3Zpxm62Nd8HZIOSdMmQta5M0xTM0Wj03la2TXNMV0S9aDFG690IumE13cicQTdamr2N+OXlfj8ul94YhYejdIvntKsVEMGzuBukmk6TYK7FdqYzPgxnlA+KIdKZB7zEJFRm+SnqGMF627F6qarWxjln4SFfKeL6Iy48wpcjV1paRRokVnaE1fbDPEF8nFDNicmQdcxY40mrlkxDKbgxorI/OWdB0lbU81kH+Jt5oBjCJRWvc6Y647UYJzE5Cw8wN2GlpGm5AVaqXSH+sDOEZTkfRNWlg9MS52qrhFkB9zB6hZW6nyxLtbhL/MR16c5evsw6OIOXlmkfoE7R6spS4uM+YpmH18GGdZDw66qDNclBXy20pxJyyX4gyGksOfUjQ+FEz3DS2PXEKGSGdgzs85bgnXyxvgvDx3/55ukf/+HvkNjvu+336/3T0/5X3v5d+99w/3i8fZkxZ8ZEGrI3L7WZ5nn4nIZewwr2WeQPtEiIwtW+gp0V6VmP5trNEpFmnmeYxPJKoEDFE3o8BQwqT52CCgzystG212n/vs8v6MEto+IPM2ZY2742mmvl5dIFx5ygQiGitl9b45wJTrIXniANcCKmqO52cduMz73f9v0+Y6bVKTXH8GZsDZqKoUi4gRapZrUUsYm0SHWjEyEoAHXg+3cfEJkzoGmXjfyNEV9qfTvmxAppMYOZ95pV09h6L4O8uaFCShWYUjfC59yBIOm9b7q6tVruDtj++dPT+29zxIiQU20bhx374fCQ7YGg3ZTm6I6tb6YgktSMgXGYWXOqWfT+IubWXnn912j/u//u7y8//739/p/ffno3fvhOk+2Cfruam42MiAnLzGB/urSGIyoUqNS80ZrNOTFPp59xhddPAQ43NWN3ujOJOWuzWLIb4+QVk5IUZm0eN/eWGcdxH/1yub5Hzt673fYI0BLYvT05kUzM3cw0RXrfnsbYl8rIO2Mt9ss5EsbuXJwrQcTMsgDSL/s4fnm5fXl7//T9ZvTUUIawQoUAli1cv2n2bCm9S0eXZdm18wyrF2ZRmwtxX8Ies3ZGb55gzKNFQvGLKoPxEuSQFL2w7HoVV5nC+VfS3SqSnlxcxaP6L5h2Ffq1vvdxquHU2+icTk7PGtwdykdiHQt1LRnlyR4uLKW2khRQQFZ8XOXyKyJLzohYh/0JlBestuaerwjYOuXqZIuM6oNTa9fhmZBTwA7Oua8uXzzkuecoUqxmAUcnvn9CHjAo5a0u5hoU1o+2UKmltc31iJ4U6LoL639rTszqFb6eMgQW6s/ze57zHyOivl1BF1C6u0oif6IpK0oGAq1FQDObPVvfZvLjDW//9hOenr775//8zbffH69HHplx6Pjnbf/Rjh/x9hfsP/nxK+8v2N9y3GdmRpoaav7iuhJL6l9uVi6sqhr3crKV/KFOaJ5PaYF7LLWI5rkJJpdzqtjyelFKqiUCWbNXBQU5SO8H/KcdPxx+wwV9y7SIuRY/UN5rFVqU9bzuW/AcyUNMM1IxVTlxMMsG3RUvS/1pNKVDl4Z31+vW2s9fXvchsjZChOV65mg+xi4FW6dZDDGSaED49QJYjiHb5pjbphi3eWuWh5xQWK9sY2ttG+cW0qIlzV3zAACaWTPzPPaaC9mgCGsNJuS0XGJWM1qOOcJmWJO1jjkFmm+J8JlxJJTWNzOO0OfXj/vhF2ncDynnOIi5j9uTv0/RDZk5pREpZBdbuzjUtwtlQ5N9u1/efcz+s73779cP//L289+//+E/7B//ND731x+vHoy7u0X6fr/17ZLIzXpCUTkbOaQJ9mvv97EXKzZHbB002uYjRh2NzTdr/bgPweiWaQDglvOwsgI0Vnynb8/IAxyh/Rh75rOZd2tb6/s8Ugkcbk8ZB0iRc7/75YnFYIMh5IzWmveWM1ePLJlk7pWG2HqnITWbG8LG5JfX48vL/Q/fX8w88lj2Enoht14oph7wbypL8ZmZcm8LKIK+rsP4qsLEggrAjCTT3OcYdMZyxJ3yfJydeYlJqqzWe6nkMmARZT7C0hoVq1eS1cVE1jn0CPaxRSGc9Bx8qehKqp8La6pqXl6ndYrwEY+qkpmm/NxmmKrVNCtaUjgXTZ2oUP1rgQQm1kF1amxk5jxB9MISjAWVsFqoIkFXcpwtG1ydO1a7dEooU/c3q9MSV6jD6jyhU2N1/qOH4Q2gLSnO13P16x9DKZROp1k+tFA4r+tJWKwDtWYalf9U/8OlWBH9KPXUqvLnLAhgMb4g6rIvPEpF/KvNYzKzm0x9IGbo0377f//lX/7hy//yX/7j/zT7uN/yHrnH9+3+x56v27efPH9t+YnHZ7/9jNeP9vbrfP08x4E5GEeDwEDCGMvOvjibx6GEhXsWFlrVH2fS6WOzwzo+mBmEaDU2BmArd5zLtU0SkZXDZGBCbn4T/3qLj8PDNrDFGGwMgL2veUupnL6QsgSzdt0oyqqeBSEn1zga45W4U0fEi63d3PHc/f11e7q0ULzNnPCwzbwTfR6zCkcBqxJ66wkC0/sFgZgx5/DWWveRmfPg9TJmXi7X1ujNJtg8j7m3/oTLxtwRS6/RWpegsSsyMdQ2N0tj3o/cLn3rcxyFEdbrS5pqoKxdTt7MvEGakyJF90ZL69f56Wezbiklj8SEbXPC+PzUPzx/2Pe+3zYTGr3aZOVIIVJtY1CSbmO3nADm/paOm2E8XV/693/zy79/+NO/vP70T/PTH/1f/uP46Y/jPo9bzgnfjhk+YkcEKaFZy365ve023a7X7eI5w5pj7DFjWF7YAEOI1oSuNLKl5qwCujmhGHflJmCqgGC59Yxh8jkVPVNxsTZ031o/5lGJZDFubNdMs8Y5BqyFwqEopDESnKdXfIgmjchGs5KIRIzem7VmRqrHaPtxvNxiRGveqZ1Y4tEA/VHOsARtq3IpTpMjatviucV34TlfqwlUEw5hK1PBFzv2iOpkCTvrfTnHbjePOSEmE1/5t6rkjyWHq66fb+6JunLJJIt9WE1rUSBKltWAX0teDSuLr9B5Fq2PeRKzPM8J1d6rRQycAF7t+Upnw1qzXjTsqpLS4gZxChzxqJPrZyjKgws/OKeF4kLXobiORdVXrIPzNGkIRgROkrJ2957nkU5yhVwDxNpU9vg+a4pbmkVo2WO1WAll2aEeDjassx0L06vUrrWTpeLAylR4+rEfcFx9mPrYkbUcYnXlrLTwOg+MmWpgmk3qRRrCDroav7x9+uunn/7zJVLbTRZhN6Ffvm0xnnlv2C+8c958/9LmS7z9Or980ttHe/uZbx8vx882XhmvlUEEikYhgNN9xwdPLikYhfIzzsHnNFHbGYGyErjPHeoTFVLE8wxVnlNHIBPyDHy66a+vedNFvqXMbGbKukfFtmaUE6fM29V8uRKGCKxGgkAi6umHAXuOu+GoyRyQQY353Pjth+d95H3GkazwZ7G5q5ZLRAYBWKvMV+9bjmm20TFHemOOrFo959iaf/vtN0/v36fQepuxNwUUdGJFOIwAvbmDEjNgzsrolwRaHmNq1LXMYyqzu0USWWZBuBrcuHZplN1xhAwjbdzQ+hia04hGeL8+CcbkdeuYhzlb2yLRqreoa0NYs3bdSCo0x0DKDURu1zaGcszRrnvn9PaRl7+Mb7/nd395/df/xS//sP9sr78+XzqZ++0eUmZAF2vbhveH75CY3ttlzx2KLKnX5DxSWyLZeNnaJenQAFVZOuaW+zDbJCDVNs8QIyOCZcE3izmPY396er/1y31MA3KmHKlpGdY39xbkPEZvW22o997ELNrN3UM0epSuZg2SNZ1LkWBTwNs2jtvbbe47+rV9hUZTKHl7dUjmXEDqatuBBDwzafLmkbN7X2jBahcJPEwwa4RYS8BP3oNLXrkmBp6dM1ehNJCKCi2TrSXypbmAkaFKL8CZShdurQpMwZFmVmfAA16oMr0YDeEkdldhUqHPWL9VA7d77UQ8k+6BFSrE9XjZWvBbqzpkZgpxrcEpCWE9/EsalFppSOuQeIjLV5uu86wDtGLx8yT4HolExZhWsZQqhcRLD1okTj7cakicAJGtbQg6vWElRaiVjKglo+dkch7+awBZ+xUex7wWC1CfGauNjrJLrY3q+SAG1knyFUfBmjwordgE8je/g4UmNdsuFpfAEHfZW+K73i8JfL7f93GwmbZ2jLajuW/OvOm5c26eLaZt966JD+P47mb3V719bPdf729/ay9/8Ze/bMcnjht1MJMKE6xxwXkq0JConT5glNjJapeYUAal07KYkJ+siM6OoqSiijDQDAYZghLZwuynXT/u2HkRW2SCzVy03loTnYTX2BoZhY+k2AilMUkhg2yJtSvVEVza5QrgpINOPbk99a17e9nnPURzo8mKBXJpVEBNFio3xa0ZE/QEaK2WOWqldGjc7s2723Z7PT7IzG3egr1lwCIXC0BnuSjHVGhF5kemJ1PNHDEjo7ULgTkHIodotpmDipyRyrbsqOHNzSxjmgxS7jf4xtbS+lC269Pz8/e36ft+bNc27rsYdRyKRFafatvTk2trrWlOQcccjWgwCPvbHTOVMZUTEHDr2ws//Izrz5dv/7397r/c//aP+d9+P3/53bM1WI43wWUYc69krRyIJks1szF2pmjb5bLNiZEzJeuuCCYzKlZAzUyQNZs7wLReWhnCumkHqUiza8zYjyOf1C5tO7y1PmZC4fQch7V3AGlOj9KG0fpK91vxOzDrqVk9qYS1uCUAMOeM0salpvhyO97u493VzTyOcdaq6vB0rjSpNn2tZqJYi6XrVtk52uPrSJ2rTTxfdRKRApYVK8s4wxVMwJJ78iwXWJB9rQ5Wamqa1UokAAjkV+Q5q3k8qbczZqBShqoC2VlmCuIgsSLfIbKA/jzh7URlqq8Jw4r4ZWksVKYnrLlnHWQLsJEQZ3D8Mh2dmMjDFrVcc7UPEuei41NflaplysWKPvr0avEtS+GVJZ+s38M6Ekq3SgOl+Do21eerYzaVKPvT2cSfnASwCAmcv54nM59Yq2cW9bNu8Lq5JJ0WyxCm827biaOUkaBa/nVS1Ga3xbisoNnziFrSWCAzzdjYXFNwQ7uG2Lb36dI4Xr+87Lfjm999d5k6Im8pucG8yzq9Gbrpur27RzgynwM5dPw997d+fObto7/8sN1/stef/O3HNn7G/bMdNwtRcpSkUSzQnqla/7y0VXo8c0Vw2LqRWdqHgvwLLX1YP2hiphvHiO3SXpN/u8XHQLQL/DmGyiQJ+pnkXSnqUxGoABqjFJnKnDRY1IOUOQ/LUE5DJo6MnQrSEWNzvOv+zbU7sc86SDrgRpfgtk46mamCjmf5lJFI39x7T+kYd2uLxLtsnel//emnv/34uz/gCfnaWkuAVA5RgTnNHBIUiMSKzBPIMgwDgKYU2C6gzJCRGYRlcyqjhp4JgqMR1kxREUgknTH98nxAE8l27Zdn69vb60zqy6/75cKt9WPMiIR5KSGb+9Wsec+Y95x1PpAkfdnAGxKMUeO8ZRra05z+s18/vbOXy4fPrf+Xz/+yzZ+frWN73t9udDk5FBCsMTShUVH0xhaib5cRusWc0pyRG4ik27LYGFLTJPdH/AvJUtdYluAyGcIxjzGP57615s1qLWcyzbwh78z3RfNFTnNjBmjmiLPNZLVqvvbWlkZSiXGM1vuxH1d30mfg5X683Obvv+tg00nNEnisfspMwur1MKu9STSsdbSntudE3VEZvmcrW1d7LdgQFtSyOqZVd4QT9iiE+sTa67wygGefm3WbcqEyXDz16c3PR9Wrb7H+bbWottpaSiih+ldwu75Fvd32tQfHA1la3kuQJR4FAGbU6tRSB5lUXbnlb9YpLHXQOhWoUsgtv63WJFTgk1ZVLY25Hg30A30iVcFFNRxnjSPVyzOVzuUjW1/lpG50fiWrNKQF09UfqMskiTXFrH6+ro3OSg8tT9O6MOeBsbJCz9Jfk8SJ/izY+7wNj79ZlVP2mAvXz1+Q2WkHRpOkJrqZX7b23vtTMmAae7693b/9PZ5azJ4h3GG51i4wiJGcROvNwMhoZjnfY97v8b2NP/n+T5fxYrdf+uvP7f5Te/2Brz/4/VfeXnLcMgaQmpOsBdNtvTtzMWRnJh3cVvtf1EHpzrQQmK9vUmZSmQnfWho/H/nvb7jFNdolI0CHUWmAo3qtGFQgUhG0WtJjmRFCMgsol1K5dq1QiNhTAxgLYCWa4anh22t35P04xkSaEV6pGatIQHTvtFlPJSpf3LxtZgKsu6WpebPk9d3z/fbj/f768W//ni1AttYyB8ScE5rNLEsuRcCShlQk1r7Tyhw3Uq0/kMMTsVRGuHlWoHS1Ks2xfCUA0Lpz20B3ul+vc9v2zNkwADb/8nazdm3uAiFmaI4pqFmLjG5cWQEGoglIN8SYtUMHy6YJE9xGpm/bzBxPH/6Vz8M72/v84X/7/vjbUwz27WnbNGccu2JsV3cTNDwBS5giknM/zF7FPSJ7kzmRRnRvPMXG5VI00syjIHUCcNhGNw/M8TZHREzbnjbzrXl5O8VJDVhEHvRekEXdTiNFWjNEFqRrRtRYVDJsIiOVyshL72YOzWR/28fL7ZjzXYO7bTEHKNXuOaJ6xtVdnqK5Rd8xyYZTWdi84ewMT77vBHhNAFwmQblS0vSAEgqXWIFoC5BZoZ61nr6k5bUYrwjh9cXXUXOmG5cL2R5AdQHOfGQ71vu4Sju4oIkFhhdjUAmLq5hVXsVJnQIwr6FihXvSKg+5DkUTE4naNCLgBNwlqVitxWF8ZT61bKfrwFIVAC50pP6r1CssiF0r4GYRHFpNtPTIvqgT4DyAF3kBqcAiLqLhgeSfS9OrwZcBgXVtvp7UZcmzR+RR/SQL4KojI7MCTZfG6sQcYeZrf3Iu5O1BVKjwJUdFuZnbOTcuq0ebcyq8tSc189ZINjNxe/n19fPntz/OSWTvuOY0tnCMY+4Bc1tOo0QzK1TevLX2TF1nPvnzh9Dh+5/H8bqNF3/7xP0Xu32yt5/b64/97Qe7fbTxihmSZgYivfgCrz2Bp+gTaliTG5RMOQgndGpfjZloZYU40q/Xqf7Dq/1ytCPfmV/nTPpFrcE3mpVDMEdS8iUjzrJ6BXLO4ipKv0AAjW3BUhmat2VZkJTRoQ9buzbexnibc6bUNrRtDjVn5szMUPbWaM7M3luG4GrbRpqxmXFIEPrzNd72fb/tyv04jIqcQB7HsfWWgmkgR7LwTM6AeRLSIZYDcCGSCRq9gaqVs9ZaKZ2A9NaRFKIG1RRmJKYAy8jsEBtsOwJDFhIa+/XS79Sm+3F/e93zfpQaoXVbhaNz3Ma4yd3cG5EjU4h9Ts2MzHOjkNF8qnY+0brltMjc+/ZvT38Mbffv+D+/Pv3x/tf3fEvEZXvq+/3QjmRrWwbg1ZGlEc09jhFst5kxwQtiHrSyQlimItIrMli1/BqVRF57yQVLya1F3vf9iEu4927W3bM0AKaEEOpuoiJGowuIOb03Wz67qm6O5Su3M98fmqHSYqnyS9se4/Pb/bb3D0+2lDNKO/XhtLMTLDHpitkRa6PdGcFZKDxpKg1leSW5EPNzA9jqZ8XziTAQjKwZV4sAPU+Q+htGe+DZBurRYxmXUa2K79ea83WM4YNmXUfBovDo1Ck9hOxshEng7Fer/p3KT5xgvb6ysucrfsrll/02jQ9Y50ErMzNthd1XamRiqX3qrOZZGXXi69VqF4xSY5Cd3O9vhoPFD5utnLwHV7lGlxKFnj+ABHqdJlWyT3p9QXiVNPnVLlfTHE4GqGa4ryfD6vC1vH5uTquQQ9kJ1tUpHjPcvW4hzrOq5pOIr+uKCyWrLyypmRvbBWarLaQgzylHfP706cvnt365Kslka8gIYcklzWypY6Jsc+ZEo9JMvqG3xHX26zjep37fPiTngePG+6vffm33v/a3v9nbj37/2W6f7fYF+2vEUQuTahoy1MIqydSMhV7U/gCjpQoPs/rFSqmjHNjepv1w9xd1+KYpWFdz9s3aBWeqA/D/v5VMyogoeKme5gxBSSFGcAwwiJDSvZPayA9bf9fsyf3XMfeUvANrv6aKRDK4OYDMGk8AAonav3hE7Zg15TziBWhmJDu92dZdjiEzzGM37wmYr3RMkCjAu8KPEr60EuVqr4WwXrC32QYNYql/2YCAVVa+eyqMtNagac2gmPvNt3d5KCfefbPdmi4Xx1Wc19efPw/O5+vz9XlrT27TbQMUcfjtuG3pndaft9RBIWOAQmhKLaHa7YjlGo05KiRqKI7W//L0jfCf+P7bePndP7/+q+8/QnOapbURGjJVxXWPkU0293l99w1GO9jgjraNAboXomK9NQ2XwQwKLrNigPTmCwmXVe7HPo5jHtfeem9291TNMUJO2iAvBg+PVBqREcRa3xqIlXjipfU2wjKjMj7LF1aPWiZn8MvbfjvefXjXVmPGJV4oQ4yQCYPSyl2+cO3T2yktQZcAobmfkAXOfgUGrmWPOntLnMJSnJlBIHJ9YJ3YUAFKqm9xwloVQ7bqT6VrfYWvTyhiYf1nIQMkleG8UCqao7giFBq0utNmtfAdJ0m7eM8624RaRsb/IW2u5JsVxldEaq5unIuPqbyer38hf1t3T46kjq6zuINnbDUFyrRE96bTzyTFecLVV38cnwshg2DGXIyCuMD3VAaqh2SVl9OClzqzSgv4LaZHa2hYs9Aj1399i7JZrdOKQB2Bp+Gjgssef3+dyXWmLeK3riu06PE1rhnQet8kmwBbk1lUQh18jPjl518PJYHbMWd6BKOMsVImZTmy9lbUSilrSx2ZrbI+3eDX7ZkDLshTGRPzaPMW+z/l/sWOj378avdf7fWjvX2015/9/gXj7vOOeTDFCMfKPzKsiFp3F2RgpVKVmKG5a5+gB/pPu/9l+M2b7OkextbEbn1DactoMZLNcsQ6Zhe6ed7FAitpZChCmtBU3IEb6/Va5AmeN3//9NRa3+eY0wSnd5Gt2zqQzJMhhKxHAJnmDrhgNAdsRhKCeU5s10szHqK3znbBtJRC4SvwdCJp7pCDar3nfpAr3YvrFguSVUOXkanmfm5LFl0K0X3FldcmauYqN3ChUdLY7fLMbrp4u7T9lo7tm2+3eWlxPz6/vby/duutWMsGoDEv7RgekUzYmN1NyjGllDfDTHkSVnAziEoVVkB00Vrzkfhxs2bX7eJ/vNr1h1fhzfvlvu8w7BgWsTkAzhF029wP8ADuorBJmpIrmZYKRgE+jUwYMydqNWBN0EGaw9ykkEfM+3G/Pr/rrbVmNlSi3VpiaZXMnJ51FROh5FbDRDVoaSnAIibd6w2HKSIisvULBduukfvtHrc9pWY0GAUogmWGWtVRPPt3rBpAkJGx9BHnQukFgSyI3k6l8Vd9h3mrKhKxVG864WYhcTZZC5o+E7cKfVr9u5kUa5YAFLXQZv2Fs9rhN9h3dfe2ltIvfhJff6ZTDrTq4OJw1y6tU95xdr6EUosLOaeO2vWzJgAruRPsdMOtyJZlHFOuD78+gpCUPY5VoLxsKiYWwNq1t7T0INcmyEUnL+1Spb+dx7bWJcs1Tyw+s5ZZoUzU7l9PqfrsfCTvrU9XhwtP1271ow94bSXyUQajMSPNzwGlPILVSZASQ1EH/XnUaWVBr5+pWAauMwAk1Myb5E7OjMMEMIY8EeP4+eefPn96+f3z+0QeaSOF8xYQCqI2pcIAWdqcsDTz2hsDlUYkShlDgmK7sG8b3uHpQ8vhufO4+bj5/mr7p377zP2z75/s7aO9fuTtV95ebN48D89p9cIpnWtjgZaxXKbpoe4evH6S/7d9+2Fe7niCXy79KXqnX9mvkOjUHJxm3nOsIRBIUKJZycpWozGkyBxOECGMjIEczTtB5bhYfvN0uV4vU7yNGDNjdt8Wa5+aa8x0Y42lrZT4zdzhFshxv2+XZ2sO6y6HYT9uAcAd2yWmZaRmWNsKB4USNgGoLz4ohrDiB74G+qIekIAyJ5aYvLt765ByzEISSWaZBkFVIOF+pykH7N0zPGDt9np7ebXtw8Ux3o6Q2/WpNTcixxE5o+Bp7+356Rr3o/kqbEt17maV0JEzcrLi5iUaU0iEbcXwCM1301/TNb+5Pv/z//zNL9/+8n+4rG1PGPecidhT7s2uzczMWkvjy4y7REPr3cNL10yx964kMRKMSAEmQotEZW1gpmWAaHMe9/3A07utXza/N7NY/XIoEceBy6X2gK76BRhMTDeL0DI+ErUJyqxot4EYmXsknQbzzHbb72/3nEkvlhKxQhoejfVCZa04xZD8lMDjDP8pZUr9gaxwSuDRXcIWCrw63ErnJU48mlpravGAIAqJXj0nCnp/bOzDia+c6ZVgFsMKipaaKwsOYC3bkqoZJ5gZZqfNWIBBUVhHkcOIGawIy2pXc0UnLOOYraWVWjVQ5zHz2OwrVK6y1j6FihiIXLkwKpb+9AQ8Wv7E42eFtMj8deqgVkPkoxzjDN9nYLEND1caTkYdktKsoUhAWq39tJN8yIcDDmsMWusNVn5nNfrLf4fHqSpYbeUM0dajYAscW2dARiX0WUSSljnde4GFhUytgU7kQ9/KNe3lTABNy96RRM6QSKe13nOOTx8//vTTxw/f/92ceN1nsp5mS8Ssx7Ee/MCZlY0JxKzrGGVD8WU9wgy4mWV2M8PV+5X5rm3pmT4Pn5PjbvPgeG37G+5f2v1Xu7/4/RNun/r9F7t/9PHZ541x9zkEGHJr3nvj2C0i0/u7p7f5/LebfR4X+Tu7PM9o1jdyI+ibCwyFu1kGvWWJ5n67l0MVWpyIiZhEZkyWQwIDy4GfVD5v9uHSn58uL0e+DoyEb5dEBwgFM6MeAvR6Mw0Unc3cu0xK2/rVljMmaS3HNG4ZTJlvF95oQOsXW/rFYQimxCBsHonKrshMlbd+qUOsVNOpmCIDvRWqu6xrY0Bp7jFnNzez9eoSRsz9aNdnEk6wtb5tffT72/7y47y/3u+3t/fXTSPuMxW5uVE47nPrbbM2PKVyd9IXi1ZceswVwiejGahT7DDHRIrZJUUItL/i+n+L/OXyn//X9+3vXv7tHe6hX2H7oc8X/o6tARNzn3uzd9+9zX1EwpuyIUlfZYDpzAqTYAV8m1HwIiAYKLG7tY7MHGPkuO/703bZzB2cGeAkHfKYwTaRtSEprbmb89GvAlYQykqEr33ELWagQmZD1pAhR4v017cx4mllIC2lX7VTwSVPeKjHT2H/wi5YgppzZi00HxmVPV8bYo1gUkaP0n4RhCkqV/kEQgoCWT3tCQ4tb9HZWp6HxGJli0qt8LcH9KRaJisz2AOSKjQiKvMIqohQks6spy6jwGy6mVsoK39dkVzaH4tKi0kmJeWMOC8Bqpd6bLdaHO56g4vQyzNBD2dWfOEwXDzB6qwfzAHOKq8Tq18jM9fKYlQTjd9CXzo5gJPoPUGe+tWl6i5Irb5bNffr3KDpdFafRxv/B+Mcy9dNSSvQ/7Ggpsyu5zqzkt5mpLvl40ORGasxOQ/vrybypdyNhaC1CGBN7zD3qIc4odTLr59ePn8uSbt1Msr8H1AFOlDCtfdWbpJ6liNmaGaIopOevfeIlGVMOrObjUgDdMDI1jYz83ZVHt7fo2d/Ctzv/i487j4OjxtuNx9f7P6L33/e7j/x9ad2+8n2LzbvCUTsFxeQ5tep7afdfzh4YJNtQQ+sQDlv7Tw5vRzo1fxGiTpOLXNkQBljWAbmKG+ndAMGNckGM0c20/uLf/PUNvPb3G9HRBLNYma7PCnH8rshtbaprt2OYDmzDZln65Al18kJc4t5zBknQ90yR2VOldpEmd47yhV4vgo6PSkJgEiTVfrVir5TZnQ6VDz3egMcLH6CXgKsRJrYWr+Cyn3PeZAat7df3ti2p+3d1ux5a8bAuB+aen537W77MeeclkjkiBkjGkU3yEwnLgWFkinz5u7w2ow8rRLVYoLY5yE6ev93XPf+p9uz/V9t+9Mv//Kdv0UcMxAuKq23jBuYb/f9HvMQDiGOe056f4JAtgxQyhH+kH9U/HIFI5GMLB2KwATHmEccT7h05+Z+lJvYQnRpKidsUWTIoDdhqvS0pVuzynOktNKVYcxMHZM+DGjeEbEfr6+3eUxdNi91UoZs1WKvyOU6EM5MIeZKaIgigq1U01VMsrALe1SkEwViKCqwfdWmR+9ZhODJ3n5FwbHwnyV3NFt7hk9AvWbEKBC5Kj1oZNRHOKvYqmPgWY9Vz5UWqaKsKJ8awipfYS4aw72+0ImVr60IqjpY4NiyLqzf0Yqjr8q8XNIn/YFHwS6Z1Xlaryi9YoaXU66yQrkIaq5urTqJNTeQkJlXcVjHMNbhLSz6rcaydVAVs7hIABgYkebrRpzk8vpwCyqo4OF6MMroUKVcSUMtxKGz7MypPG/junX1eChZguBqEkoXVLuKtb6geIqBM9LNG4SsaGykKU1gTIVTyPt9f32773tyszABnlJmbeKmoGSDNiublmatIyn0J8OMDBuIBJJIGLJEclnLExzYZwK4NBo3tuI1SH8XOU3BGI3JcXgMHq82XsbxBW8/2+tHe/u53X+Zx+fL7cc8fpSNd++++5zPf9nzl4GBlr7F1ExzGsrHqEDmGENRxFEAZVg1Lm19Jgu+T8VwU2oAgfJ6LpjPlePS8N37y7t3XYp94BjKJN2VFpqeA5SRM4Mi6v+KWwsZkTNVrn/zY0y3FdsUOU26Xq/9cmk3N7Pcp7z2mwQgM7A5iLgfRsZqBlOyjCRWdwDI3FJURuY0ks1Zi3dJUDmzdQeswOdEGDDHNHMBY4y3Ly/3HPscxz5l23N/t7nN4ZkxEyOTkRmFqxpF9jpDZlJKeDeACFGmmv1YC7CQgilpWYSb0+jM0HbpFiAYvv2IHP172o63H9/FZ6rD38FNqWPeO01tu6e97DndpwwGN2NMpM8czdxR1sA0WkQU/U6rHTCgmVtTjOqhAvN+3N5fn7/m/VKprJFCGTLIoFQmRoibVeA+8SjHtd+4bLBF41JEzKm2CSA9w+9Hvt3y/cXplqPaxAWt1EQtnIhS9ShLk+IF5kIrUm3NCisNeDWbZra42vpPrLWSqYyIwpfwwOTPHrZkmiUoOotWwey5VF5cUvuqvOVjaXaJ2koIVjEt9eO5df3kI0p8Frl41gqoIUqIXH/gESuNejRWB7xcwjhPqZWCx3MMMUJVGAvjX4ljS1KjVZ4f60S+ngpVnn5zRqwTBLV9F6dRYekMgeWmLsa+lDSrjivPH3RJBoUyqYmkwUq9Wqeue4nJ8VDg1JOz7nauD6US4C5mW4nMkPE0rqEewloQQCkzK2znnCRoFUBEmDLgJabMrIik5qUwq6bM3CE1jIHahsfAGDHLiNdd8u1p3o/9iIk5ZoGvZkiSvT5RiYsiK6yRSJpo5kYEEZQwj8NaSzOUn3sqE9nkhLuH0skZxUpas1aTHLWZyXRBg2a60vV93l5tDt1fdHvl/naZt/v9l8vrj5eXf3v68q9+/f1Pn29/efvlZQBXBzusta3DO2BxDFjFP4RzhaplJmpdQwK1JzMSMZkBKueoZDchMA/zZsoRh2t+eO7fvt/ePW3HPd+OOEbSLmsT2YJEteQibjCfmdYuqHktc0b2SwctAG+NNMWYY5rTXb0Z3GhEDPfaO17hfZHdMxPOdmmIzBGS5siNad6Qi+qkWbv0jJhDifTLZs6IMedABdJGZlqOeHgnrVknSRsvr3q+4In3T2+v9nK9fgi7fvO8GebtYGIbY2YkM8c8jsMAlkOTZsVTdG+UQmGsjZ00GlunJKU0l2kB8sbuDQQ0E5ZKumfG3PwXvf+vxx+3d7f3sX+I+8WvM49m194vsX8xakh3+OsxcHHzNuJoLgDeNlIGV9DAwoMBuTNSoNaG1PVKG20bse+Z+zy6s/fu+5iQIEOKQU4SaZYZVConp0+aWV9dLk7eUZScBGsNrJRz4hTXh+y+x+0eFZORUiuhlgpGWrapFHnmBVTRXR3+WvJVd2zxBcXGSlF0MFRqUC6Q4bFf/hSyFOCzAtYW9O9VVx+MwMopBp12soty86z/z+qkprE6nhUjupZZctmSTlcBqhM6u+sHOyErwbvRzApkl2QmLv//qYOtRIrMx5SwwJREfN20JUmxkmQWTsPCQnGyuGf5x3nv+bX4l3Cv1jyua17A3Fp+cNqeuWaI1DnyLC5gfQfppCrqG63BayUig0Wu5Jmiuv4iTzfCmm5i7Tgoa8Vvforipxf0UsHgBE9xFNeK5Tp5jTj/FcUJa80Bi16QZEmatbzfm5tE2FvqYKW8QK1fcL2yWcy4z7gNv/QrvHdHIZKqE7KkTN4akmiVlyUha2NDok6WzExfhA8lTdAwEwNJwm2FG0XMaWAmgRL4eYLNkLbZBrsgAk/fzecbI484bE5//Xy9/fL+5S829v/jl//Xv91/kl2sXeckNtAbLBPIDMwAwxIFU6qokrpzqYxkBGIoSvYzKaUQ482y2gFKiZzbZt9/ePr+qT93vr3G28xDkLcM0KhYBFpVvuLpje7WrbVAljjH5IDTOXM9/TQ1MqiZx33s6+R3M9qUHJhMipmzeQN883cTv1LZnIiiIBtJZZQ61sxrfVPbLjDamO1c2oysPfKseFCIC0zoPW8vFjS4DLe3G5+e310uClDYtus9Z07AmnTMxH2fabzvo0FQWqLV6Z8+IhCzFcBU6nZCE1NpQO1fMDoy/OpjZFIgBqeccptmP8X7/3r903W8/E85/pwtP/9k3QGAHrSXyC920YWybZZaMgbMzOYy+DAiQ1nzp0tpkPMMeRFUu5FggEdqj7FtT83d3TISytRUetYGIk0gSkGUR8J79llW9ipogGKAG91BunfNcbhxxoBbJ0TbR7zdM8IXXVxwB9ee8Jr/SVeKTLcmJM2rYlZRrCixhVGsHMsEvfhOoyqUBkSR3lXhqjc8v8RJgFYRKq8rTinQ0rxX8TrDxarWnDxsNfhLzIPlTgJLPCF3Pj5DRgGfPFfGfv3reGDjj8K8fiNJizL92HlgcC1dWRXsIWfBQ9tS7/IDxuc5PLH0OTRXBs4NN48svRoKF0IiMem2fH9fzbcifpPuIFrkbH7meS/oaf0QduZtFPwCJc2ae2SeE43sbNjPqQrmPNkFnBgdHpngJSInaMbJ84gzVmg9iHNd8LlrHYKtNcqL+gDgVntEv348QFK72Mj7K3PI75LEphxqs21bbZuAzZh6u93cvNNDqeTIjKneQKGZOVu9dQkZrKkWd5SnBSNSMLoymQHNSE2xwea0mUJvRqA5qIxUQt7QhBCKm6BsCiQSlHU8NVa8zpzWv5/f/bPf/k8vP/3Lf73/fz5OU99CmSh0AtDacb3OJZ0brS3ZbIVaReYcptCczCCy8EHMpFx5Mzs5M8zvnq9/+vb5m6fNAsfUbWTCQac19xYREJUaEK12TVRUvEgYuhjlBYEJMqejtUAYwx1ojIg5D9UDE9l6b4mJaCWHmMJmUuzjDQXpgW6tkAHNLDk/oniabNsmiZHjGCX3aW6teeut9K8QEpiR8M3o9vw+eclwV/v2u+/Zvvm06+1tNoTyGCPGnOYm99ba1tuY4/XlrUPt0pu12mMYESIiUqZOg1V/lclovlQZm/U5j3TyyLa1nCFg9Ue0VE6/fLz+/l9if8/5/Dr//H7MeSD2dEd/95b+OQlevD2F3FGIGkovEdX/Q7CkVCuvcm0TYOQkaNaMUKLZJWM/joHL89Z7rSkoQ0giagMwNTWmUuCFNGFaKWrhGWvqJ1OYpZeEmXnLxXgiAbNtaH/b85it91ZU7ZJWMlmqF9rJ+nq92MgsWdeSdy8ms5Dncz2hoDOpuIrmI8jhbL6hgFlh5XWerTPwIRkqfU39Ek/1z1mqE6QKSEA2p6TQb7hZLpqFZMbSIK5zaFX2hUfrgeqsGFHjV8zmN0fE6tytFvyUT21hqWYsKLWe/rp+hX6ep4XxKw/htDPhDlJ+DUWArD2EoYuTN4AVA7EcwaeZQ1ChVRJrBZhOMvxxeq0DbFHiK8Am0+h1Zmdt8z71vGfxX+YG8ExFlmBUqPIPHpNKdc61m6FuMd2UaTCtmfExgiwErgJDlsMsEkBErIDxWrGlbJivjC3mnbVG1S6paVviwhHj7baPiSG9jf1yHA0m2qE5lGPMPtzJ3psr6KUgZgdQawqlMTOBSMpz3nK5IASkJSFVyjrmzGYtgSPWETkinUDAW200yUaLRAUslyxhaxt5aQC6oT//7S9/+ddP+1v0bbskFZgR9MrzMyFqASGLL1v5ombKSAgRzaA5iHwk0Wap73WDBpWkZUxD/N031+/ft/fXvg++HbZPG2mpBuuEK0IoTYvcW87JthUdTCk1zlzDMPSKs5NEWYoJjtQx99bMnxs+VkSPzM3kBRhv10u6K3KOHZlIWCtTrkUqY3Y2M0RMSjDmMc37jMMyoIpBNeutFNQxZrLmacO7nkn6JdvFuF2itaC6z4QlInn7Mo48nt63p6fe31+eL5fI3F/vI2ZtoS0IsptXHF4KmgEAnqnpZRUpHQJkDiSCVGRDS0IokEZ+3TKUOWZ7+nn+7l+1fxufv8HrB8v59nrp3+1mO9ptUtbN349xYGn+qlLCyGR5bq12SVbpMTdlQIUeOyAj0lrGMSIi0929GaOqXphMMRJIpXJnjRPWVxO5Rg1lijJ3SyXYZcwEbcsYWUkAjWSfcb/vc46LuoNVE5Kkrd1ogMTyEmTWMiqvjw+TQPqiCE7IVwLoZd6NMh9lkb4sf3umzJjrmHjUklVQVjDw6m4XlgWeIffrCMiaH2xZv8soxxKDAOcRBBS4T+JhXsOCbApO19c6uaolz6+wgs2UDxVkaSKTC2ioCULnYYMz0+KE0ZcGCssvW5oaqyyUBBlaa1K+fk8iQ78t3af9thiDx3ByUshaBsLzY6+o7d9MMCfRh4LvdZ4uOAPYRHPFkq8/WJAl++LiAtYzbBXW5gsuPa9tXRYzz4jWt1mpQFFPyOmwTsmymVcdO4GudZm0YOra0mxt7G+ce6UruLmMrb9P+ojEdWPbSHNya9eYM22qt0VOJo4IwIR5bdscKWLOnBY0s07S5kwYohQUZoLBkwbDyuFf/DVcyAgZ1gqwMdMm3ODhhQVO5kzQbAU7Ntt9kpboh/HLnP/28ePHLy/oHX1L2wA2azSte4oQsmy49do+NIMZgyVtjcGcYK5cRiXyYA5zmizjLs3v3l3++P7yx6d26e027GXa20DChVoYHVQWJylMaVrvNINxjkmQ22YWOasvRE6kwrcO5RiT7JGkbf701LDBG2JEprmv5qmor1kg1uFe1jiKhdWGN0IRkQSa+VRAivutWidDwamamCZjHJmCol2eUvD3V43IO9CsXVsDj9f7zhloAeKIkRlpZn65dIS2q+1Htsv27t2mCCYih2Y64d2QmDMvjVBQZkCvEC8zFrclOT3nSFooM2POBSymkbSGNuf+0ra/PP3xe779MT9f9wPX7ybtdsy3cZFv403ZByVZy7wrWiVeaCTphWsVApspgs6ahaG5bPoCJc6QjzHn2Mw3azcOoRYEgmOiCmQWg0kYvWa6QKoWiVWWi8wsDV57TI1Ez0TMCG8AI+2+5zFIbnWESKtLV6G5xsUQsvYQaZHA+Lo6vGSn5EIXax07zRhRSE7h/gtiqsCptX2WC0I/G8QFmtsDHK4qfqoVC+ItHaExQxFBMwkZo06DxUs7y6O0aBWUkseoU3GCpWpdZcgMJxFTe2DwwDzELJ+XCiFf7MRyupTQGATS6HXYZMZ5qC2MrCwIS4xZ67TE0loVvnWqX2sW5QJfTqa6YjW/Fs2yRpBQViTF+k0V8an1K2fjvY66dXgUo4vIaO6Zs2xpZpgnzEUyKq4d0Hk6AKjYR0mZuabN1d7X3T4zvFZHUI9S6UCECvqrQRErjHqBTjyNYBSJxkRdEPcumrcr3AWPeb+0D998+13zzX0+99Wbh7LWcM+ZURE07Cs3YU7QZmG+zUCHecxMJR1zRjKbWuWsBQLQzFqMI2UzcHKCSk8ADHJFeJbGLqNAQBAJ68YMEjPHdtiXXz7/8MOPxzF73wIGulmjCQzCMkIxUDoZiQogESECSsU0EZIikJUGUdfsEIYbncg5iHTlH95f//758v1z9+D9yC/3vE+mdbEJqGVgld2IhFKaaZcWEWvxb2TzNhmkm7WCokyMUoCZG11Ia9vl+SncbSgziA7AzNJUS68x5daBWAO2YI6+bXTVZGD0yOzNZ0wqDWzecqa5p6CZ2tL8FIoCtm3ax/XDP9zHX6fUttaaDeswPw7FPXAcVPaO61O7XBtn0kHL6wZ/vuxvYfR9fx3H3t3cMPejsAIkoNG6I7N3BxjG4tMSkckQl0RQojEiR0wXmrcx6N6/tPa3/LvPT98/z1+f23PJCm7pe0L0CkY3GvwiTYPnHABzptMyTVr2aKxkfMiYVuaw4umMlQs0xtPTdmnd+TYplZHOqBJTwq308gl2niXCikCptjKBGtjce5VLEAlWeLDTj5H7PaXqGFQhYwXOYk3yhhNZUWaBuGcUg6had3CGDgC2yscEcPot5SVdlyT4MpuUskYL4AFQo4k9GlSt7pJOLquDMrNWtEowUxSHae5NWgu2pOqOTwXLSfMCrFTnB4RhZWd80BmFiX/tocvDdTrd1iiwzsNY2FFBWJV6XYfYml8WXrUWYBXzuY6xszTXr69pwthSwTUv4pT8FE5ch0/BUCIWRaLzNCKJXHvHYsHFj1K8psLFH9dwbe7uKEyOVZBXuS/i383OjZgocAaLIlkpvYt+ADLDzUpkFBlSDbLnb9fdNRQK7Q9CCF8J67Xgc9EgaoJlmvk2QbON7QmORs9QY9vM5j4R2RqRop2aUqQ5FGbgnPPYc8agRMFac2+ezTYzxTEmJJMB1pwWUMSo2FmHQpqRaFLCmzWGVE271wEvZS6/S0D3yN6aOYWMTEvYDARfPv746y8/pdL6BviMQCsYtqXy7P1nhhCj1nxlTcslWD8GIhbiVly7oEhzODvmJKWId5v/x++//fNTe8p4PeI27G3kBEi495yZSDnNmowaQYpwxRCEJKo/n+NsFRLFP1BJNXdv26z102guRU7kMiAaLJmRQky4AFpvCNAdtEoYcO/7flOqt16sUVKtcqHrbRxpbJXoS3ZztzgSGft925qO/Xj7GQrMg683Pe8f/q7PD+3zzNf9mDO/eW69W3NuboDd9/123y+tNe/RiVqiHjnnkekQtu2CDBy7G8249UtvHhkd3G8zEc27BOas3BZzmpPwQLqZYkqa4u6Xn/v3f7n88Zv41O+/TM2w7c23vdGerzPprROW43DfYqZfGiB6o9LNMrK8X5nh1VgHkWArcMQJILfUfcQMgzfrvR3zUCo13foCFcy4gshZGazLE0poQfYeEcKMRmNzb4m05iUEikg1O8bcd0S4aS2awkp9gTmtkOCqQ1UIajpYSkTRLAvAAqgATLXpUKVtT3dPTeC0v67mtFrwko/LzVdxehwFXMhP7VI/4Q+BK3atKnXFf1VdxPJYFUlQcnSW9E1FmWrV9rOwnmANzkpezIGxWq6HuwvnsPtbwxcXBmIsTP/kGb5uBjiXJxebXlKdYokf2fyqVWIirORSpxpsnbhY7Wae3oGiW7ScdFwT1IIaT6/Pb/I7cX4nIqqvRm3qypMzzxP1kJFahDPcjKcnmaceTMqMJOGtZS59V2nR8+uAQqMFQl+D5ySoSnysM+/0EJbY/wGzhWRsM2Tswc7MdtnSHCm4z+OImTny/np/jRTgfdN+VHTsxay3PqWIPI485iFlA4hWM4qEjJyJmFHX7rqZmc/Kaqk978HIqEvkRBnIRFrzMkbOGQiM6nOLxHPLEISZkZAJ79jmsf/600+vX341M6WFIs1NXAN1DWQTkDSnpyr9n75ObECR8hWSLITEivsg0wxTy1l7fP/9t3/6/ttvrvPi49af7tKr5iGvaBIpVhNW+lgTU3SXgvTI0HFY6zJKumxkxmpTVsSVF0opeqSU6QCbIw1I8zZGyhHHXo5l08o/mRGt2hjTdu3jnim0fsk4lILTWkdEliIvq5GByCTSkEzvLuh4e90i5tve3n0TY3fw3dP26na94IADebna9QI3ODOliLmcOAaQxzhmjFQGJM3uzRyRWVTYlt7Qnaw7GJGtNTPfOvdDQhSmTPrZEgMGNEp9Bz7r8u/9D99tHzHu10vg6d1xu/j753bn8boXMkPvybTW3LbERJSK0JGhRCLNF8Rd8emGM7EdSbZMjRgzws3akm/Vo5PAVLjMVroWzzwsF04zrVkzs8yYc5i3wqCRLIUATuByxnjb54jr1f3EN6rWn+0HJaQl6AZUwt1XVrYsWgk7AfqKBlrNcHlnq9Cckphq/lYnWEVIjx0JOGkBLha0SnlmmHsoFuROlISm6vFvqMZHYu+Jg6uUDmfrXdGgWnjD+gynZEXLdLxwkhP1QUY9VI/TC8uTVUC96qgpS8CjVV8+LBT6dAqgzMrPto6EEpUW97LmAFgqFvpVGNDyuJ0LCc6oH+Oj0OfC4aSV6oYTW1ON049jLr9W+axuv2yIsfZ74gTragQ+p5iMhc+UmnbOAOFr3cVJGqwToHjjNNJ8bV02t3U+abEqX4EqLlnAiRehteeLogmGbQtvOotSCRj2Mb68vb0dmZHb1hFhZn7ZtudnczOHAO+GYLPegNa25ltzCjhWMQAAh5k45hHllSCUAA1RGSDNzQORmTKvJmeZaspfBq5Ftsk4OfkIudI3299uv/z8t/32xWn0zeiJtmRwddpFIsPW7kdQTElzWmuQSnBJGOlLCCGWfwqKmQdi5Nw35z/93e///g/f9uOn63YZBz7t8+XAkFX2gJtNhclBAaE11oHeJG9+ZgCWCFry1upkUU4TQNcsp27evny63T/3EZhzu1zVWsU8G1Ir3os5BzIi2Xo3h1kjSHeSOTMbQYs52sXxaJSMC2o1kJYzIBmhEcppSWYippmuHy7tenn7NF/wev3m3TS8zFHR2fNA9BgzxiEEK607yRGThPdWr0u7NGQGGQnGRLMZQwoRM2XuYwQq2Uoax2RmxgQ6nYsihpW4PAMv5L/gu+7/4WmbLdtUfyUv7z7Y/qIUZiCS7qjQtzhYXlykKpEmJcjpQFRKYy12WbAKKgXzOiPv+/G8bZd2aTZH0ghvHUB5ROR9BaCUhGHC6DJa65VFWQ1hjAmHWzEFCW+SFMzOfep2xDF0tfMtSxSLUPyylMYz5moRqGuX1lrrXHn5q7/UGpPPiiCtFV2sV0xnnT9DwaqFVCUyZXBVvUeYz5oGcAbuVw+7ApmFGjh4Jh+oNg8XlrdiyM5aiKWoAb4qjbj0kam1V144Y9vquv4PO21sLRUqG6ufa7zqFMNJMBSCvJyVOsWCa5SoU5CVf+5sWocTItPLg5IpPVaul2NZvoj0OjwoIZGrqq6uMUs0xDIV00FIc/0mzrWYK3KuQjtAN/vNVQLgX0OZ1r3B1/QKS6zVItRJGoMR4Ys2odEip3ulq8pLBprnwwAQKzkNOrNSFrq2mIMGbwLpnm6zBCozjGTbYAxxRuaMGPO+j7nv1r0/XeSwaKCB5sjnbYPk3qy3mj6jDtblpRNNY8Y955QMat6IqvLpZt5qZo2ZwgSXoK96fdSB7iRnBhACDeawgEnzuL389POvH3+I494bJZNozSKGtZ4RVeJdtBU7JZWJTkDpjTObm7FrZi6mHxJSM8forTqO/NP3v/sPf/7Tt99+779+nmy3obeJfQjW6C2lmmZWe+yV3gASCrHT26aIGYPm5laEuSUNphk5AdDbtc58FY9BpFlkNrfJWupbjJwho9axJqCcMJeSbO599h1mmsHWEoeEou+c9AfB1AxMIU/CLs0SjsDkuw0dY06lTBTUqPv9mJn3HU/vWj0J4xDI1iC0yJmZdLSnjdJxjK3U9GTGmJqNGDHnHO5+2bby/VT3ioiMqWIApKhnj63gcCVkGmOw4Vd//9/0+z/hZiEMRLqiXZ7fvb7tOac5qYBAVNGQFUlk7jTpcBUIXFxauDjFlaEhd2/ilnmfEUZv5m42Th6tWtTMrOG73EBEy4x0MkjPclaU6hcRkmWTRCXMHN5gM2SH9HaM25Efro6SB9Q9qSRK5Ep7OKnTgiBq0gfBJT8vASTXzwRatcYL5V94Ak7ZwEIAsH4XQrktU3GCLVXpFrBzamce6L30KFWP42TRcYtOXPXGFvj0qM41LCxTE9eUsDp3ncHO1WLjtEfRah0sa+JfqJPO1IsaXysGs77tknLygYuU80DrV2qKedRZ0FOzxpiluyXNXFIqDcu0cJLWeV4GnQ1cXX+vAzJmxbJzUcJYRi0umKhWg8malxUqTylnOXgfBEjp+gEWbZNlYSm2B3ZiTELZS0/AP9fOgzVuqiKmlQuZ0zlvoYrfA8eTnQNaQ7K1S/olbRwRymitR4bQ0FrOOWPeXveLM0fM26F7JU689esWgGiXrTspGIyJWeG0EgQja4HBmhYjMhXNvTcza1m0wBLkZAX4Nm+NRliQqj46kilAU1W9wV5bmngx4uX25Yd/f/v8k9sgOiC5yaz3C8BMZUYtSVqgJ8plng9Q0r0ikiPZFMFytkPI0RrdGGN0sz//03/69h//y/H8YYxOxuf5yy+vP95nyFtmwhcVEznXstaCmeiw0tutFW3GczfDyoWocTu91bI5Rcw5bjH3jLAKEGHSOrAehSLTkiv5lqIiYwx2xCQCzd0cKTncZEdMpOha5gM3WMFraeXgb46ccxzpvr379rAWth0ZKbbt+uvn+fpyRDM2I0DDSMyQm7FanG65J2m+bYrpUGueyoiZc+1Ojamcw31IqIUZ7vWep1JOK+cds/CZiDK/RNJhxlC+wX7Zvvuv2MdVl7gfLru8255gH39Zm7xnMjmOe+kcMmZGEnPZrQIMRNkm0GTpljPmwhQA0GfqmCMV7t6sA4ekyDtsyzPDsjw7D/hY5zyXYyACQFuZdCGcTq7qlFujzZh2H3k/Zohtwdl2ZgtjIUJr/5c1tyqjNK5++0yCACpr7GQITtycOHMdqnLQH20wVp9Lkk6bESi6cKEiFb+/ksxIy3Nh+pLfnK8FwIi1GHIlfdaZUBHtPHMbahf5yhdaysy10CZrwpPBVzId+LU+ExGzpLG15RLQCYOtH72oaa0ml6c2pyqv6exvz1a3uNm6tAtb4oKYspzz51eWmCQSK4G0ONuaqeqAq1Ji9ALrK0EYkq1MHaskzlRWfS/YFYC5ASxS6kEVPDCadR7Ulc/E2m/jEk5YFCoxgjIfeRvSigexOiiW7xprcfwJRWVJm5Ba2iEVBK9sgJRNlBStbdaa6Erlgcu7J7/2IaUUgpu3yxUE6TlyxzGRSmFu7s3a5rZSz4yNZiXNDQqVwBCRObPwLU9z9EqwK4EVdIFVS1CosdZkXJM8Ys7WrDkEzsTQvHCzyJePv/zyw9+Ot08NgiFiyrvRKDf4jMMsgaAV6QGDKStEVvUctOYOS0iDK2ArhzE0wxuZMWN897vf/f4//V/wx//p5/489bsXxU/9Xz7rS9iAucgcs/bYn+t4CtC3TFlbjo5FlWklNitjhShUD+LGtiVbqKLGBoA559a2nOUdq3IPqLJUWM4cI2SsmCXmnkqDW2vKkBQxjYS3vl1ijsKLcwQyGwSwwYwNjrb1pPV+0eW9vBkROobl7ZZH5tNl672BCWgcUaZaZaYiM2BppNMSuXnr3evVj5nLFICREUqRw9buCIYixqyGpV42K8eQUmTkpDInvXmkeePO9t/0hxv68/Hr6/Nol3e6vbB3zRtINksBmW5t7QFPxcwie+AowROxWjXC6KbIM/bRZNsR95mxtda37jFjTkXtu+voG9iqoERMtmrerVx4YSu5TEzSYVJM9kajlLCmSaGl+T7zvmcEmgMSMqp7AEg2rJJuhS34CaGcof0peWScrCVgWgsfzvY3s6CMtQe8rq2bzdpj4R6R86QE1oq0QlWxfGR1Bpl9pR5+C/TzdKKRXBY2nOUIC2SCTiCoYKlcS+1FlK+F6/vkGlzWqcPiZEGmslnPZRUSyoWJh7nM5rkU5dHJLeOTzvKKWkrDZcGs3y0QsEAmQ2oxYhkQ04qMXWhMQRF4cNIFJdUxlAjSKnq62ZYIsTLwuS7NGfGwRqvTCpGK+mJrFi+v3zKDrwexEgtptrb9zTWcQVgGOumUGT1O/TSagKJXlSvnS3oEcbAuSD7sgWYUWuYkFQhuYg7nBe7pbJetXZ/b5ZrW2nVJwWwzd/PWrVlkYCrGmCQ3s1b2pzRvPJUrJ+GckZoJM6u5lxBiylyso8paoS6RY87InDPTDL4Azozs29ZLSBlQTpNRc7y9ff7rDy8//hvH3b06qYrcTLoCwxsleXfEaoyQkiWMTDO6UUoKyFBGUC1yIqeQrRk0RhyZ+f7P/7z94T/fv/2PI3nkh9s+PvLTTZfAqxmUkUQtNlYRf4Uzw2UGmGSZBdmtuBEJ5l7hWRFJiF6toiU4VfF5QkoTaLKKDi+ZoGDGOYAEDVXfQWZmzgEJOsomas0qJaEyrs0dRiBjBrWWw5g1NM4RvF6ent+NgM3oT/Z8Mecct/vbK1q358080xKgm6L1ljlHptHgvDy1etpVgRRGhUbM5rRs0iFZTCXlrnIqV80zh8k1c4+BTJghG2mRIzRrFyvKtAnsBl22/zaf3m0ZOlp/Z6+fkJNca1GV07wUKElYqQmYE75A8BLimRAJlV/HBUBBtx55zIKzt+3ivfk+/IwCNjO/mDUKMWcKvjIUVRYpKllPfa0xoWUip9xphb2t9OO+x7gfGWHWHCfKTbMzspcnegAr0teXWrSwhdR8JEkArHCgErKYmWIlhfFEDUgXsrbK1Cq6qgPrUDlFPzjrlIF5qkxX9deCJmq0KkHV6QNYRCVPlc5C/yuG6QHZrMj10pbCjDU61LH1GMJUG5OyFE2KQufNz1UwZ12vQlvqT5xoyAnuP4gOkmebv077Ir+iMiEymvn6q/XFzhVj9TW0qJD1PWOhAiJVaxm08jEyM1SXKmUOW5F26wusfxZ0VXpboeiEhRqakZEw92KvF/tyTglcjwVIlg1C1MIhiRXeYZ6LIjIi4bUHhKoOGihBsQoDrQE2g8ZGjRyf5Rcw+3ZFpLFFZP9wdeuAC/DmldFBCG5JFM+eJ69/Om0kAZFegl+zTNHd3FLobpnrfI0IEDpySubeK2VOdswZc2ZkFmFZiTUGEm1RdUYXYMh05Xh7e/npr8fLx85avlO+SSkz4pD7moTP8ZIkLJErPbHiMDMjY2gOzYQ6slVKpVWUe+b29P79n/7P9t1/uF2+v0/x+m7efnk9fE7ZesgSEUZf8VtMQGYN8Kr+EJiEqarjej7MaOWITpr51tE32zasTWWlVrbWjWhTi8FAa0aRPpFe1sy2aW2EGHRYglAco8DUduk5IvcZYJLbdVt7gGeYdQgzE7eskV/vvjN9ytt+fRet+eX7b+ZPaMR37y9b886FxXlvpI2RxR5tLqIdMY77IWUIyLIlYfOGOSeLzykPCBQzIzPVmtNg4EwgMuaRrZmjbxemJU2Z3hoAb6aZo/LQ+zUmtubfvnvfPpEZThg8lhuOc4xt674xjyobKa7NUet1AU5P/uPtKnC3Sbgd+/vr86W3drMVOaNUwLdGeuEpvgQbjMRa5misnVNaB3yBH6yzPyKdBjKCI+ztiGM0XRxnaAHOtE4gAV9N5IJycAY6nIsPISiJ6hhYgywL6iEqe2AdgWvjGDOCZma+/KIPPYidDq36c2u3Ih99KOrPnL1sIs281gaext3Vm1bVrpcUKIXqAnBWfU6ltPbWas2w1lZIUZU4CM1bLL9AklhbYk/ZatHDRTVkLsdvVSfgZEpVvCMz6iyJJSk/P/ISxGWignRwUienvqaehgdncTIfyzJQGND5JyyVbhXNe5KW6xlbJ/TjXmJRNTWdZglGHr+Hr6PMOpIXHQRAXNMMqPIwK8qIsAiChWBlrQ7NTLMSAYl8XOHKBi8UqH5eNeWdkb31mAPsjm3M25Q2xbv3H7y1Yx/m5r2lChlLCCMkEs26uXmtrQkkMzUyK6YI1t0teNC8X66KUCgiU5NOmw4yEpnq3srsWrsGIDnNF82iUw0QnJAyDKlsoCPfbm+3Lz/H8dKM1XFUioyciS4VKQUlIlIRVmZ6a77I1sxUzpkayEFk5pE4mLLNOY/IMWN8+OM/X//wT/H8+xe06Xa5PoW9HVMRkZkWA+i0vpqFJWNIScVU1CJZIH0pMXKlFkTAGTVhLyAz++XSRjOmGc2cEs2AqB1K6/EwQtOhBZUuxRiYaK3RUIaBEm8w4SJ8U6QR4whQbm7N++XiUmTMnKlJtxy/5P3e7MmOvH/eX/vtfmfK23aF8n6bMEejIgZzxAQS6E6OiJlzRFl6E2jHPtrqroQsRXPd1koDznpBamdAxKzc+Dlnbw1CZcq6tfN1SBAyHIpm6M/vdIMY3bqplj7Wy28ivJFcVXUBzCm4SYFyTCxf6gM5oEoMQhNsSgF1t0uzN1osFCNo7v1Sm5ZynmVOPIt+kyUjqkDkgpvXsgek2Gg09y2h+5hHUGxQI8s/RWlpIuAVLlczQFhNIRnmXofYQrTL1nr6ckiLGSzVgZCpBSwBeBAVxb2729lMPxiQypM4ixZP9AawmhtgD8PUKcrUV3/pImMhhWB27lk8EYrz7nxFuiESDIVpWQJRrAYwqztEJf80ZKgCo229Xid4wpOhKOzwMSPIHmIHlA/rMRGVA6YKPSXZkvT+Ft2qbuA0SulxpYs6gDL9zB0yWqylwXUNlu70wXOvQq5cvm+ssZ4FFaCVX+fhUFwD1xl9jVUVSuhp66XJhSKcAB3Wxf6ajbFgLBSsVD9VHWn2OEhhtIxonCOTur+1y5WB+/GRW1OkWQd8jmhtMzeCsdiT0uA6bTmrkaFUKEBkRo5EHBLgQ5cGbyByTNFmzHnMiLkwrFokZD7GLFCPdVsNfdta4wIOl86pdvSVLDuteY55e325vf6iOKzbGiUjxYA5IKwPxURERDkr6DXjLo12zpDCZCupau70SSNiFIsItne/+/Pz93/O6/spD7Gbz8Dtvh8zUPldoi3ZT1Vyq6MXImqsq/sYYjvn7UiyKYNuy78zoz9/Y/mLJTYqj8HVfKa55zhqmau1ypRNIcp/PyMrchOAZrTeM4LCeitSiuhbG8cEGTPM1bwBiqjJcAkJvLmOV73c8d17eA+zY99zWuDKZ5s7YNhHUNjv0xrgFuNoDYG8H8fr7dCctTMuMi0AIarAP8pUfTPotLdZiZ7nAgRYYfqIoPe6mdYcLGmB6uHIUmb0BrPeO0xLpyfUAkI7o/eqTBVnqAhlWgpZFKKlZ0nOEkqhmzMV1kM55njnW2flFWVmCLGgFSzAyc0jZO4Czb3WNUeFsfK0GhEZk95qfiOM1jOPY+B+ZKqZuWLWK8ulQS6gW6u1rvNFpTSKleRZjeB612sdla2dlLTqLpcLdBW3heesAbVcY+dYvFAQiaV5z6JkeUIqabYwetlJCRSYX2sN8UBeamYte0px/yVGOlc2VkEN1X2vnvpEZC1qm0IpZxZDISPn6hjI5XBeWNP6giessn4Y1dF4tvAGS61MhbPEP/7wV23rQny+poSe4FQxbsWqnasXWKUGkMqSeTbu67ZVTa8jQ4/ufMFo5WVDQWp1+YyVHFXaXCzVT70t5dsSsGKXy3qmWiW9pER6iPoLijmdKeXYW7fDzgH4vBBlNnNvlgNh0j5NmIdvT1QLBMwqgDwiVg5JOUjNjQY4zxlxKihE3c4IjZnHkRJ7wGAy0I55F6jIjKBYK7ByToDWGu2UHFiDWW+1q2AGTIBD3eFmKQQRIzfyapzHcfv88/HyqzGsbQI1A9V8w4zgqr0J6bF/IWJulzN8JIARJJBr6Z9MBFtz5AiNOXdctuv3/9C++RbdVQFjqeO43+4vxzyITPmavUGBGSav07+ajTKMQmBF94SSanQaLcZo2yVyTyoin3vDGBp7m0cjW/PII48BAGYZA+arM0hW7paojDBjmtVW1XlEbTaFrw2yMppZa02GnLNyLjFzxG3EbN3MDWS/PBFzuJvZkZlSRl7ePX959X3OmGk5W1oVkm1rOXP25t0yEt78kpGzWjIzoZmmYErzAKaS5cOs7AQU0R8FYkfWEjGWPphARtT7kHPUe1U6k5m6did9HrrHYa117/O4rblWWvnMlV8HwcmoLMbaDlxlxRNRZ17mPIGF6oD7jH2M0fr1adt6GzMmFMwR463SDMcRZka29dZVS+RNWshoIR62dTOXMWbQK8USc8Z03o7Y97ZEXif2/Wgki/Ra2LzqKIBBbq3KjRnrNV5bORYYrXbmBWEp3+teoZTyNVNmBmrz3RLWPJSIJ3WbgGGxoSvwchWhKAVHahX6yplYI9Y5l6y06sUs8KzSWHETtdt2zRCPfTIA3TwUitX1nXU46t2qn/HcX4HHHwBUtug1WJxlM2WaYcuspCw0JHPZ66o7WrqmNZqsHj+CXrgQ+BDZ1mBP2NL5CIswz2W+FR/+nq+n0cL0ziGIPHNY61dXjY45z08FaFVVnekcVUHrRz5PFZrZnCEJdn700ojl4i64VrAJCpyaXdRmjPrDKUBG2IxQ7ol7xL8hhyUjDpJP795bv47Iccyxzxxx4oNa7UClZR5Teqh2SCIypyoEFEoqNfc595j7iH1qrC4sDo19jtse+xHHoGBAxrTVvGCmZsyZE1RvbA1bb93phovTxnz99dOnn36I4w20SnuBOdtKVa1NJVRIMqHVhihWIOXat7FwBUEZUoIhBBsyowhtxGzvvnv+/d/b8zex9WwmU+S8H29vx5fKq4sSOoK55ngIBvcTBijBnLw5iYy02pVpjTSznuMQOWa2y2XOeYxp5KbS4Lt5yzV3sQiDWhho3pxWBruKVGlmVg1y8T4lD03LlHsHac1XJnsmSb9sJSAZc0ZKqTlmJPz5+ZAmlR3f/P7Dhw/vMyKHjfsct5HHZGTra4BymSVyoL5T7SvyTie8NWucQhrSLWkJCzFAbI29pXvSBAfYe3Nza71+upSyVhCwNlYqMmLOugypTJM2n4a+9evzE7MuulBemEeosrFExcsPWlJgrKjfc8pfdy1PtlewYwYgd6urzEhgzvvrPO5KtO71F5S1KVGlHmGugB4SrbXm7u7KdDeCcxyV95JsY2of5bdZoZIEVtZYFVstZeTCpkDBkHI0p5ftu6D3NZYAtMrUq4iYwoJOkfg5CmFh7zrNBlwtwqO7XF+s/v182c/SppUBUbRcfY2HqQvnIhlb36Tqy4lVnPg886R/6QYyMnLNFUnQF1/uNWtVbTFapVDw/ILrXtWHBAo9K5r3nJ6KJo0iIiKjikI1NqvpJ87b9bDX0ps7q816XDHoXH/ChfCsku8lJ6tOjBQ0ywx0tv04of9iBeo712xUhuSq67W4QuU1KXzrkf13wlPnUAJzq7RH1RFWkttHa7/4lmUE0XkIV7nGohJhvuiv5s3nnMCmeOdti3GLudm7Zu2yjzyOo7WrIZFlmYesgRU8q4VaAU62Re1ABrYGwCpqB24PvXDKCuQHzCxGGOkqeCQys/XWm7mluQVpqFAAq78tm6qFpUqNef/1p/uvP1ZnlIJgoGtdvVZNR4553lzV5E6SyIWMxEGlNFIDmsKsqBgz5MzMCffrd3++fv+n9vScgVBezZh57Ps+9rTafm6CR6Z77VGh1QPkjaXRyVJtgCfiTdLszJE3zymzbts1QzRenM8Mq6HX2HqTjGQiQZ8RvTVlwiwzUrBmMGu9pwkZtJXulTmNLadgkjvdhQnIGr15o1F9VCQNwW7IyES/PAXzOALBzelzPF+bIrSHOy9b61eLSIO5NQHzUOwTyjxyjHy6NtKmcmsm2chwb947rsI4ClkwI1D2xFSULNAHZj3fmUEjGz3ZWqMUiUZoJDVFoDcRbF2p69PT5endSoMLNSMiy3KhokIEmKE2K2mlHidStrCCEgf6WW7NmuacyohorTVv7jkUQr3dgz69dXYvVVnWsbb6r8gRGeFGd1PFVoLzmP3UVDZvwJyB+9AIrJ4KUgaMUiyrF0tdflqZVCFOKKl9uQdWdsxZZfSgckWvNTIkychkvRxVrbUexxUlqbOhTuTpLFwt90JpVtUzkrRc+ZkGCacwXyFzW3j0yZzixBy4dAn5OO0WdFDv5bn5pGo0rdrTJJe7bXneqhUoHAWVplZ3DY8fXJar9C7dp4wtM4pN4mmBLiLwK+FKGSgyiuowrs19a/QqLunRta/hdZ06awA7nXQoh660VrFjfZPTPrDa/AVOAksGKy8Rh9YZfNbtkszA8JBj4TGtwOBpKAVtnn4FnrFRtQhh4T6FRJQN0tYlq2PY2HIGWXszWsawfokxW/+d98vInDPngezdSViiUczWG2RUVkAzWwHo6yAvB7+Zkw2VgUWUC8xXXvesyWph1OWVkrYagiPSIKXcEkiFxGOmLOk254RA2v2+f/n46/H2YgTpx0jbnLIC40UroByS5iTL+18ve8mBM2cgpuYxY2YMImqcpaiYyMwMeHv63R+efvc7XC7T4L4xM4/Yb8dxDFTUD23lbq02vQJYHbDMUstLCTQaqJIL5jKNxDF6v1jP1p7IdtzfnNmULcIyF00NJgA2WS38dJrRha1HlBSRBofQumdi3iegDLlTSPOegmJhRgae8gAQ6tsWY3ozgMftfu09see1t6etNdd9Xv36/bvt17cdQ+xqZs0bcmxmqX7M+PLrLsS2uVLNS8EZ3qy55eRt3q527fTet8hprNajPKgxEwzbylvlhEhHRtZYVm+PNVMyhuAU1obCI2aMMfb7u+e+XS7uTjM5aK5ACjlmazTazCBcy2NWYvBFCS4gw9wy6uShN+QQbGQc87hs29bN45G2NRB7Rqc32kOzI9JjJDxyBgk3SprHoBEzRHd6xlqz7m1THJG8j9yHgWW7SRjOqb9e5spaO+UnVp/cJ6YE0qvIaZm8SgGwZvA6jUj7bciBSjDj1KxS79U/LjpRXFwCmalKk1Tm2btXW7RiMnkWeK5ZBYJ4qvgX63JysKjMUUKC+3nRiFC6ztbMZHzg8Y9M/yVYXEDJWa4LYqmffaHtACBzX3UZIOFkpVCtOqy62yYrqmM1/IoUQVtIS2ZU1FpmFHfEc4RZmUYKO7fqYP3I68uzeojS4NOXIvY0pi3ftJRrR2YV4PUM5sq6rjTvLLcwcAZOn2dGPRgZ8f+j6t+2JEtyLEFsb0DkqJm5Z0ZWdU832YuXmUW+8v//hg9c0+TMKrKyMjLczfSIAJsPgKhHZ2VFRfnFVPXoORBg32BmoCU2G+zu+SAFRtiYRNZRXdEdkOi1lanS/UiQTkUOCe4X/SIh7J0GH2//8l8C8/nPT46p9dNiyoYsERSINYjhXm92WRJudTRHJVzDUjKTrPxfmXQUDgOBlXVktZBLETAMd1DKvTNXLDNgeELmIxiZufa2Meh+zUes9fn58/Mf/189f5CAjTkswQOJwAgYc23TcogpKESKqURla0NbzM7eKg4rCkGWlBE3Mvjx8f1f/8vbX/5qY7gbEiat9fz64494Psun2VHB2UAhzI7UNk9XWUZtqDx8ylGxDlmwZkK0OZ5fXySV9xy8KsJgmnPGc1VS7JjX3mE2LJlJmzOfTyPH9IzIHFXdAWWUCAzuJJL9Wk3ilxNiqzu067poyB0ZCgvuWyM+rvG45vMZesdez+cfn7H19u3aO30lxR251157770zk5XKS8XeMFdzIv7GtwkV0z6u6cQcM7UxDFFPGmQpx+DwYYAi7zqod8QYQ1EQTVk2c6B2+CFiv5td19s1Lx8jojDeTBqQw92d0KiFB92blAfVGvuwsBLXSa/A9Fo+5bGfK/OdfnWMLxFpU9q3/J72jcaEau0uqIpIr0mb3fgpd9IGB07vTZilwzD25ueN52KmT1i81Nk9oMiLFq5zEL0OUC0etxbHFQXc/fipt2inT4ZYi78h0iIavm/jMUo2+mqgC9pukJ7NXVlj5I1pdLGzDpPQwRsKGy+kowOTs6NGK8GilY6ZJ9ONtF5xwypSWQvtz77zJgwKfSpAzw7IfaDwP2VAUwi+ons6kFE9SkBWXFg1aULoV7Bdy1gqMLOY56aiy2ncDqw4ACIkOujeX3SPaWcQYOmI8oBgL+Smjq9uw//0Ij0o4FiOq5fvukKAvclBesFIQMVvZB8SbOC3NiUgpR7uJKBJGqGmHAmKaFqFdBLDaHZUxPda9pjvv/02vv22jHvlZBq5Pr/2tFRYmV19zvmeWVRPrcyzMS69zihlRCJg7tX4ZMRw1nSVsJLkUMdzspGoPb1o7y9lw30MzPF8fpXmbm8ZFVgD+eMf//j5+98RxVl61kpypJI2p08PgZRBhqQUvY+5RquZGaWESwiK0F2MiyTsrVipAPR4+/7x17+Nx7vILXgmYu+vz8/P32N/UYKXk6c2HopMckL0OYvPgWA+svnBeoQs9i/izaw2EG/IrseFvd1x+bAtqXYMphnVaXufNKZyI3E/fY64d4bcvC89zXxghBmN3swkYUY3JzJKFwAi042DI8vQvdNoNmbc4Y6MOzP98fj5tT+/7r3jukYltPz4cSuTaYgbjthZDa6SERL0eDdJe6e0HyOdBh/K8lcLLK2cabgi3BqtHG4KAQgxd/gcXtl156Ay0umF4mVst1Hitm8fH3OO/fXDjHtvzlEIqI+Z2A5LpbFpx2rF7PSeTJRep65SRNDNbCq5dhB8G3Oaz+kRz4wEjRnMLhwclrv5L/cLQ1ptf6pXo7H82GZlnA7l2JEDvoSvhUxvRUds88kapK2ifRGS+kSxrNg+CZF4CTz70S3MiJnRZad56G4tTznUL3zmrCYHmGUmayFRVVDDq9FuzKWLGQ+GoTpD0KzMOX1qBkBvODgYu7rk9aICa/C9O1vSSwYqlA34aPOLoVa6udq2Jys8BVbBcRXlk8ksrV1xDAWINQ6Eg1b9QtTrtetUUCoPQ15IRp1iJDGKsdBrlSYAZZq9DmZTCwvqjLJENlCG9LM97TUb1fmLLubEi0I4S5dfx3hd3OrWhDPm4PVZCs+hQBUQDPDsPhIKODiTbhn0aoCp3AsIYCzVLl83yPNj+x+w+fj+L3/7n//v99t/3nw4fT/vMUbpNd0mCQraCK0EQU+CCGx8xXYfdCc8oYiQkhHt45K2ZPRR+fW5V2wKLiNNWfxZm8syBYOFRUzeNwmFz4v0cecahGL9/Pd/Wz//4UyT13a/yhih0d3HmMzaJeq4o5DmSgc1d8UtKWMxQrGVywyxs3bkgQkFTZI/vv/rt7/+yxgTSaMYwHM///jj88e/Z3xeDiYjxUmaZWYYkWs+PmRuKTKLn7JR69olJOue4QHoUjavHTGnA5H7+fjGh1ulRez7ziUMplgnXERmyK/H3rebcbrB5uOBojGfO9aiWWqbj9IXFrVQ+I8Myth3IqHhKQ1zUvt+Xo8HzTH9er8y9kZ+PX/+sa8fPxfdP75/fHwzmt0/vpChfU/aygBxPR7M3BYM+eAYVOH7CBrHHETeXwEgAkKMaSb6pIDYQcLg7tgZsXcqKaoytmCgcmc1bTTfOxPInRDC/J9f9/vH98fj7fnH5/BWO9gwRPmrayeimXNnMmuroVppI+JVTWyr9eNASMln5Fa4cZq5GS2ErUzur3j+MLwDBs46PvqZhuCGUnz6dEt6+Yw13BNZnCZoxNgRz4UIh3esWB0ZKOMLflUc46hiGBknyl9mTKU36VWo+QGti+fqfrPhjl9MZHl3S45V2EXlr2THjJCVQmP4H//TPWnLXBri77JUkLpZp6glzVzIQrGsChPqncF6xXHjxVCaOcVab1uNf+IsMwF46A0UUdGpn/05S3hSop06L2qqHuYVYXNsUDjjQYuG2ipQpILarnIOpqr4fbFqi835c4XNIRvY7SaSbKtsy236nOxv4pTwPsLVl/ll932RGK8Rqg9HNFClX26xAyn3udbC02IXOMzzdJZnZkS9Tj35pQeoRT0F/40MrET4Vwh+vb//9T9H+Oc/f+BhiByaa4VPjIfTZ2rlXoydPgDjGCypr8nSNNNyoNrBlDJpLWuKTOGL/i0jRIvMiKVE1jRq1IKQ9oIoSLjlDraGA5kuLib98vXz95///v/L549phFs0rSAbV0UuGQRnmuUWjNpZrg9laaETmVIgb+RCbjFLNJkrGIssJ769/fVf3r7/xcbcVdJu7Of984/fnz//SQXN6+6svWVmpr4VoKwtbwApVNK/VOZFyXyX6bT5uDF5gy7tfTk/nNwBUbUPBNqxV8AzMjZh8zEyt01Xpg0bY6YdfUMkUiRipWHzZSNJIFJz1BoSCkLmHeaT14i1zqKFIeR4e2ByJ1fu8Xibc4I2fcxpqaBwrzVla39h2vvH9fHwlYbE2lDChJ1Q7qy8h1pbarafAYXZJFAruw2QwXxA2pkBxoGAq9mPimsVpBw+SuWWEYgAfSH/eMb3j7c5Hz5sl+HquHLVDVdL8CpNE2ZVYEAYrFZpFqVJlhBIZoywSO3YkzbM6hnKyPoi9/PTfJpf2Mv9EnbTtQfEpRPYNnr6Bi0jOK3SXdxMC2vj8451Ox4OOpSZeSRsSlUHoq429Mw9fBJIyRQ1g7xqccEnVTo6Xz5EwL1sVmchSaPnCR25Oo43tlQTAFjHqKPJh9LWl1SKLx/YC+Uvb2iB+Khh4hiDI5PmODXtzA3lCG6mWVC1v9mZGr73agikKzSqY/o1wSRqg3oBKn/Sv7z+FQBrjCg/aZ4tX40h5f+g8jGyzGhqtqFJkTp0ZMcd9yt19YxEzXn0+dKXhOqaW37peruoYUNAc7Z9+lalqdJejr46qVvZW6fAi4UWKwbVxJMM0N469NHOBMGENfBmqv+lEWk8ZwyrxkWOWKnBzOeY89tvf3sb5hDjkwGGUw+5I1MrlKTX6qNUfrlfhfUDqlAZSNLSZv5KGwJKhyCRjx236DJINZKUXAVKRcVpVC5oTXg26lmlrLIYaQ6Qa339/h/3f/wbc3G6yNIIeZ+RytiRTppfExFKGR9SWH/tCWVkcAd7B2Q9uU4OInIv4ZkZvB6P3/5lfPuWbqtOf+Tz+fzx45/P5x9mBjCUHKM7N5yupPBv80RUUemxtG9T0tzFChRHpjLMp5shYMgrMRQZiVg1nO6MiIxYRtIREJHD517bpsPg0z9/PIe5DctwMErmDP16/uvhNmBHUqVNBIi1n8pMhPbeG+TAXjue4P72l8eHv//3//ffYfz47eO5mbGR+fz64eMbBEU6zc3E1Jt/fZKSgQP4iszUdHs8roiQcN/PaY7Bdcdeu+QOw3vnYjF/SPW/HSt/GkJyH9M9lNyJXQu9fO3lUoDjuloBWbBe5rGtgjDUg32K3y86sTulCksiC9HPnaLZ3PH1XPt6ux7X5FruzC0apZ36pN6FQZhyu1mSUJQBeJhlKlsjb5IUCavFn2HuBMxHKL+e+xkjj0wugUSv0Xw17EZkqHxnqCfQIJQJyBpyObNSVXQzixSOIfiFszdKgZNvow4gQ48URrLc8tkg/i8URi8UutFp9k+Q0MpINWnfKIEi5MNbX3RkM3p17ifAgPVGUGn1VSiPU7e/qt562NkF7BUt7A+mVx/dm1B4/FcQ4SlVj3likWuCKHH1ac7ZA04fTPWkCoeE79SdStlpB3iBZfrTuyAqQS+iMTcdqYFeK88EpXwMKQqiQQiVHwUFW1B+hp7q6E8bX7wBy2hiZMtu6kOr1POZJ2gababG0b2V9qWqLXQgawxAgL9d9tv3+T/9Fo9vX2v/R2Li/kGMnR5bGqbgJpcgu2gzypGnQMqGM2ftr48l4aSMuRm1S4lvzNrh5ZQoprslw2k0ZGZmFBdDt4DgxSgGG7KmhSn227iU6/nPv8fXP9xUvtA2pZSsHbX1fA2fbiPN5TU3W8RGUYUROiW7YUwR5oSlopX7ks/H49v3eV2l0IBSudfz6+uPP3Q/vYfEBviIk1dFNx+k1/gCQ2RYOig7AxyV5gYqZBwjIDhiL+be+VTQtC23OTE8A4Bh35lRArWSO5ftK9Y9fJC83h6KNBm4UIl4JDq9q9DTmgfkwxAgKpavdqcbco/L7s+f33/7T/H1TE9z+9t/fv9//vc/Evn28fF1b/5MjkyT+6MWXLZRKweB3OFuGbHuJUgyIP606iFixZxmsIgVe7kPZI7HpYwt7p1NXrEbm9CmXAEjrjEKzVJuZKTSxzQb+/kFf/g1QNmwWilEZybKM0FnRlBWevgKa81qNZrhLOgRItRhxenmSltKkO5OWwT6YYRBgdyvpehGulhTZSaSMjOZZcB9EFH9de6AmY+SnczI9dy5diX/9Fymk65AnQ0AwHme7U+gQQMvOAIQ9oqCYuBUT1xD4KymuzAve+mXeFYWslFjwACz1koo1XkyVZOMr1d1vjD/ao0bNoEEdFKpnx9Kj1h2BD3ov9kACO1PgdJ4Mafst1R+er6mF7Q1wV44zRlrGmEXJB8jd7xoiTzkiZmxzrtaHnNweRyZkzqJoB201jawZoh/9d+/2m0vQO/AOsiyw1qfJRm/SHhAitrEy1amlrjTzuDJDuWuD1+v2KPba8Apka4OHFczTYT7wNmUWTnHfEFY/T31Id4ng/TK+h7/l/9E8/m3K//T29d7/K/++b9tTOYj4aCBvsG9POlBD39feNt6hF8aH9ve0sbeU7llQ0bDEA2JJDjroSLNifBh1YKZe5YCZlxbMY0yL7aidL4QklLuRNN0JXq+xsy9lF/7x+96ftqpFuauvfv/M5s+kJHcIGRK4yjn0oLyVrvPX4d8X33UnBghBSqP6OPb49tfxhwmZWSG8Pl1//OP9eN3xpfRDLYrGsagLAW3kRXzOVWraKKhALdBZZZ5pLQjmWTtR7OM3MwBpfb74+PyXgZrY2ZtqVE6SDBRQTSk0oZZBRSHjG7Tm6isG35UDKhXgaJbCtoxrwHQjLlFtwpH9DEIH3NMfyhhsviJP/7xfPv+ePsLt/Jt8u3blbqdOfkW967HyY2xdzAlRFRSR92LYFpurYy1I3dmKEfWPjx302lOi76e5qmQuUyhVZiyuZcFHw2xOSsCz1wE6aDtxJgP2ogIFD1X4ocUjwGS5HDfWVHv/TQdHFsvMBg4pKVZJu/YeVq/4jPQbKShtv3BVGEjRrjRyFUrnBoiqBQdqt6qzJzCmJflVlgI92bI3QbQYmWg1HzJeiERRfVVfa++kYceP/9Hp6vnEVMefpiNkTRg3h7iPAKbwzywa8Tpcw/00n3EgVYqlajLya/TCWfM7QpeAEU151kykwNeZ3mc2knQe4/lbhVYyxcLwuoHDMizYbhNJIUYobPn2JPQweA6LL8y4Mwr7/eEdzZ8o19HjaWiYZ76IX2dq1lm9gmZrGhmHY1RV+E6gar0NMLOQ5tLZzVl7Ttq/+2B4A+xUHRLAYy1l0ZoPRSEQ26rOzZJhwEv12Ema8ciUQz2GXBr7/wrtaLhwsKodSA1jv/Hb3887CcXr5Q+n74cw01T8Ei1U2FcgmdK5mFvgbftb4GP279te1/+tj8f2+YeM/whWpqnDz0N5mYjAJsjNURmhrhJC6nEO7F3QnYojqxdV0B51lmZyWa5997rY8z4+fz6/d9jfV0+SokPJcwE0t3mKBwsYqH8IpUwcfQQit1aGCQyoTxHqUtRa0lEA2x+++3x/fucb0vQFkPxfH7+44+vP37PdZuhcFop298tlfV0jBEpOhW1eC99HAYmKQPsgLspM9DHem67Rkpm5h7mIasFaZGh3Ct3uB+weIWNUeKNTiGsYJMoZF0E5J59VVo2J/VzD9E46K7YNV+ac4wHVsyPj7WfmJe/j5zUnPrMr5+f11+/ffvt3S2fn9v3esxr056fP0BbSpN25tfKAIhcCqL0194KxZSd/ESd9yCl0bNBhCSZcjOkbZYYA6MAfCNDqQyauZty0G0rUhruNJvXLAZmuq9dz0+VJk8tGsbw0p/dRTKfmR+NfXdNq4HQ3ctAHxn3DnLQoHJfqeVEuVMws7FTsbdNN8lpaVEyzAyV8Add7MynGzDGrIUUpK+9v+7cOdzH6eit31gKlcpZEESWM6LzJI7aDq9uVDruoSpEdmpHV/IqUczMEzncR4Wx/K3FaoooP1SX2CoorXFv99ap8fpzxA9wCmMfrUeqf/aQHHVsI06FT50+FqyZp9UtOMCFHTgYrWxtlN2o186uKmQw9NrErTynDmg9M7E669O9n2lFoNU+AJ7InbosyX5LL3qfUAk9ojg2Ni7EM5B10y10mFIRxl17K0MGTYu3GKk6zwo5LtqmPSoEXur+9jtUoLiKPzg5H80Yl7ug0s6PDmrY6PsBv75rNkJ3sE+jMsd/ik/sCE2AyJ2x9YVa2WCpduVZf9f1BsQBn4Er7LHw2LySb+GP7e/LrvC3sLF9Jmf6SHtPn+muOaIaKBtJCzjHiPINdU/gMo5hNnwL2JnSGJZEZgC1hnyvP/7+/OM/jIWqe1FZIdGN7mUiZ27FAomQuUUFD1VDV5xWbFXGC5E7OS7V+rBYw7JoJn//y/X4NmzkTibyXvHj5/r6Ec8fv+Zhp1cVSeUO74BfAR0RIdBq/XfDVGeQz0q7kTsT8HnZHJYwjsfjfT7e9IONM1I9JOqF3sp6Jq+qvr6eCfg1rjGcb3M/b4W2olKKxjWs90MYAG2lG9ZG5roXIXuMZNAl1nqHt5jDP9722vmpj+vt/fuHAwhcPvfe977pV7rjCMtV2T6IgoB9TqujJQFhmC1yzDHmHHMydwYjtbVtcO+YYyRiZ3qGwN576QMkzJQRUV7cBL12Nhksssan65rvbpfizuq5zBIVS60iY8p9JjORFY3C4uORZwJhwwFlyBIjsBB7x/W4Rv/tygtiT09r5UDJHRhbNlEND0n2HtFqOqcNISOWj6tMeBlS6nnv55o7xvsYaS6V+bxgvtPsw3TSdQ7yQGSiNl9WQg37pDmVtJPnStniZuigdlbx7SFBSXpUOeso0D4Ce/1ktD6oZCfFWNEcmYDpZJyoR2j2WwBA1KYU76Tm3q7lbsjKJvPD+KQb/1yP8QJ0q6m1X/H6545/Mb3Fi1SLmkZ3sDYsVR09zoZCWQ6G08sme2wxMhKSzOsyVtF3sjTcFf1vxRhkVnQvUnB2ooyOhpgnYKZ6LvMSJVpEsN1eeWzffaa0Cbqa3T/FvYGdsxG9W/BMYDUG2UvZgLrB6kY5xDrcx5/m2tojgnp6VBuYWcyOERiMWxiBzFVge40qkRIhMxCp1RESw1JZy9BswBaGY8guaSRHcG67giPsks3NkT7T38Me22aOkZywGeMKTY0raOlTpIbJPOByM166AbhnBmE7q39H4G2MseLHH/+h+wckjhGRJ6qVhJXRmYVA7RTCjBGJg+G84oHbfaANik4bI6nMZVTuJQY43q6POR8mUyjWjs/n/vy8v/7Y9w8pAEOZYrI4S7hNwhElciCTcjPLM8qJZqrNbVApOr18w5Fmw8kde4zxn/762/XmNiyHId2qr7OqLh41XnY9EucVGawtOSRsAIRZlNHBYUWzmPf4XiSFXdKd0qCndpWD65r2NpMJ8/i6VzxDGV/r+9u3x8f15hbPYEqyz7XMtVkRIL4yoyQNJGLP+WGG2KXBx4ApjdLb4zEfA0SEImPvvIbvtUkTuTN3FPbaWdhZ5MW6zd2HK3ccNjdDNt05tFOw6Y855/r6Gj5lCiBiqdyQBlOPvTqroQoC7brDM8z/qQVkOy2xM0dsB9y4i0YDoFSFXmYogpTSEVZreWrBJ2E0y51tvkcZPBNkVDojfSe+7rhv4jEJT+2M7KHNKENZAmjeubkFH3McVBwvKQ4KhxS6fOM01on8FZDXWqYuL4frjLJIdoxo1DZRc08GXtBYVYskTF5kSAEiaGFLi+iiKhTMqMg6GHRU6mx9zUuTA693UqkWVfrqPOvxoOGMVIfk8FcpRHmXQdbmGPMCuOIXA1rK1OId1HzpLza7OvEihJDKbtFel7QO0OLbDwjP/mctZWmztpUetH4zzxxp1umudZacnu0VXdrYfkE9kckiV4q/IY5Oqkh46NdEZXXlbZR0raMH0FbqOuWRUdxP4T7nKDjwZ9UBIMfwsd3WziCl6oVBWO1KRdn1Ko9ClOJmr+zJvc0s8hZc+EnRfQwOFwWTOWByT46Ah13JK8xgU/T0a/t72Axcut7TZ5hvevqb5kPPgTFdM6BNg0+Jlp4pmwP3M/74PdfthQ2Yww1wO+1tD91KZJqyljKGQllOLJzRNmq3iVKgE2YK01ZuUYrg2/sYHiuen19fqedz3//8sX/8/vnjP/b+ad4pp1mO8I7jCGKutWWbxiIwDG5mkcvnoFmRkZA8maFEVmQnh0NScPrlm3imD8e0/YV0aBAyuNO8VkeX9DapRBgUEcMf47rMuFddi54Z69Q3MxHaASumdSMDSjiwsrYj7bUfeJ8jhef8+eP++x//uH98Ph1v9vPvX/E23qdBWHuj3AQr5cIY2vtO3bGnYdgFYe/ca9eJt1bmukENs3n5WntH7L2dTnrGvubU3pF5r2WTZuaizwFUh04fZgYFyIidiaC7FFKa8fO+Jynazpj1laKEmDD6jhW9RDXb9c/kcO1oXFg9JfdY3lEqhDGBFfGQ2BNDtFSmSM5BQsye7iEIMWwAdiLOXs+8MoKDIhMlw1fKVuC59x0hGEgKNlwFHjZyXzYiVFxooboFRdkLln/VyWgov+fRSr4gIkKGgtoLtegwGUpCkUY9thLTxgtPqg5DZ58JKbPKrY+DIvR5yAagC20S85Shkh+g1xcnalNeo9ISzAtYL30vqldtMZfQCs6OtKsm/AwyBw1ruMkQysjs6L2DtqulsX2ONPTSUeFWz6JZ+cgKOPIzCQrs7e1Fg2Tjq5SylzW+KER20ecBBiRFHushahdFZbhSv95Z0xEJ9n0lgd5C0HPC4+in2q9xnG4Z+Rq6Ot2mBkbz1mO97NgHbDuviIMFQZnjcymFgDxDxNoyL4lMGn/ZoRGoaLPiM9BKMYFqVVb5ksyMFhsEWFsEYAmXTdrc1YS4J+cyh71tjvC38LdlI+xd48L1keNt2iOAPR5p1+aIoPvl8TP/+D0/f7e8/fpoj7cNcGRT48iQIrC3JxBhgJd+B9lytRRyF11USXQl54+1M4IVLMm0a7rPW/r95+dnxOd95/25vn7s5z+VT2bvk0Lul89F9cNLFBC7HIy0XldSgWWOkRkN3pR+AKwv1WWPx9sIE4SFwq4znx2VnEwoLXGWQtwr4GAER7mhjcZMXW+P+4ZNZQbF3LEJLlmdzpnmRK7Snw33Xf4ICeZxP7XW9XgH8PXza82gz7hsLfDWBHZgL9igTZ8cX1/PiJ1Rdtis8SICsTN2zAuq7Z6xYt9v397cbFUTKiD2zqRBHpEZsZ1qgXfhm1YkaprR2TMhAJtuBp7Td2UcKV4qk45Rba8bZaVoue/nmFfs/YuhrYbNfpHCB4B+PZweyhQjkrUIsMoZjWKnFtNOnJzBRia9fMi03AvYscV5RSwbJkS90BjXzhvmyLF2Pu/aTMXKKPWykpUxqhy/PEAwa+p/iVJaKll6Pv1q1gG1qa0o1So0JGv3eBlGYKwkqdoSTreMjEIIWCIDhLKbY0CowswXIJOKF4tffSZLa/ZiUY/4xBue6p67Sz2pigTv+AqeJbKtUMyiMYvUZg039c6EmhpOtLNUynM05dCnZ8v3oPMt1WOvRB09QOIAhb3LtmVK515yHF+FTtipVBw3e0lLxyzplP4q3A3YNPuK/rrwYphBRdYE1Pu5AIjKaNz3pJxW7WbbjXFojz712S7Ebv1fpHHNPZ0ZUl9M1LH3yow6MtA7FftlkBLoe/e5bSgva21Mr1sU9bOgohZgVuAQUQlK9Qlr9dxCtxtp0JPmDzMvxxL9YZ5gBMOvsJnjkbjSRvoD45HjI93XeNsYuj62JjRsjB9//F2f/yBhY5bpk5yVGOjDjB6xEBt7ITe2k2bKAQaUkVD0ZAqoYzaQIvZGhvanYdMI2bzebdpezz9+/OOOvO8bz3v//Gd8/gNrFUygBGiCIoPmVqXNK7qLXtlXjUqfdi3TYblDoyL7p/mIzIxwc+Xe62vf9vP3++3r02PbcMNcnz+t+v24D3xIVKK7USFYMQ2UY0cA2vumGdxqIVIBrqU0G2aEiVl4BbSs1gvAiDQbK2y/XRxvoM/36W/2tZ7ALGBrXiOUdfe/vz1KljOINzwsws0i9o5ws2vMQKZp3ytz2Xiflz8X6ghURCpougn4MMLNpUj4nANQ7Hj1VqXqhLHzo1DUipjKir3yIRoUY1zPfffIe2qTUnvdFQ4MoSHsYpqr7lNgVgRnCU1AGGch/EYnDFbLmyClmV2PtwitfNrreQpubbrRzHxo32aI/RzXQOGCEoBIkZ6bsrHivhcybdpI+LFb1TL3wviZO7wkt0jCG72ONPNqAvEymhTT3o15772pNdQSMsuf3u+DwjAHZKObRHdrNwYEeB1KRUKh09Cyq3NjMy9BolCjCUjKQJplWQr0Au1bbVHNbb540IQM3uRnu2278fzVvHYw6S/euRdInBkRhwg55wF+/Udmpg5z+/VWoeiRBb8QfKHvNKnQofz1L70+hhJ676xO/1YfLwWT25CyQvjJUg7gF+6Pc0NCICOzjmQkRHjJBvrYC72ml2rk+4rXecC+PXAuVlbqH86oUTVO58SsPrOfIZ03T9q4QxWj1TIuJERT31p1bNvRSSsN1nOVq5G4+j5fFJU5LXNnmkBntZ2AKTJVotdA7YKtoWphjivliZky+IUx5TPAHJd8xvqQX7T3e9nPz0+tH934Y2YcaVzVQjdkEvso0lKxSw1n2e1/pUmz5b91RQxS7lVUmCAa/Rp0i7jjx+8rpR35fN4//7GfPzKCbkFTJMq9Tqn0BIUIoQ/putXNrBYPS/JO7R/s3d0OMJVr6e0v7yY8v56xHj6mP2z/Y1UgRGEffs3L5440ePVQyqis+dhpa213M497l+WiUsh8ThDaqzHBEvp122A7g+aRYZRdQ7nw9l1z2NsVI3euj9/efw/8fD6vB/94bjDMufeiOSPfH6OkDReHe2YUjo1Ye0zXTqcHQ0xzujnMImpMeu2pHxHpTAPhpraRR0G0GTmvh4SqXc2YAaSJLm0TOCyfhM/U8b2TylAWRNZFYe8oVKQl6WCx1tUtiUJpP+phRU3BrM7Hi6nMRGtVEhk+DKbyd+UOupMORKV8Px5vmNfe4WOUknU8nIUNhQYfw+Za8Vz5tWLn6IiUSGM9Wvz1xFq/Qdo4foHmBo8gBI2rsKeZYqagCjiplZfdZB705gUhsREY6hWyoDpRzijQlfMF5be/kw0BVH0rXLtzRFXFpc2bZcP3Sietb89aT1/iGQFm5yCp8axosspwQg2G9Tf6yTJW1WIH8VipNI1Wc1kJfxrvh5rb7IfpiJdEKtnxtFLbUIg/18s67wGa1x/wY7x6AVNNxtaEgzRaBVOzVzWrii5wDvVscKlD4k/jnhaluaompb1+HRQoVWAFDh53TlCe9O/SOQI8xH6l4fB1iCqFytw9VgcI47m2CHLk4RmoTDgivR+IdnufOccBVCNNwttGf5Y6oMOLoaTSYSv6vdS+4tZiAIo67eFuuXYBpgNIGIcnzMG8prmnz9BwPmjv/DIGaB+06bwqMUAageiDV6ncrAD3jhWhpCwWOBNZnQjMSg9gxRIrbjJY8J2SsGGOyLyfZoyvZ/z4mc/P3E8OSxV4CWamUXSvbAya++j7jg0YvCTPPsb0mSsVaW5kmBcbZ3P6mFf8/ANKU2IQGxiItY00iGZKFWNg5kV45K4PrDLN6etpZK7Ye4/KUt1bIL1ypGFzEHCaMnwQGM/9teIel4/xGG/f4v5h1yOuxz2v8T7++n7tx4h/vx/vl5l9Znz/PsfFa429U5X0sLF2lGYHYIJEmpcq3kSZo2wvZlRm7F36szxIpfmEkgaHJQdBZMLRYgxlNes0pw2b0i2IWlsSEpyX7Is2zRxCxEbjBskia2kZ0VyleBajEwbsLH4YxBlmW10Bc6UFM0kfc9gwxmlnJaUTUACRkrM6ewx7kIodmFEY97d/+evXP//I2I/5UCJWzMeMvYAkZ2Tca+/t9hhJ77p22MtfJrBGDVqobiVBaXyHQIdMdu5NcR2n/awo3j4HdVAvNhws9vaxwo5x0pEKZOhSbiwle+8QKcVJxYopTqNdfeABQSLMRuNCZ8u8+oARXo4/NKdaAujCD+oEb3i9p5lufwk24UprIvPgRQfAq3rXEGJBYWqIs9JIi3W2VgOz9ed2zAGU3I957WTJqQJQ+QL5eQr3SXEAirToDppMKVCbT15QjoqQ5+EYKvSJldGQSCWTr9PUflH8Zy5Twzil8FVF35b5Di8f2ck6IdTz1Tn8epKS6kYCBIw7qqWqGF0Z0orLpinQzrIijTJpngXikRKc3FlwUAXYEiCSHXIHSmSLu4rrR+fxZK95gDIj256ZTY2z0tXc4v6kM2EXB/JaeNfzA/rgfIguM4rmljLkgpS5oaCXYagiOTq3ryQcLBlaHdVSDyQwKIyWe2emSGk/5mMCvBfDYIn71vqK5w+tpyltXAXJArQUaWaDNlEmr6yVIxj0EjPX4znM3X3YSMu9Q413g+AYV0YGYl7+mA34mXuuTaeJiDLQoAYUEs3AiGhvXeQtO7Vhr3sM37XjQl6L4o3p12QtlDIXTMG9w67p47sibb7D5nx8A8e7v//2L//6ezoDlrYWtTnN3y5bpQsgRDzvvSPmyHwiehR1ZxAgXcxIRe5r2rhGdAlNqU9mq9XSNXoC6Ph4mUCDyWKHhqOWDFfvnnUFKNElZUiEjUphKkF1t/lmhPbO2CXgqyyXSofOAp6L1iodDE7tE6xyoVORmde8Wj9d80Kd6ezuVW5SUt4koXHYVOr59TWu+fP3P6ikjb1CtPk2xEQrELBTXyvXluiAS+H+QnEKeMfrCFB34Uhh+AnPOEnOjUrkqUv1APLANQYlEshMttDUKg6oxAKKYC/5rQa4RTJoRUraq/Tp1PmIU/94honS3NS0Sem1cLpZWbx++iliODuZcTbU1KeyNtk08G3NqJbnuZqCRuQbmY9aFFw5mA0BvZCTOjyqTtsxzdZpURY7ex2lUijONe9DyFhNcNJcjfifWbHhIzU9I5qVGaQ/SnXsmcfv610myzd20KAW2bbx4mS1lnqrP0KhP1U1q7Za8THV4GaJkqK9xGpos94bzn3RXfcB14xjIU2edeso3W2jlIuVadqaBBJmptiSB8CUFQHRJxmiYSlSROkUyveQh5QGgJLhszVc1nGv5iZomO2CezPNrbIdKbjZGMjc2qlF8Lts5iYu4/BoHIbQGu7z4yM+f97Pm7QE4/WhTRnnKGy3tEPqJV0l66x5F/LxPn362v75B+Vy4vMzP3/E1w/sxUwqy1iEgmk7Tupkvxxwroz3kUyklz316JNE7hXOkrEO0FPaa/+nj7d/+e37HA4Ycp/rzmFl60qA7r5jo+Ql6jg9d2UkQrTRz1XmIK2WpUQYmbmRTDh3mNt4XBZfO5PPW2+pz8/5/thfP/j+MfLDg+sff8T36/163JSExxu1lQsRlEi3ve6vdQMaOdVIAYXw4dMtpYhY98rMa77NOUPLh/mwnVZANlSPjQSVLIpenMJRNYAkh4/enrszI2swNuNAMRrmY/hwZgJ0lpgdRA2qZnAhWUz6kYpXx3cUfMXclOSvUp7K2q/QdlyzVitIyqxW6+vnH6RXUkSGrJZu1eJympjuHs9A/rRxVSBXacCcVKezjYDdO1cUw1lqyCCd2QTGYXLBRm9BtML9BSZXqCB+aUZe+smumMThNrsqnv7feFYkQpABlSwPCnb07ec1Ctkk0LmB1UD1kXAORxJwqrRnKA7gNcIYCZElLSwUwuy8pwa2y5h50CwezKqwlKYF2E2DhdKtkajCVlun06fhnyx/bcoAi6gD0TPWqa8tPYCy8mKOOUvZKg+pQwQOJkbv6bsPSjMaY0dkmllkIvECedw8MurNZdV3tqIjonaoVMTL61ask17nQrB7gBLRHnqg7oQX+ld/3gzK3tKMc6McSN/6Ep7PPuC2d3pVSVqokuQK7Uw2+GMZ9RhpR8jqbcC7QzHFL7BJKeuWt8LezxwrEgzJDJmVVdFCxRLHpR83N6jACIEiNS/e99rLdu4dSHfYAF3ZJyS1C3EaNr59fP/K3F/P3DtCslnjVK3srUcqWBioKcstXaagdQIVRbfHZUN73J/AyFTgft5/cD/RRuWC/bq9YQcQqCYmltO6tVRUBoExnAQyh1+3tpA+CKrsCbFjjkfcenu7vn38ZvYV686vp49Riz3q8M9Mc8/dc7e5EzLzwTPsUm4gR+SmMK5p7ohs5fXlOyJXDBNtxn4u7Z/7+Ygdb/f7x7f0sPXcf/we9h7P+55IeMSOe8OHD6Tlz694foWkj3c67cKYl4Zbuc6/7gXE2/t15zZhr7XWcpj7qN2sTnMbqR19SqlAudybZaLiS4yTLLFf5nhclnreOyUGQjnmOA93NW8FlW0fQ5IiQe+iBFZOFMxD2WKHw8ofAz8ObgdjaW9QE2REUpj1dJbl0k3C1/11+RuFgFixVFEjt1UnOHxmG3sbA8FOBGAqkB5Eyu/YdwQKXXdDFtnQeZdoCrYxB4JGdxukS2Hlic0yGp7G9hcocdr/JsMLEjkhDedCsC9Alb/0youvx1DZxG/XuIL+m95q6U7Fk9RWblmbFUUJdDOmurbw9K9V4XH2mRyUXl2uuumvApHdhmdd08J2yKjcnnr8f41uzef3NHnwmlrFVE/MaacbSatu+EDdfaKqQoohVdSEKg628+MKu3+V1Ig0wks+GyqXQzcV59VUQ61ZK8+zMy1qoCvsq8mNLiNo8K2b12Ysaz6GVe9HABnnRxX2peh7vjaclZTgpTKC6awJEGDu2jH2Fo4PmweWU132UN1hNAKWiTg75tvMFPgzTJn1xaO4GosC/QyFvxLprO2gTDUGX7dW3QERZzABHIjdz2CUVtnm5+ZOxxhKJcIfVzFZpZKL3GO+l2t/Q/BhVBbd2J16sgJ6yNoFlcoChYypXCQAy3y+j+/fBj+wxvpMeoRy7xnPz30jM6L2jqUiUDZpsih4L59LENQcF+t4r+86te+w4ZkrQrUVIJTGtOEJf95fiPsx7KIYz7hv1KiVO4xjPpCxWz3MUdO6KuMVqBkzo1pjB4ABvfYQZCXTMmnTkAkfJdJlmskitXZ+v/6K+cz4A2JuhUkjhTTp7TFBPj7G25WxM57LmSM90wZtEpePe6d0FAUZBJ73k6oBbyqxvtZaO1c4eVfxMIuUCYfN6zYlco86HqMIm2QlzholVrubGUq42505kDDE3lbQZcnbceQaVWUMNalBMBFmiCj7UBV9sZ7gF5gCCAHtDCLncINtBBTUVGLdT7vcbCCzXhM2GggoL495psy9RyOhl+hFKmWoqmH3znsrNAYtc5sxU+6HXUUxpejcGwDA3suHDpxSgwmFVL6a67DKYX7hzy88vbNlvMIYKuKmzrw8OEGjMOVzat6RJZlKnQe2+b+Gz19ZOtW21zRQMTuFpDc3gQOtqEr+ie5pjWh9XcJxVNU755nT2RFqoFg+QbKUSsWIUtqnE9VBhLrDr0iiFzZDUGxnwgHbjnao+vtjRT3uB9gRoKk1WvWafZTSRnKbaD6L4XI3oS4vm2avMDmDHfO1enPOwaOK/sHhXPv7BVmC5iZAqlnp49N6oxH6dGwmpq9Ml4mWfVBnpEMb9EZkArWCnjyHVV14o0WKtPy1bKJynawwuBRoHgkCcfh6gLsoAgF+1l3ucLM6taPPx5eQt2Ch+hxADzJkh7cjUmYegZ0z/IINED5G4zDRt/2Yj0h8/v7Hfe8a8U7XQ9QK+x1S9GXOWp+0DUKGUs6pfNbZ9jbng/lYXw9J9GeuvbatT8UWDGcNABTmyE1eo/V3PYSn0c8VbvomwVq8rFTu7dMwisOgDc+v28yc9+X8mH65hTDfrr125o47MS8pYA28Ajbn2HtXENEwgyNhduZrEjTPyDGHahgnWJMcvfaSSvvxuP7y/bfn51eSK24y066bzustR9Ei675TzL/89nFdmG5LeREKxM7nvQGBs0Kbn3d+ft3v03LHituE4UO+93N//kQKO/e6F1KA+5ix9xjFfLJyuSu8c15+IghC0Bjubori8QXLUBbPuRS51xtJHxUkFtqiGmQ/qp7DkeIMZtnJKQBaoleNtrHAZQKZZl4DnLTncHPvTa4A3HJ/cl7gOEWq1ot25wirEn6ypkqJQ/vzqiz6qDt/hSQHWwzJ0763hN/qGVFmbZhpR5zBD1l0ejL8Ehsmik/s2l8QenfnQpnkO3teDa9Xf16gTfE0HWzR6pvTcte0ggqDf41T/TFLTAVCYJQv97WiC6oNP+jvBYoswENllT/4ehENlXlhZlVVzCqAgdmITk2Q0WH7B8s5vXkPX2zYo+tMv+/e1FRnbL9bkESVqJq9inE9J6hRlcL2ohXAPiPrlM1aENbtv/ooU3scesdZ+VBxYi3yjHb1rfDwzDybimrOeYGB9Ws1w9WU0xSzetsMXqNb6te7PNMjXw13pkq6Of50R9ZBJ2a4mWghWGn1xmh1QLKC/upTZml+CwRLgwEnRLXqbMekKWnaxbUdyFCKirRwYwdN9EaAGggR5SySjEwidmalYzG8+IFCtwpcoiCu+75j7SgFmyVl6n2eTBh58ksya+2TkXmuh2oBXwg5wUfcH/vnB2KnDBFfa66bsWu0qyg+KSH4HGVLyZ5n+1Z0sx0RsUdJryBCEau2ups54DtX9g3hNgTmx+PxPgz3ImHTFMyV0DaYSLgg5tocXvmnQpkLS9ZiEioz9qAAkTJ7jP217fgB6m+tewt8fPv47XF9vb2bmLL8eiaIb9cXeLtdf/1k6/khAAEAAElEQVS+/8gd+/1v34aB9/LL1k5S47I0agWAtddzK2mhfFzzMUtZ49dwRSaxIj5/PsGHSqEcoBk4HPThZmDwXinKQY56yBmREQLTZILgLP8BvOtIPQ5zXli3nGOO/fyEDMaknLVmKhuU8BLe1SIUiEB0gSUFmMFSuxQNWVJVE+UVRWU0b0aLTsouKKgFvJ2mqDKRkp0fU/16IxkvYboUgjdaKGViJ1cwxMsctFT46PfUYD1ZJgAAkbuQbsJViQtHoncIyc416wddv2pHPSSpF2pSRTRfAw+JkBzWDp+ubiKRUeSk4Masb/Vg6zxLp/kiNV8dag/3/fPB0yP2/6tfDXebwA6124XQrJ/RanErXKHGkFM2z/509eigA0gc3L7fQuG7tQajJHxZ3453wam1zTx/oXB/UiCPDdgOYlavGPUv9WuC3DxSNB6xQ58fqTQ3h4Vabnt4wmpTf6mVC36Jxu6Ek/0AIKGedbqf6RCa43ZDT22FCcLMLCJ66KryU61JnRunGxo0i63XepAqxJ16QgquKpIJB4r1WZHnPK/hrI/Jgr+bEylVdfYlA4bwJ0dDTYCZ58PXGYYyNpJcCUhGIZNuRiZ178DDwQF6obpHyCGKe0UyqVQEAWTWnWBl5qyXRNmxt6udCyQV0SG8CpJu9j79nfFYPy7ds6O/9sybuetngJl7F9dohclJCbjEqg8tvagZmjgOFqQSu6qFaMMvoY5iETDtb9fbe26LsDmllVTGNopQZJqO4QtihsjhzFCm3ElWLHZPc5lpw3OFPS6bjiQYZUgAveRwnuY+MGjXjNzKleMCaXMse9s3AD6Fqfzc0teXcvz2t8dl7xB+5ObnnYpIMwdye2A4S6xu9GGWQo7xdaC84RcQz72HGyqDWps24tUrwY2eCUIRgUw6JNz3Njcfrggk6AyBWTgWygfXTwD5p4O4sYBGSeo84EFDDNpp7iiLvTsVjaTXw5YyMMQU5pjDB3lbMdPB5Mjt5OAQ3NsIIpP3DfFCtet/6yasyRRG0SLCzfauWEBXCUFw2HT0e+evSUVsFKselAoEpkrw9vpLjbgAr13EYm8ZrruyAIfaHJdpnZsmZLXNBYVKqlVLKmoVFV2+ddqIemPZ1O6B9sWyUr1EnKJ1kmgd2mVT8mMyyxInvY6vXu7Y5ZOoX6nKqVrn0d9yjWx2Cl+hWejazeI6q97x1Zb2nxCQNU+/wKjS1GY7iqLAq5dmqYTLdcgdGwDRGowKmOs5wbz2nREd2kZLqVaf9gzRZxfVS7TyT0MKqGp5Twd/1j12/aqNE/Wju6il1EKrysdofIIsCGezsmmtZ7uu8uyk+JESi53AaRnAM70W+58mI7UjCIbY9oYeJKyn6PoCpbo2eq3eOFHs3p1/Hc41PVXfctaqkijHslXxtGS2mGdHyGSMWrsRWxxZUIiRo1QuPTBXmDRxDB9Jo8eu5g7KQGyVtLcytcHQRobRUnvQpnLen1OwjfRBg4e0Kvl41vfSujEaOUTLkM8JMhsFj9Qw2Jijvhj3YZWiVcdDa4Q88o5IyhCbzBE7fz7X59KP5SNZIxE9A2MOkO0gq8dAxQLRaD6G9s7GJAuxZeN1EW1HyJSiWiDaRMTX/XPMByyBufdT2vP6ALmBvfH5XJ+B8Rgcnnn/4+fnNT7er+nJH59P5F57Efh4u4bbjngqRlrsPbxYcQKIDJFzPq63a5jdO+YY3cVlklwRIdEGYhuMsLUXBJMVX1Yij3qQ8xYkDkNqJ2oVEd1onjQfTjNtQMwIv6Z1ZH8L9RrxTQUB0dy6ze5aYUDUCxKkmaJMjWm1yIFeGSzAqAVD0+lyyDJhzsxEGscw9o66CNEMATdvRBZwIffOpTFsR66dsYOjYM+axBucFjyR7CW9IvwYDcl2Fx21IPFCWkovdFrCX//o6brufLNf/taqCLX8gAeD7UFCasIujcxUScv5+oLroDJQViC1sUQkdeDxVAZ02H/X+oKd0yun77CyhiZ4cebVqLUAtGOhYqPbPWIUHqJSe/EMHcXf1o+p7z6lai/Fouqt1g0WnJTSMHt9ojNaCZUDXp+1dTWofv3w6+fkU+7Y5k7CnBH5OocaWwu5u5PZ40ef1JLQMH+16FKm+TnZgBMpWt+4uVmx1nVtU8VCM6JAw4LVmcXxVMNRZaFXANUejvI1Y/RKRNWuruSf9gcalN3BZCLdWGbKGuKsp72or7PGrOZxBJAhgJmAIo3Yap9gD1zohQaqh7H6n7YRlpT0MANCClsMetCyyBPVwWduNGhTWQttJaMfVew5zl6viMgKAa2pvH/OkrLOD09egyMzc63nIkPzWsk7LKMQrurUXgKzoa5NVjcmULlRPJlbHjt8Dhqj41wcJdmtOBKNTJlzR8TeZu8AFNukXiZkyI53K5+S5Za51Y4BHREL6qiwkxgVMiNs+BgRQSPgKGGaOYi9t0Hr+Zn306535E9J2AaMnbrd88ovBa/3kcPf8fwjfeD7xzTwudZz3Vu5sacPczM3z6RyRRj09RUmuY+9cj+f7nh/n98/vu8d4TsYKxKEO8sBAyQNuZURNqf7iHuZe5plpqIwPqy1I8IJF2uL3KZvaJDZlNWx9GTCzcEEEyMzSitZ3FMeZKNvfVaBEmBCsAOd8gUUpNLN53AzShUjsVuFG7eNR4YEmcNLlqN8sWDFRFa8uhBIMKSJhpJgCh2XcoM0cKhETK+OntWJ24E0GieBKkanH6aab0oLZGaKcvdTiiZP1StzqlaWe5OoPRFnB+TBE15tnVLJgBUSQoO5m6SMMDPVGojWdkOJtCyNdW1h1nEGNGYqNrRyJpxE9HOhVyfd1T0yarePD08UFueqiOMaMvK42vubNDVI0l2yCg1W1NxwEKQEapliNYUt4S1wpq4wGu0+Zxj7AOA523BO31S6jaw9nOp3Ugd2hqwaChm9KMKOoqkYz3y9czDO9S6UD9YCntcYc/CuVjDXpSyzS2lPBdUOIqDJ6xeHUb7iug0q06WOzkGjDjJuktJkxVIUf608I3QRwgr12qJCUFv43r29yt/cOuUz15tVEW8RflKm1qN26yFWdIlaAEshytdPAAoh4DdMNqolhlFbw8zriykoHEIoFEIZ4BKJyIysLXBQLamvAVhShQJnkIFKl0QMm9S+n/ll+YyI59rmnzHvfUWSHPVFU9DpODpvBagsQ3NzWu5+7E9YHA0IyZob0mVmZiHtZ4xJJ+iYl4HLGGDuzxvOXlVtNNrOZIitH8PwK7EKgsyeIOt+YOZGWa69Oiw5bYNJyp3KzJ253x+XdolmYl7vGu7z+3y88fFtTvvbv1ymK3/s67KviPf59u0vj0D+/Pr6+z9/+PuDPnbkj6/7+9sAuXZe00DpmUnezx3Prx0xnW8f33zOVYEYBe+YDTdzG/BYS8oOtCOMbhfLd4NMpCICQOyoYbaQnTGmDy+wpfTg51vN3KASlwHI8myXEtQb4RQsM9jSehKmpgeqz8pR/D4o2Zbc7HI3ciNL6qVAxpf5R8rBUchJK0C3YVZsfu82MmsDkJTaSnOA7kOSaEtcJS6oxlNZqzQLEH9h4796XHRH2qK7AiaiUfA+M/JIcVq2ePhr1QBZQrxiPUVCCTPLDLWVHzg7TxqS0jHPVyU/g2b1sc4JoHQKqWOHQVYlqxUOGW1KaIgETETdyTySShih6lpZZ15BIyklgsadi7TyUZiV7wkvt0F1/1bAUFklreR+6oXBR65SZ6cZRbl7da7ZKIJ6WkD/qCI58CvbpyGwBlLM1agvM6Osdi9eWTyoWncfWcBdc7NlJCIBlT/5/Om2LxdV+Ys/Od999XwgvMgkoTdO99ABCG6dZtopFy1UOdxppszGqyMqWa8Ze/NPqXfMJERxIGp+vOP2+v0X8ta8TM+kkSSip05GhDmEtrURB6Zu5RjLxlIVrGaPZgU6cpQBLFjYDJB0Wq27ohe4p1QcMZjaYkjIT7hPCbnac1Du4EgeSUTmsgp9MVhwgLHWk6DlABMWriVfjakYM6DWrWu+pMj89bQUZt+fRaBH5hgOIzMyUjK/itgg6eYxfCiej4tGlN4xz567Qb+F3NratUrMKyQyGLlhrEA0nDasGghzV0qhiF1rjM6GKd97T7PBypce/sZEWDps+PWfMS7RnY9p11+//7ef//HvpsRObb09Lpv8/cfX13PvnblymK1A7v3ciB3Xm9Ug4m9z/fySsGMDOR4Pn9eGVsZGyuTD6m6rO52pOaYsgVBmZa+JJmzSEhGdDoSybscKo4xuVO5bpHkNYZCybkYvsrRuiVLl6PxmlX3zXkvZeI/LNp0oo1md52Q1FTBeXo8XmAFPkKmHsJHb/aqJofLaUAphE1DxQ1LR8xHo8b6ESkkhMSK5A6SrI3HqGS+Pj8pMRLTLy7wnAPyCtKr/OZh7A7RdNgrrsBer2LhOoqz99YHUShBVdW7AuYtJFe3TFbt6u1YzKTi0wcGmavpFRqPmJZXpBKeiLc3Qg3lpKpu0YK+Pq3WY6pIlkV62YQFZOU1VCLP/wDCP3s91RiSYeAYgVCENtkUDOCxdTyfW9Q81sjR2Ders7EOTBR1CBxktaldwS+ibgiFZWwfwC9Hpq1JzFfuv1JKys+yjUxyyemi6V32sb4pluMqwF1bZ3zpIfxk1jHxNMHXnVFrk602gG4nKTSpDuEak3OtWo5KhhHlGHe9l8EXZKnskRFc84hy9DQuVXhXE0UocMqo28zSmCCbbJ9Xe76IMBALLmgZhi0RVIlX6CI3gkA3aUBTaVKePIkLBFrCqwP+DkxXQfMZFKaRdgwXruOnJCKzF6MMHYci98+feYw53prSAcCs/PA2xksbKkRal3PQhWQJpMKJG4Mht7mXVUDWIVElKtCMsM6kAxIg1vfIkHjCkaGNsX6jGoFQmysIRYSWQFyFzz+5py/QRkJcyrzeRgYVjDsCda0sR8JHM4SMtE1HRoEZEflqEvf/L2F/5Gb//f/47bH57e9uxH5c/3sbvP9fn15ZfH99GmogQ4v19MOLH5+ccdl0jQ26myf28OTDnfDwmlGuvyF1ZfTYMMq3YlAgfU6nMJVRCt5CF2FgqxnX5GBlJadDNEhk+J6wXv8/hN2DGMk7oiDeqDSXNaiVnrePqlsMzerxWwvwQWWr6JHE6FXJnRu7KA2IDDmF2SfhVQNyL3k2Yscd5M29ekshUihk5jBVzZBxS7NTejCwWlxFhY+iUcnr3Wa9iqybQmv4jjK/M+mMdalgBUO4OZULSXBkEd4T1nYHiG3kqXClqKicnMr1AOrxI5VOFfwXNE0q3AQIKN1uxzWq6YLnlCdG85TA4ij2ioCghvBv83oJLdv53qtA/mB8wms17t6SiC0UFk7CynSHLFyWJQ2Kg33/1qCRaAIjCC88nK7Mm+mDsc62FsSCO1PbQAIUr/vn69P6fJudx4J36XZ02tz9mEXg6kB2gOlwPXlcCSDvvnp39kFE3R0ODpbPF6yAn0CtlMqp0oHV0aPlT3UIsEIXgUYyaSs4Fevmp2TS99YLmMxpZvTYKrK82q49+nLdL10sQ2l9E3zQqp3CtLipOwgoXBNArY1uVVBceJnjCk9P4SL8ozywjeyS0tygNt9yqvtuOrT97DVOeoSd+6XSpzE2gBxqFIa3GSygiE8pIxpb7TeZ4yGaFX/VY2DSVZbVzilp6h+E0dy/CsBjtHhTqQS4KrzzVdUwrkoOkDZvIW4cnMpqM5Ii9jQGzTNRCAXo1o9FpHy/QtoI2q3MZdQZs6zGrjFohgeZbsjtMaXNgkE7hZ/542uN9jLel/c8V6xordiyQfD5v+uU25ds5MPJ5r2G8Bn8+b1M6DLHXva+3a7hhXNSerjGG+8gMiAqZj1EuQDJ2QuGo7dCZwrDpNEzLTGa5SSsXQ7TamgSrLLAdY2DQWJrNSO3gY7C/IiQObVe3Ec8crNMY1sTKc4/WI12SmCIqLSAv/HNQ3iVKzcYbhECdAsNEecWZsVQ31da56LIuclBh0KSNokwjee+IlGSUVUBs1a7C+uxPXfYZ/VVtVv2K/iR9RNOD/RjqQMypPMlmOW2o+g7rRh5Z8XPnyKtDzOsxr5u8yuVp6M+MUu8yev+4DAU3BSKyxIhukUmFigauhrTCmtgIUijOVFGoixeVwHYk9Qr1mgDprX3p+aNa+uydOi3meYXzQLWXu96u1OEMKryjOnYAtCNYV2aUsOeUO7wquBsiugA2xHWKAFlDEmul6Wvces0s1RPonExleqhCX5PWIT/QMp6mNNidQGNZ6g/S+xJUQZ39PR1E5WVoa5rylbFWIxxqAGgUa6C6d7Is76X9B+m06q9Rn8xeI19C5cV43afnKKor26dulDmWwi80hEXRCTLBytZaV4XNf1TPZgFQtNMGShZ02CjAL620afVT06rikXDWEokkStFfoSW1JFPp5BBWu6iVYCmag+pzwsVBI5XSBCNEZpqHWcJEywwSKD25GjKuwI8BujEEd/P2oEbuNB+92ygr98EEaO/xNlujsHPMkfvTBFdqh4Ksw8PL6JFEVE0HkMjhDmPGri97TC//n7mhhVnN6KHYEKSwUUJ4uBTDZ0jaOyVcPt6+S/m4PHZCyFhf949P/LGmZc74mVTYY5qN5/Nrr3CzuNOJMfx9XnhLhD4eM3NfY8y36/ncyHR67g3Ah61buaLSQWq/diIzV+7t16Nb0gxlRODhjzTb+VWUSWm04AMKqwztcoqnhrkpMxQru8QRNLPhoOdeVm1cIqqFM1HM2DTLyGLv0U8aSEduWnOqJlEWufbel9cyRotMM9Z2cEq5F0dUyVnqSAq1N3iiC50iEwgKCCEgY0Rhhtihiley4npOsS+FSpaFuH6leyi4W8kDeIiybD1g+XHIbu1L/QKQoZ5OmxPpsbuwlwa9Uip5e4mZpBLhFGItdkiGMtJoJQ1i/WRSlcPWnGmdn79qbCrdrdbZsdMmMnr4qAGmapadY6gBK7JPiFMf68hisdb9Jxtn4q/DHc1c/On8ZHWlUoCjANbqWg+PUmB6sDSwBSDXBuPM2m9sJF8L3EvVCsse67sB789SBC8Jr2dWZ/boM7ffFQBawU0HK+pS2IX+nFHnb/xpGHxdkG6Jzi++SIxSiXSbA1R30mESoBGZo26w/l4bkxNY5zZ2Exrgy27wwuuaUkd1tnURW5paF6jhmDL1MaE+l+umrDSP13hm/dUZLYDyltOM8CI+00dgwGciE+LlConJ3Eg5FAlYH2fVlSADUIbGOIBmhWMqlC/PWh22m4qaX1xR6TQCqDD6FsWZmAlm5nAS3P3GRW+MzHpkSyBtGGpPLqy3+QhlRjMi9uZwIH1MwWyaTBmhztpOg4YPxAZs95IyudUwLGRyzoISQ1HHjw0iSJrhTBtAZgyjz5GqvDG0YslpTqRrLwKwR8aV8QzR5rfgEH3vnf6cQAyP3PGlr0jlHZn38x7Ayrwue39/zIH57brmiHWvn/Ht/e2a02BfkbG+Hu/X97e32Gvdz/V1+zAqMoM8cYalluM2H3uvir+8V9vJC1Ul3U1uI2nSGu6RcZfoZ3hb7Y0ZinIJ+ylq5KuPhWheK/dk5oqUmZAd7tRzdPeWak1kiy5OEE31VOxxa07knVqW1VPVXo9alSU46iUzgiYjYJ5qrnbvJZmbYmckVqiwToiZaVA9iUQaR80vp/DptVsc9J75FIUHFWTPCrNr9KDZYLVEeETu0xw2W1BKj1YmNv3JguZ1hiiW5YWqUYUVfkYL7KM5bT9np1YUE8vzTkFlotvKpi7ZSADRss+GZjMPpn5gnHOQ9FPXuce1qbjrh9WgZu0NOZ3l6/jqH8kUjXnkK5UHjs7DIKgGrClVXCVJ1Bb7TJ5vwax2q+aOXaBN5olBxvFbdi1mRrCSuoAjTq+reso6y6FUNx3Owk+2vqWzsk/5fR0frQPuU+T00irWl790v6dXV0NH9REi5O5NxyjbBtJOEAHMRPbFoVrTaj2J9SCrdoDUiVcqhZeyusXy/YX1PzMbhELl8FBn8MwCNaPEzlH/RjcDbEs3LDgymQEWMFBNR7Zm2918mLs1bFrX2lCabKnYYNXN0nu7yEKHSk3lxGV0YZBuLOl36fY354YLNrwqbytOXm++nv+IXWSgHWQMUGr3GlUrrMGcrhoHFIB6FypgZoOaI8yj8oozesEI3VT73U/sWmel9eq/A5VZwlQLGndGKnbspFKZTQeLXtgrjKQPGxMG4Jn3vr/S3z+UIC+Tvj8e3z8uCRwjAeX+5+9/3OuWYwNmvNdez9vH9HkxtXb6eIx57R2IjJUQBhyRsSKDj7eHu5ubkdecZpYpv3xcw91rtOrmanpU+AvRvsrYyA1L0HZpMgBIsffamXXU115FpA2z4e0QOE3mgUPaQSWcufio0QoANwIKqGAKkFbZlv3oAoJSwce7zw/YAAlLI6xLIkBPOsHh7sAwY0gwyEg38yIivBZy2XhufS0l/URUJpTIUyNY9aFg5YTswJg8eFIxQPWdenZkTL3PzGKEunxAOltzUIqjahAZcdwPnd7OU53tzA064FJfiVAmwmBmrkr2NpdoXTV0bFsnIvpQlB1YUq9ihlfET13eWm6sYm16NLGDFx0Mtf6PqSeNXukuKVsJpsoWEWpj9ymoFRuQBR3pABOHGkcfn3XL6GS+1/uSkLV2JBLk8FHh6xKEtF4Uo8ys6y4ciK579f5S1EeBVJ0wWqL1KlsnJbdU3v3G1J+tu1YeBEz9SXrSU7YArOt4DVEl3imcA4BkhIHINBFFY9QvWV9K9fBAmncjtqt76WvfX2oJLHvdNkwV0tTjNERFB1FSSckS1iwOzsdlXwJBmQw1M06zDGwhlRtYnEsjMeCTh1FOKJJJE73slajtUtGeB8CNEyoIkQY55HQeSMsoqKrtIun2P6CnCURy49ocAhUCTZ0GQh6oL6UVOzMzV6rrNqBX8FFIkVGqhfNtk2KmEFHoIGQIjWLzwJ1BR+0vy5V9NHeIGLPsQ7sF1KTlPjc0kZap7U5F1B5dVr+UqQil9r12hhR2zXTPXPvnP5BPmPvbX2wwYM73x/gYfuVCLYWLtfe9AA6bNEtF7A3h67nuSuF47kx8Lf34Wj++1t5bsp3x3LH29gF3IuMXnC2VFLvMvaV/8jEqTTcFnxMYSt5rr4yVKaZPo5U9x8DKXi8whs4iVw/scjQz6geYUf1HK99f+rW6pa1b5SpJdcGgBEJckrnXc2vuEJzD/BIt1ytR7NcERtq0WTZ6h9WpX8+XxMhfqrkEd9rXQsppg7TKji8fQXNIh/s1lgHRyxfS2IXqYSqkvmBr9lNHr6fRrLNRX4RBIwkNz2Y1bqxa8VLC4PyXiNyqcwm1faniAiQmEOiQ90beA0eZRhroLd3tWloMNs4vnLraJoBUR02UTr989U2J43SXPFhT6+2IzhU7E4fgJT6p2S/1Qo54XvWF/JFHIp+S0l7hEISyJvMeQbL1QZU2mIfG6GLBqr+v4+KQB/XCNT2onn+dIeGgVefP9+xSsME57ti5xQXq1q9A50TpfCEcDlB4SUurgzlNRCNydYS38mmUaPK8PovfqIvidQ5VwlxLryi0Y72V99aYEettFCHtVKBTLPSSy3stnC7wBxkgZaPNaZDTZBS4S0GnbBNkSGMEB2xCVIZqPmMWv23VvhOplV3rQdA0aKl1JxYLha0ocBkVSihuyCClwmEmMUCv5rvOYIbPhblR68PUJy5AeG2RRoHFDRkJbvV0gwZytFGhBX7m5bro/eA8G2LdR8754DXmO/y5BUXOYQhiR80YQHK4JEVyXJTEzEin6ObTI4KZO5I+CkBDaSM7ocrMlGQibVAhXXAvO5NMhLv82rEwHzkfssc///jS2NDQc697OXJe1/vbg477B3Ov6QOpr59f97qfz12z5ufX1/28jYZEoPwxEVk72TJhobw4Yu/cCVXNNkpzTkRSGD7CECkbHhmICMnp9PQxqjdip9Hwed+PjzfQKvXXjDoqkDlnRagSLkbzdTVCmTGDFW0r1fQpAeYZUYVBx1tep75VCGtCXmdpGkbmED73M/zR4WvJkknIaErk3uCQLAfMS0Np2dFdQaRCa+t5bwU4StRo1aAaWmaRpXmwNAyQkaWjO2GDLKm0ToRYK5L5p4RLqWt3SY/bQNBFhKdmdlteXWbTsOg6ZA3cF1BTrtJW6wEEElSoDPa9+lx9Lv3COqoxzQxUMrxbL1kQCLj/UngKYkuI2G8f55AVsixSEMscpyO3Ny9dX1XkWirSB02/p67gpbo5h5AMpgrXfZXROo1aiNKwFeh7L3PvrS8FspDW7qSiD1C/3ogQLV9ZPzzgYTfwfJ3uEnz0llyd5uSF/FT97oIKuZnUixDUHojjoKqKJGWtr4HMWl6qbmnQZx0JYcCoaJECqC0A7AhjtAqo6GYK7qYsuKrbZJysuyz1HsQaBMxUh8Y5dk5vUju56nv0LYl0EomoDqXUBgIpM4uQOAIDdpGzgHUxU7u+cCC9QJ7sBpc1bHcQD3rlRezYGxWdnt1vQO7Hysza2cLzzaNJGdGCs3Yf9HWVaDxbAWoErYihM9n0XQ+qFNwHyh6ELEsLTosVmQJHvaTJxngrcU8yx7Qa0pTUTmTa8Bp3q0/gmJF3mZdiZzpJ5t5KMVRA6nhcglZGZVuWdVihNJGISEVSae6YDpu0uH/8vj/+Zb/PmJ7X97Bxf+2oBs7s8XhzH4rNxPDpbnut/blTuXd1RlvMDE3TdV1737FzM2MHK4FdKTEicgvJXJnO3OnDjaqM8KwQSLK6h72TKTnNxq6vM6JagLXjcicxzEpB5u65d0TM6yqb1wGBOgodxe6ZveR5FXzTemQIEMs5gtMQ11QLq9igzKwQmrePv+b6uvcPZUake9Dn3kl3y9Qu01pkCppA+bQnCG+4lIRzjNB9r9hhmKMg8nHaNVTDJTW5qfJwjlQ06NtISCs4C5Foebz0Kuelo+/6VKgXyVKj4/hXcASP0i+NDEUwUifGsywELUVl40IwDhRojqB1EO4poy3sropH4jVmWyNwqjZFtf/zoOSdvVPvRLUfqvZLsjqiGqecpj7QTAQMW1maoELBT+9cT+arbulcIrwa3YJrOpy8r8T5S9XTGccY1fQ7rSG0+kNNF9SrHHkoVAdblXGra32Q//PTpVobnDKvRPNz8c8ZenCSJkZeOlf1ZqEz+rwmwTpEWyGV5ucgbxhPeTgiE+UOUG1y6TPqAK8pszNLlC0CBSH12yqijE10nvMtW657Ll3v6KXIAAGzU2ALQJUlPM/v9LiLBv8CXGT6EFngW9OCZNWUgqlKZ2dIsywMgNzKDW3mApIIlSkQIQpKh0k1P1fOg5Q7lavEQ32TOsaUmGcDdY3cxnoLXuvJEeqNkltGZyFdNVzUfIumuFCyjQSSZR7yMUjvZFyblUi765b2vo2yfMfWHpPaes/zLYDebAHJk2425vQ5xxw91EvFkKA8gMOsopihtVcikwF8Yn/VjXZ9e3z/20eCJzTX3MacoGXszVJVamfC3coJIuq57ntvGzYfM6gEduaOEFPk3rH3rlM+Cwx1jDGkjIyIyIgde+8dao2HgESn6sdpAyOwMjGGDafZGG7DWjB5dOvKzIjIjIxdGxF2GMiMlsNmN5RgASrNYLFuVzTGCHPBQhnVCB0rgPK2DLNhdCtDMuoYFxlmEHZm7JUhYRQuEiUhrcYgEWYkPQJfd97hyVlSoHrw6y45TSJQo2+q1ySoc1W6unWcESWxJpTOi2q0Ry3pgZR9JKqRnjwaGzRjfOAUdkPKV8goYEZ3t1K8EwbXEeOfK99gBdHll/0GDuZ0MNkDFRfaITcb5kSr9+qIsmqz6l+7N0+d5zOhnSHU5w3iICwMWPLw2NWcRsbrZ5JlCG9ACHawwDOovKKE6hpVImmrJw1SVvAGSW9y6HXZ46D2uXdkZmFUqQy9uA/rm/tMCj2rCWgLcJ/DBQypv45aB1YXudgOZGblWAAy67O2vsaCMith5GCUnpnoFCzRNIr7d2eEaGSoGByreVIl4UWdS69/9juqZhpngOgXThGZLY9BnnwovU6pwmF78Kwww/YTl3e8BjqzSMnGTgZHmAWJ4fRRFEHkBjXsECvZR0rdXmapWELUBmtktuiHKBGizBCLeac2USZnGWowbzkpQdgUvHm0PtLY0Gs1GBUplQEyI/3qu1kSnNpho0UmZ+KrjokpaYeNS8aedsGdJUsHBbMhRKl1JEPJo/FaHVerNL3oQOTJs21HD1CwL725ngJozUjVbJgRujcMhU67+1CaczyMFweGntuHfXxcKzCHG0XT+oq1cz1vRAWqoPxnoyxpxBjy4RspYSsdtQgFkRGxBRnyXmHuUM7HVALusTeJTg0wKtNrSfrOl6TWH4MqHC+Nrkw/+ThGM/cgIrbOVyQhIlcEe+SVctFHZhZKmF2kzhcjNL9qHSnjNNJszATuvVKZkQEZPWLvXAek7fQCdE6yKbGVqZ2AheAuYxpIB3KL0raXRIzja8XnSvk0HzVyZ2aXH5PlsfWdRs5O0SeOD6z7bpEWO2re6Hu4a2jnvJWGtXr2FoKWlNAbz9WBMxs062bUpSY0ebrAWq5DMV/2OVRo/mnfCq22rvEAOuQdqub0IDo0O6+IOEwscTjd15Mn6KSs9iHQiAutlAA01NoGHDT79M4lFesBvZEVnLcivl6iT/ju0esCqLveI9ivas1syEXnOKUx4/gAcKplCuxVYij2VI17mbqxs3y9UwHIyN4ooRf7rRc9TTIjGuWHXj+z6n1moe4vnYQ62e4MCC8pWpUYCtwJAbnR2HVvLj1gxmkXzMrSzV0XNINiKN0cpPWUbUUn1MiIRv2Y3cp1Z519AsvR4SGoUspmourcW5npjxyP9Is+KYf3zp0CCFeGk1GXqLSqyoYPqNb8ObOnmY436XCYprp35SKVKqnsmpZ0ozsFBEp2UnmElaki86lMm3bETcfHCEqKSFQQkPcz2HnyZGXaZ70iAWXsnbSIfSuzsgruZELREWaiJ5g+EHuYVSxXBgj3MSApsxiCeoncO2tYj9i5sTadkFXrl9abSiktwYhxzWHXvp8S3i6UIn+v/NL6/i/fPp+IXBzmE4ZYX7qfd8RGJCJJ+eyoG0k+fAzQGTsql8q90omIXQ+goJOYNK9Bi/sWko7Lx50RsRGgz4oEIMzgynyMqZ3KKERGuTMtqOsaxXNdYzzbyKOqAonIs8BZUfxQVnRd7m1Wqqk8k3r1NK7YJV7pkdiH9o028WXPf6bEXveX9AKrQYnwvRZP4nuofdwpp9UCYUad2JXRlJkBm+N5f32FHt9+m28b8YmufIjsFSpVmcmsFLHE7kb1eIU78qox8eqssmRFGcGOJrZu9yvGp9vsMhYZXoWiJ0tLlWioIWadIhhRnncCPO1+jUbMVHt0qtqfxp9nFmmuGKbWGqaZC0FUS0eoXWnZsOqxzmS6j2zkvc6IrvXF2VsPPz09qd58dXFsTSqOlj+bQGFX90zzs0wP/cb0Ous6hbSyeroNP1xF1f+OdqgbHmBhz6V1rjo6hvd1aHCuN3nVoOZueQgAdkQrzZh5iGV2NlxxBKPIsOxM6S702fRDKz918OyCElmNqA64BAH9U2CWNVEZUYRPAVlV0wTSqy+IyHJxpnJ6JSdadax1R0QNLUTN0SQzX9IaRjbOVArQlJq3qtgvMcqmQETIUWLcIb/ob8cXX4RISWIIMTNqVRDPhXJah0yZwU3Ro5pIbSvBaio7FqIO2gxJUaux6/AzS/rSSJlIo2VEWQkStPIqlzUrWMtwmEylY6TE2gHdVBOT0mtnUAFd3XEwIsbwjNAwOLhvN4HIyMitlYUd5q6kDLoPcGeEmWxOI2KvXIDEOQyWuZSKvSoMr8ubdibGmDZGx/majYftvTJi7Sd3mHvci7HdbT6u3/7ykZf/4+//fAZhnAKpvbeEMa8VnyrtpcEHCZmh1oNMg5grt2sDl4/R+9eaPNJ8zGJOYm8BEfF2XcPs7vpDY0V/B5p/MzPPvUHX3trlnM3K3Q3AzOb0Z5njSrbAF8pqiC3UhFb4Tk3Ttcm6fD2J15JwmjKTjZhHFAiecNRymORAAtarvUsrsncaK2obsW/rLmiUJmive4wP2ijywNIpDDvOGPgKbfBv/+X/+K/f/lXrK2PtVKzcO/ZasSNjR/1XmcjY4e6CZY0yoTMNZFGSB3E/E0pV1QQOBsjC0GvO6Db8V5PYFfSgz1XgwGqyIhSV8tDeL7Z6/sAABbmkINNL3X/mLBzYpX6riqaNbHW/V81l86BoUoCl3pegyNrDaiUeUXWMqPpeZmO8oBZlx+RmWXR6DVptIjVB7q5MOQrj6cz2WrHdgE+Ng9kM4AuMPfNWHSuVW9yVu9JwmhAouEnRUaZntY4Osi6oKNBOBIKErqmqNFlVN19FvMx68obpsmehMnklDgxS3QEa+lPRUp2XlQnBhueOAcCcZ7CsRriJS0Prn3U+iU6PLP6aBlSFuA6dwvpSMnMjyhHZ5useS1Jy0MojXd5j51atlun3X1OZpGuOu0LtxaTvNDMnLcomwNfYkE1Mqf+Zh0QCLEsKVpeksoHVqGjrRpDjeAYFBOBEwkRuWh51t9BfXNXusgmTnqDKRNHB9C0Jr7MqskAboJZigu4TmYj+ausYMJ/28C1Ps6LFduQJnzI3WAhMuEU2eq1IKyatNqWw27kau3JvFP7sNBtkRigdwycycwdNXhFGe+f6OeV3xvf3up0wHuPtw//3v//8+dyfGN/eh11wJIK6zAS75ro3uc1hxQKk7nvzzaZPwGNti4I89lqbMB+I2D7G4/Em5X7u5DanAu4Wu4I/95s/CiA6dYCDNFr0Z0gQbm6VzMC81wqJw+iAAKeC2YNZNhyeAWBUuLYfCvRUobq/rR8yq83t3akJQjcNde8VKyaAHGD0QmCV2ueuhhSiTTca7Mq9jclgmsxgmYO1ccRJe1zvwM+duYS//df/9r/8t7/G/iP2namM3Cti71hrrb322vvee++1cq/Ya+/YO2LF3rv+/AEuICTRjyg6HbEeLTutOU5RLlyAp+GlgZ0VUU19kbQV0UgJOsk+2+GqO7CmW4URR0lYh0e75V9Pdv2jBZ69J1wq0VSlyrOe0loOUZRH99GJaExbAGDwNLCx3zrGTumrPjnTGtaqxA5564PrAkSvVzQgRZnAjBYA5gGEpKaYxXzRs9kov4wJmLR5NBsHkTmwOXDyWhBHk03aC+93M9BqISFYCffSsdRmO/tMkrtL2RNhtkLnsLa0XwvLml3JPGlEOnnRauKjSAQzDh7Uiy1XT/8fFiNYr3LuIaPUl6W06U1mtdKBVSBRMkdKSKmzmquflraAgFeAUN2S5EHaihio8Y2teIAZHfTECHmISTN3ZZqQigLIlIIL7XBRqroIApZRwQY4nWULRAsIiZoh4+7khgga6KTVumNbNrfNXSdQs2oCYcPVjRVAGKx8O9ZLNtzN24cp+WAk5BVJCqeVSbU9HgdTpRPX42teP2/fNq/qcCpMRqmd8qyJG9qWIuE2UMtUptmYDWVkus+s8A4jxIyQh5kPJ1JZlyMzgQ4rM8P+N47/CtmYM90l/Hzi8+f++Xl/rvj4y9ukvckwtgbuZ27YnOY0LBPx3GtOs4H3eQ1H3ptuPmz48GHr3kbYMIpOjscg4UBQJ7rAMmqIcjPLHWOaagmJQRgdOmUWuSuHA2Ywy1wr9SAS2mImchjAKlGZQUk7qiQqgzT6AM5gHkehKBmtgyJsG6y4NXXQmB3st57u7k7dCVz3l1AihxKxFFxsVkJhVegZhnEgmdpe69sMghO8xoBWhC3o/S9/+y//p/8r7p+Zd6ri0BR7x157rx27/7t27LVWZOzagrrWvdda615r7bX2WuvesVflQgUydxpMYBkCVLpknA6zZ/W6F1tN1McGaWYhvP7TYI4A9X7sVKIMB/SG6Yn64Kr+D0gczWL3881glCemloKQ/QdStWu6fKOtI48MAW6iA6kTeEXSzvmmpjC6uqBpCBXnASuHdX9KodWRfchXO3/4AeGoccrZVb/Vb75eKLtA13mXtSGyEjhwGnCpFASWFGBuLTAvwVtkrxFRX6ru2PuDdBWHo8wQBa1E6kgQ2+nLY9uucwrdhvd7YGGu5ztTnjTsQEKDVcetpb/u1srwjpTpVDK3QusQwA7UiFPPyeGoWed1i+qyiO/a6wSWaFmFL5WkCVFIvAOJXacfLFJEAhqmeQ3MS7hkl1RoD7ZguU3JBAeIciEW/JdIuU8jaB6gIiSKK0kbZmNGwofH1zP1ZEmDSOUazL9c9t3pRppt+U3fsrtOTDPFJlRS+gKLI5aX5x6ku3lrjw7nFREkzcfYucfjEZ9fMCbzbc6+hKndvOIWLGx8zm8/9tutgefXkESvOwdTPMCUYkMkTbGBuSPYBdGhNLqs+QqEzMzoSiVry3maMXdECrmGOQ2MtPlfzR/+eE9y7R2x5byFO8VrTh8We//48sFkDMfXzztzQbQx5rRJAHm5Z+75cC0z9H3qBkzfKxsMNM+I2LHve4zxeFxaGXfu3AhFLGQosXfdsjmGb2UBO/dekVJirT1q21eozagc6v0tdHeYa28vWrJu6aiikGzqKwyWFrkpZUd5qgHmOpdb8Ih6ukgO89kCDpqUoMZ0g5mJXot3BtwHCZqZ2Zi7FnMWJJiLQNa9Qk+KtEwzv+47v4L++PaXv/037i9kJmrDlpRR4rStXWaK2Bm5MzJj7x2ZO/aKGg32Wnvt++4jof79vtfz3mvF2hGx115rFTIjFCNgmWE2cMaExn5Pue++sjJ5lNlCA9tK6/AwwYioLU6oRkmq2J9zcNQapTYDs8KLOqKpDHmklNWyW1WTmiu6pSWr4LqRUjSnmr3bMlCULtuBVLhn/fqOdHdSxf8VXFGF1qw1oeyK3aT6SzvXhfjwsaesFyhfj9qLRWo8ubEdtPOj62Q31kwIoWClpZUjqYiWXxxv/VYq3b066yoo1XBXw8xOpFAbppvPPrKinnS63T2DQh1yldPnGTHqo+cGjDSLTBN+bYCvU7O2DfQyMyMRKWadq43X5CGDahphb1LuC1HwIlmiTzVbSmTtHZXq2yNTFpVvMN/G4/2K8cZ4Uw7Ri+S2lEWCOZxWxiipzPYssB9kDbRm9fWbmxLmEzXCk5sWK8rxl7kd6/vH9a/v9pcJM6aNf962t5epgFZgAVpKRAoV+tmuk2oHOJxmASctK0Yuts8LdTNVbFbugjvNbL7NfQdiZ4bP4T53+o+8/q6P/yw3In5++eS4rrogyCXznTFAGyYNZNzPZWNQyBWghgO0zNiZ9SyZ13YKZeaYV5IVBdP3SMS8hkT5SHJM/7yfZsihr3Uv+xym3x4fmGJkxIbjr399+3qu3Hj+/Ez5u9sYJeGNVLgz1h7wkjaZcY7J3t+V+97TLfb+/PlZkNn3v3z/Y/9I23E/Te2Q2LEsp/mMjZ1K4l4r0szcB3YJSO9sogww45zmPlAEkkyK0gW3qlhxlhSpfqHSv8s3nKVzy1c2AIE2q1QTlVkb/mi9cBVFcEWEW5q2Uswn7AP9IJv7mNOLbkuAmcRdjwmJcNvasuHj2mnFR/18PiN9Xt/N3l4VExCwX5RwVE/YBLMQFSO6s86DOh6ilsutnXvfa8fa991HQ40G973utZ73Wvd93+u+4157bWWxmyHghMnoFdFTr2pE08ludRnUSTVlPq1OAyqrowgXUrTe+9gNrh0Y4pdDrKlbmqPIiZcl7GxG61JTuiJJ0Io9fBSUj2Tt3KnXqDsjQ3VgGKHsrbw1q4OtscrIAqtxeJODGlUfXYN9LXOtGpB8SfWPcukFl+EwF2zOVqlUrYQzUwnnOvane/JTvEvbVFeHh4zvnr7/pdipUl1msxQvoSrP67468gK+zUx9kEmZNgZgAjIF2qhjoy6uUg5LlQWxWAF14XNIAdSKGXO5UjaqugMUQmedF5BiG3Rb41VPWLHbJXRMJUYrfgA8hl2POadf13g85jXs8T6F6z/uK/85988BMxvTnsC6UWV5jPGYv654hKTMXc4bKOqMUfPelgGQfl0UzW5O5r2BcOLN+F9/m//nb+MvUzbGzTf/sX/8fQc94ZTlDpN27GSi7tExuJ49Nrf65ZHEYz5gnrGNtmPbCEjmPTQaGWtl7e4RbV4KZeYYFz1X6B9p/+b/+f/A/+3Dfg4UgeLmtlKguU3l4hhGbmVmmBFawKz1TEZSoVhSRfwJVAdXCRkbY1p6YreMglamDYH3/eUP9+vtzvhc+zmUI//67f2ZGdQz434+3+l/++tfHR//ff9HxMTCGIYe9jIips8U7v30McooISnW9jHifq6vW67IzA0Y5rzu+xZy53Zy0oL78/5ye7fh5kMntYU2Ym8+ppDsKPmKU4rrcqP5nPOaddSa0eR7RyqgbSWBbU9M0TipskqWXbuJ3DLyeYQKz1W3XoJxZwSk1iU2rrDji/TQZmDnntNUKi/35uy8ml2CnjDbqQNny4zDg7GRQ8yN59f9dd8sz2PJVFH1IpFpSJgmVILx7G801fr8hLrWZKZ6D96xX+84PuzYuWKvteqf973Wvu993/fzue7nuu/7eT/v5/78XGvte2VkRCaQyNx7OE0qwgcJZSTdUoDcLFHSNQCqkkcRiD+Rc1VqWmZ96OLG33TULOfio6axhg54etzCr0m8bFMqeVn30C+fAIHho72bnX1QZTftoD2nFBaE3OmkZh57lx3QkGOMiMzSZCdf0przemrBGF84dnUOLdIPHEHlaearTPfnPVtd1ThkX0z1XXrirPvj1XI3uhVCWWddWbCahT1xFBLRa2P7iJGIyBilAs9ws2GliKpTvg/eZhIASekV6hYyt1TtXy0owqS0/qsovd45glH+44TAbOHRJFykZu1CnD7fx/UY17yux3g8Hm9v8+3t8fZ+vb1dw4zD//lH3v/7j/xZSwIHMDdW5bgYOgetgIUKYFImhUw+730NtjQusqyfNPfropsCxAaDuaFAxL/+df4v/9O3/9vf5htW2viPL/+5CWakw6bMUD1sZ3GdTHmactekCCIU5pdfj1TGLXPRmFsbWwS4q3sbY+61DQQ9UhDdLDKkmNcHvv32z62vj7/s3//t8fGw2GNeIphhPnbERRuciYVaLNjTZsLKj9yRji8JhR2wuxhsq4ET5nMyarEO+9kePfQ6+LUy38LcBeaKHUrF9+/XX94f//r2QefX3/b6uv94PnNnEJGblu9zCJaZ98LMPYantNcec+6IFSsz0tzH8PlwE5CxM3cQuuab7YrAbvGUN+IWFFhesNhAKneV7argYz5IPq6rstUQ6rJJy7wPwwLU1vJM1vrrntb7f86Oje5HMwVYTbt0U2yUzNy8hLnonitJwV07uIUH3L0wGW3KqqE12NjYe4dDBENpWmZXIpixF+djjvH2+ZX//PklwDHUsXASykkjQI2ovk6FmtMHm+PqTjolqJcP1FhTfWidB5mIQnIy946duXdE7rXvvfd938+11vP5te6v++v5fD6fn5/3WvfzeX997fuO+zNzl/M5pMxtLlgFRy9FG0VRS5+O6PlA711Vyq9Ce11vHgVfl8cq31ZYXDMHPdsXkFf4kFX+R8Gu9YlffOX52dVZt7Cw1+OUhkPn2Smq7HiGo1+nYhhKKRCZrE90WMvcMsdRWfUX9YsoaauuSt9vvYriSD7y1avzFOo6MLrmvoAyqGWGmZ2ecE4Q4bzmnwarXx9fqgQfq9SMrBGkf6+p47KzjU5HsB7xAmD+UijWTl2SXpdDRYaD1LCa9wCITlB0qHdb1+56+fR5XWP6eJt+jevtut7m9THnYzw+Ho+P+f7x9vb+9vZ4vF3X++NxPeZ1jTkuN8bm//5vP/5j/6/x//o7QfeRTo6hAHK5GXLnZvmZIlCrhYoGuP7/lP1JkyxLciaK6WTm7hGRmWe6Qw1AVXehieZji7A35BOSIuSOC/5sLrjgIE8ofGg0HlCFrrrjGXKKjAh3NzNV5ULN45yqRuORCdS992TmyYwI91BT/SZlBDfQFmloRE4UMQ0QyBe6uq7uFbRmtF+92f27bw6/ugFfrRoiph9fSjV3khC9hm4o1sETkm+b2jUSNeJWJpAxs0AE1bhpSLnauvIwuCoSRXa4u2trKBbvAhYGVyA0YdlNzflleffy9F1u85QyODsYmDdr1syQCpTIlerbFRAwcEZwNQsBg6khIRATMrgDI1pQk2Rawd21BZ0eEY2QhRABCdStNJ7IzGu1VevagBIhYGnaSgMFc6+lNDDOYcG1akBOJMLo81qQQE3J0FQBwQmaqqohAyWWlAjBTN2BkFSNgCNz0lpzM2sKVZERkdENwbuvupm5oisjchZVFWIDW9aSGDz2dRAgkJp6JBWSczwvB2vNwQhDcdhLapdQbGynt9jAh92a03tt2uIdmZn9qu/2PnwaWN+J5w1cwZqBaRXmaOSBkF0I1GJz3zVXDpoDgDcEp1L06enFzAC2nTYAeC30ALCp5DZoqIfDIH2uLMiRVLHNvtTf9L2/7T/BoGfyhidFzd1V1cK13bSVpnEelFbWpa5xAJR1rsu8rvO6zOu8LPPcymJNy1pCmuUEQABhREIPajcKIlCXqBLFRmJX3yBvgOAvASiEmF1S7p05JqSOPzuohy4gzAQ9ma5HLfezOXpZdN8O6W2zDfcKG702bt8ZVYu2ktffzQhIzGoao1RgX24ep0IPOus62E4KbHU9/u0OXTrZgwA28UvU7G35mveBKdKzA0q3L8JINrqFYiFMrAjtSgQwM+jZqxv3sMHwvqkv4bNtAmNLDwbJq0bCgg7IPRkDAAS7VLHT6T0JNqwTgYD3F8nA+w5sAkyIYGkUFsxjHsaUhpTGNO2HYRqG3TjupmEahmkcxnGY8jAOwzjkaRimcRqGYciDSM6ShIWYmQV5XUjSp7/7wwXwpC5mHOkHcRNpxNwaALhWByQHM2HCGJLAWgNVbQ279stjJPLmoA5gYDVuksOYvjmMb0baS9WqDEyIatSAFAhAAng0A0BCITMgYPNucnYzFlHvKzKQJQBidACKUDyJRruLoSCUoRS3BhKAmpvTMADR8XI6H/Yfh6/38u4gzlqgaTj0CIJ2j8xjMHCg3uuGMMm0mlnQ/sjkzZz7jE2EqkYArrHfTUED3SWtlZiY2MxNlYY9SsrD6CiltRVBQdoKObGMGZNcWns+vnz8+GCGKTEJRbQLMzlYqVZLZQIirlUBbMjsYK2pm6WU0pDcvZpqrTfjDhmZeC2lltC5Yms15eQItVYnFBFUI2kE1FBBG4t4wqYKZsa4lrK/u0WrXWEctFCogp3da2jxQpUbAzEljBXy8TaNOz36lhiIGQXACXSjBQgA1YzcmVHN0Q2A3ZUYgcSuzRUAuHpbzcxwBBUiQSB04AifQAREdnJFcwRfASj28yxzeXi8r6UOYzCz0BcPXlu7bazrX4vL/0XjiX06sPjyFV4PWCAcxQCw7Q65/rOTdGAGru6RrKDmMSLUZqqt1FZrWVsp6zIvpSyXeSlzuVzKOl9O57JeynKpy6KttrqqmasSbduxwTEAWIocN4v217qDBrZS00X3HSO61nHwOMLjS9ux21cZUkhIYz6Cq78GEPp+sMDQkfry8yi/gGDqRBQgTmfyfEseDcrPlRCBaFPZAERrcH1poU/dUdghFqsihly/F1GzfvhtoVTRKG/nRtfLOFkobjsUeT3HOjXRx6A+IMVeX3dz4O3gwi24ok9J0PVOvgVDBd4eTzPaBzcXQDaNKSM0+y496iKyksOJuElnYyhjksSSURKlIY/7MU9p3Oe8H6bdMO6HYTcN4zDuxmk3TrtxnKZpGodhHMacc855SJJTzpJzTjlx7nVfgJEAiZzGtGurffz5n6r+Z3VGHlRJCVGIyKGpqQZ00Vkf6PGHcVpaFwxABIu4eifxTd28zLPXFaC5qZmNw3C3n3aJxNXBMjE5zA0akFFCYq2GphjSNCBkxu6Ugx7YBrGq0K2XXGBmUOuQn0NPmomcECQUimuMiaw5MzVr0SgupT4rvOfDm29/dzqJzJ9oXa3MTCqZyb3VCg4igrGi3A3UkMS0USykRCFvHpA1kraWRQiJGcm7+1pIGEG1AiZKiQhtWWXcMaA1w5xXguZORIlzLUYMyIkSGeL3Hx5arbVCN5+DY8LECbzVUktTEc5ZTNG9sRMRt+hJIy4/DCzVxmEQ4Varq+Yhg2tgCyxMfWEkaFPEKCLWTElo2O9CYgZLLXWJO90c8zB6N6N2RDQKJiIjsbWGQLFTZUt0i7Ycwan1fnAD/bFfHN6KJCIQi4Enptg4gEhsSJEIHQ4WJnJEd9Xm2sDJagMqTg40OoBrI0CkRGThlxUHwoToyCQ0LvOnh8eH8+n5sHt7zZf16Ouv7NrnD9qqDm2YQjxYwu3PHU+JT/ZNGNdvu4b/2OcfBw4MYNvWU7CczMDCLW9mTau5WmuttdJKq6XUUsqyLJflclmWc7lclvlU5tM8n5fz3MqqpbamYG7mTOxuW95KizDra+AlbABPpys3f1fgND3Mrp91kT0DG2C+qTK77SvC5jBcyF0E4M5EUeXj9wSTHXB2f5ldEaiZIqJ57JICQwvqvyOnPVW7P8aotf2EChYYOx5vPWE0rgAiGl1D6wA+36Fdw9l7+I4nw3bmfSYKejrV5t1zBNCeXI0b9uvb/eoQ0tpuZvDYDbcxDRA/lpAJTBwAiFQtMpyRoLkiosaGZUYgkMTCJEMehjyOadxP436cduO0H3eH/bQfx9007Xfjfhx30zhN0243jMM4jdHpD8OQ05BSFk7CwiSMzCTMiTBW2wWeH5o0toYJk5Iy/DjPUJUaiCI693MPIjXIw24NLKHNIFNjiR3HppGLCSHgVgVgAremqq6rt4W8gRui7Ua6HfFmwGyQxnGtohWXGv62tB2u0ckEEMcA2DPNQyJhRpJUG5RVMfbegapJSP4RmSkgK3PovuFIOXCIZEh0AZZqUBxO4Le3h8ur3zw+wPjifPo4UEYrQx7X5QToFrt6kK85S8gUjlwgAUhgGvkzgQEEmxHvGUBnBU5DbZd1XQ3WcRyJJQ8j5x1nbBdvzZoJSRYZViML3STbsrTm8PrNYbfDPKynl5OuZqokIoS1amlWStmPkyA3NqvI4YUlKIDCEqxVQMGSUy1tXQqE6h8IiTRsRKpdlKeBFJmalbWNMoiImlszZslDjvaylgp54pC6qRJxFzMSI3kLcWiM39Cxia3kAWDvAre1KgAEFlLo4OU2gWhTEmYkcmjuZk5QnSAzisISyywBQu4mEMx6LCjVFtIOZGQRiKXllFNiBtNa3LE5KeBP9x+ej5dvv75Wbesmky+K/pcffq3h9Pkbti3C1xng+k//rJgIRzpsmTXx5a2p2ZpZ8zgIthEhg4WJxE3NNNj7Zq3VtdS1liUOg1rmy+W0LMt6Oa+Xy+V8Wi/zcjlrXa3VpjUwIhKOXxmIWmdJqZuUECJjORyjoRGlvkoFPrtVfQPQHYN/pq0HjrCAnukEGPmX8fTco/5eq7H3Xhwo8BGIokzYX1oLT8l26mAXtYfoOOyDDt5znbrgYEv/jwfcw3lCEgMc3XsEssSr7/16bsof6Asa8Tq0bLCee2j5Q9WGRGwQ8tzoyTyiXAKtgljzvvEtIWN1g65wJRJnhPA/IxoYMAKTZBqHcdiN481uPIyHm2l/mKbDfncY97v97rDb7Xe7/bTb7cfdNIzTOEzjNOZhymnMaRAZs2SRzCLMIixE0mlbIOzExrafMLJ1Y+u8qRGCODSr6/k415dV10Y4TKhJDSTRZi+PChzHGSDClsRNQAARveLhW3YgZDcE17rEaQze3CsgAEJmnAbJg4gOCLkgPq+XczMjQU6A4raAdS4SiQyAIYCVa9+AboYyuJu3sFDGPsCoIIBXKzJAyFggPKporsAiquaqwLCU9jiXNA1vX33lmWmir+rK81nUVTV8hwAOKcgrokhOJgFXqxaMlkfQNJkwAbhp+KyBEcENhSWhGiu4q7ayjhNLmqwuhChpX5hL40ZQS3NhdCrFYtkvRYg0QF0LoZjUWo3MijdHMG/73UHQa22AQCJ95jIXDhxsY/U0Vh+7N3d3bRoZ7gSIKMSMjCxMGJZZAGFBaurzpTQ3IvG2hQM7DFNuWhHM6qo9vSAFvwLU3/rsQE7ECbv0o4M/8e5yd4VIlo0hIHaeNGL2DpnGE8e+yggBwRHVzRAEwAgdKJtWpARAkUtlhgANwN2TczJyYnGozT1REiKCFRFrbTyIqv784f37h/t/ByUg87jBeqk3/OIk+PPzgP6FAeH6Vb/W/v/65Ogtqv3FNwMEvBRexPgu3+xhto0c0YcFEmpq1bU1q62VqmtrpZa1rEsp67LMy3ye59M6n+fLy3o6zZeX5eVktbiaxwIoipszVFfOPSzziqx3lB2BNu/rBtM4QGTPBU4NHfrwYPLDhBcTOmJ3toU2cTP2XX94F9MjOSh0o1p3gLgaxCb3XpEBek+xIX/uRGSuAN0ielW4hqQsenA34wgy6NvlIU64GHF6kBHGM9pcabAJiPo50DcfAITS1PsWPOsnDeIX5DAiREjFpjnyLRMdAJDYtAkO1FqjDDnzcJh2h5v9q93NzeH27m5/d7e/vdnf3tzcHPaH3W6/m3bTNO7GcRrGcRiGIY2SB5YhSWZKQokoCaSOe8b/E3V0D7D3HH9xG8PnEdYpUB1msVM9PRznp+Nsnp0yc3YztIaqsSwRPBasBKNhECvZNk869UsX6X8G2GlPbcXrAtDQGrgx4T7LSCBE1tBU15aeVlwKIGegsedYhyCaUMNRQgwOQa/GWlYEAidzhL5nHJj5igvHrRbaKmIGQDOva+MBhBKgkK31vILSch5lODwN/IknOHxNYHI8D/M56bGuJakRpiQswgBkZe0NW8CG5NZqhF43N2yQh0xIyBz+Q+u+dwxdnJqXWhgBWCKwTS8zpFzBV6KGLHnynIuWZS11abRr0+GwlmZrmS+LgRERhTGNUFsbeGCiWqtVdVcmCvwxNo8SsWrTpkCeSMzcWhMWVbWmHOn9SPEOQes3t7kBGjPmlBx8rUoiZt5q7fGkiM2VU8SENTN3TRCJ3CS+3SqRKYZI6g3VkEjD92tgGHYrAyJGiKVeQfJ392rcnAjEzJKI1gAWTKs5ABJjAmerF6TEAIps4UggVHdSRQQgJiRAU1MATKRiLdaJRQPEJC+Pzz/+9EMpi9AAEDSgX88As16iuqjkz6r6v3gGXKv8VrD//PNf/sf1q/7lV+M3m22/EeIIuGpTqD8gExpMLINtwehqWlWbWmtaWivLutS2LPNlXU6X88t6eplfjufj83w6zZdzXUqzNVbosJMjY98HGvwxOeI1BTSCFtAAupQTPUSo7q6OTFsxhcBRiOL+D1LFA4sN9UQsqNjwv16JETdUyqCD3lG1+hnTk49xA6riLzpcmdQ+l9gGwUV20BWQI2Drzl7YNNRfDBC++Q770Np1loFwhv61kwHg5k6hX8JtiMBNIBeRfD0BEzb5QV+4EqcRIcrf/G//LR/yq8Ptq1evDq/u9jd3h5vDYX+4OdxNu5tptx+j4uc8yCiSRbJQZhZmIeDtybEDhUEZLYanL4v7F6PoVv09qiD0a9qvLBhAcwBzW2t9OL48nxeQyUlUCdRVG4G6NjDtlmwUBGdKGHlAPV8vhiF1r4iRiRGYiRKa+YrQ4sIxRgnRupSBfSnLw9P8fPHi4sDmg7Zqqhw7lJGYGEIMHpeAGdSIBZHjpjKy2Gtk2vdW90Q29Fi6553DiaYkpHINCc2aMJHwsrQnbPczr0ifLod1/A2+Wt7i99P8THXOE6KZNWcGgtSsUvQ2Fp4vAYUaczUF4s6ICOqOwAyA0idqAwbJCRIPBGi1WWsC3JDnxerNQNOgjjnRNMpSl2VZsskFFxKo56Wu1VqVLCkH3mokkpjUfa2rV8uJAm90VwYya6bq7qotp4RObVmt2ZAGg2ZgxAgGZlBrTcKq6rY4IRIQMxM281b7GN9qdQJk5sQOULVqYmQyDxmQReiEA7gROAUejJ3fw7BQMmO0Z+Bu6kEIxGhPxI4EHl5Ni1bGwSkaVQAEZGL3Bm6EQ9Nw4yg4W3gRGEiIcrZaQYHYqX9FmYyZEzWoDcJ7o8bCnMfL8dN37384nZ5e3X6tVhDBVDs3EYbwXm9juuFrwCPQXwA+X9b3v2i47F89Lf7FD+o/4y9/ZGDo3eNKEQND4pQGQhcDMyDfHAm1mba2trbWtpQyr8t5mS+X8/FyfL6cjvPpPJ9Oy+lc5iW2QwhKELhO4Bp7giColii9hBTpzx3cj/dzfxF6HMUmqQRwJyZwd3OOBU5R0DEIT7e+j+VzfcSt4TfrcFHY8Lb6Bb7hY9tc0C2u4BhBE3GrRCYPAJorGDqquyGJw/YF3GBJYlWNu9RgWwPZ30eGmynONz6c+XpQdCFb4FXxF+I+9w4F4XU2Cu7OmjGT/J/+z/+H6fW7u8Or27s3u8PtON6M424adkMecxqEk3AW7BgORiyjIQBB5AF+xhwB7L95Y12hxi8/cz0nrmH5CGiACA3cXi7Lp+PL5VINJlUEYMLWvAFoKO2QUK2hYN9Gs8ms4gqqmYdnw1uAAETmrmYrggFaqB1IJOXsKJdzbXxZip9mODdZKykSJQJAVzVvyAREgOimTqLaAnlnFCAGY0dyC4FoCFK2rHNHDYY2HOoOYA05xYvVO8zmsWOaiHjKl4o/HQsb5JpnPVT56t8d2rfauBUWcq21Vm8NOAGRWQMohCxAwBQ4X3UgJleHREhIaIZoTkys2KA1Rh+SaMU8TbEez5qCJOAMALU0cRwkyTTOywWS8DDefHtH1pbLrNWysIzJzAidEVkSIDjoOi+ttoE5/LdqkBMhAYss6xpDAzjFHneMXFRC9mis0NyhLwXrzq2UkgCgR8w61fNlNSemlJgYKFFb67S7QTchiTgSRCRCQDIkB4stumCgsaMCPHYKBbenrl61ww/hVex0D3MA1NYjoHvLyIyIhGwArou3iuRA6K7orakRT5EZJzl7kFVsBEQ0ISVkzURMpLXY6imJEBcChIw4XOby88f3nz59Ouxe6drcoafrmqEwBtJJhITxM3rkKQBA12JtcBB9Uff/4l35/1f1//OSb1fm+drNxRkQv5k8tJGfmz4DAw79TiQDZTVQtdqsmtXW1lrntczLfF7my+nlOJ+Op+NxPr6cj8d1PrW6trqCxoZU6d6I7sUjACBE7ZsrHQGBIkTj2hH7hgdtXCwidD0fIm4jFqBr1/Ve++6w9npv8ztPEKgLBKYC3UDuESna4ReLnmKzdG0yMuyc94bAoGoD7EhS9wbGT74WS4/JoD9wh841OwQq2dkD9271ouuSmRD4xO/eIijcewPaj0UIz7XLf/9//L+MN28n2e/G2zFNkgahRMQEccPxVpr/4tbpgVxu/3pDsZEo/9pNthnV4iwlcwNt7el4en//sqxAMgQ7DRGirhWCv/UoFYqM5mpgTY2IXRWsmm5YGRKgEXTVJpqqrWhKEZEBKgCga1kX1eV5sYcTPS+wWoIsjgpgRAamhATI7shpAKJYrZWHwZsRBc6IW9A+ujtxH727oI0kElnNjJCJgJlSEgRWcyAQQkCnhGlkGuhkiym1uVxs8PzNeDMdxjF//Od1OVJdwJoyCxOAIgH3dW6GAaAakjEBgSMBEoD2iGSt7qbNETKnYUTIsTRBAYGQeBhLrUAu49AKNGpQVFWnxPkmJ1IeuVxgGJiZyaFaMVXyngxftJkZAzIyEYA7i+SBIaLn0NGsLykmdteUBPt2Jvbw6Xa3igOhqzJKTkJEiEp1nddFROZ1zZwsBi8LXTMkEWEOZJSE0SHy8QPy2qDe/rb1TrYhmLF34bdF0j4AxG2PbNDII44dKSY2Qu57OuKgAUQgyhGPA1oRWH3xPIpkc3OE1lZGNBcrlchZfBoEkebm1XHMQiCI6Mhq6kCfno+fjvffXL7WAqYBWbADILQQFFCkR5KFqIowoshj/o4qgN3gjOBBWvX+PYytX3IB/7OHwb8wPfjnfwZ1DAHs2qZWwqtCibZf1mVXQGZEwGQpwCJQB1MrbrW0Usu8lstyOc/n0+nleDkdz+eny/F4Oj4tz0drpdUIpImiGe8tEEIAaKpOPQcbN+wcr881UB4Dx+CxkfrFJoceQgrXjAmE4FR9a8Dj3YVXHv1zLw1bNBy4a8iCkEJ/CoDd8RAQB/bczBhC4rwJIqHb4npCK+HGTnfasAdOAECkkJp1YBsArh4DIkAgQO18dH9csB1ifW4NfWoXyoAwyu/+9r8HmzKQ0CARyAi9tf9XP64Q4b9+D12Hg/6naBD6vdJvx9gR70gBICo4qranx9Pjw6lV5oEFuVohMsQWsrhIaA5iO15djynZlOPtqI7s5LGqkNABmvV+CrxjAq4MmNFRK3CrakV9NloctY+BaragN8Dg7gEJOQkir6UKkqBUrNBpFkCizXio1NUeccBznEYk5GqBZjJhH11jrReiA5CRcELHYsXzYI1eFD7sXv/4+tXdzYHkNj/8cz7/LPUF3UQE3MBUiEM0Z7V6aL57eomZN3LhJG7obq6FiMnRzAfJmAXMCZp6dWjuOwUvTos5ToJM55dZSwPQeqktCTCQUGsqsYCBqa5avAxOqqZu427ENGgtUVLdDFGIqalpi700ETjjTDAMOaQOAGhbzhQiSRaMb0TT5i5e6rosF0AHc0lJzYU5STdYtqYuzCRd1G/ghNq3djj2jqgrMTreaG7uggTggugWLYYhMQK62qYfM1DjiDNSQ0rIQ6xYJaaqoS8VJPZIPoMw2y+tQZ5yZnFlMsWEieuQG7IOOLZQJcsAsWDWwK0SI0s6nZfjaVbVtmqr5mBM4mYoAubIiMwEyMK+JU2EwomEEICpbx3BMKoiGnV9PcTt20/Mv3iT/gXE8698dFogOkIH2Ob4UOsAXGd9ArANmSIA+nLojzWj/UcJjUaWxXQsBk1bqW1Z17mU+TKf58v5fHy+HJ/Ozw+np6fLy/N8eilLKaocLwciaOBjXTUa5n/o9b0/WXfwCAWIQo7ucaE3LDh4WnNDvyY3wLUKU+hMernufERUbIBoxLeaGagRfFbhhreAkfv8b4qxRiXe9R74fj+B4gXE7jSN+znkPgw9sy7uXwjao19ccABsEQ2BcT6FlzjmjQ1q8c5q9EdlLhO8YyICIJDYWLJdnv9f7oN/6ZP2+cCHTZsEgBEe0rUxPVLBoKG7BtIH1gDAWgHg9TJ/eni4/3RvzSW5aQMAtwLQzAzMBChYvR4FEjMeWpeSBmhnCMhIChq7WN29q+OJGLQBkjAOQomwNTP1qulccC5kLmDkatYamZOwIwNy6ECwh0YDRnMbiokOOpupbsAfEhITq5k1deZO4Fh48zUW+6kamaGQNnODcplxN8mYYUyZBdSXlH9gcOHL3XA2fmXwhiTPL5mQcbDa51urFq5Q8EaERCSSycOOS8iD1jmiyVqrWWLDmiODNdO2MkBdzjjducOqFOkyz8dZEoPQfp9TIjBDUETQVpHI0dMgQgMRqLbmjmbMQi497wiihJo7MguwD9OOhGD1GsH+hJK4rM3UvKl5RDmiuwOTNmvc2lrWdWnappsJk6CiliqM+2lSrYgJ3EupighJOhbQPUQKHQAGlFBQ9WB/RCdz7xAWKQZChBxzGEVGPPSoyUhBcyjeiImA1aoRuqPCGUiSMDAMNzdDltKW82nNY7r95k27nGtpwpZ3NOacsABj81IboCExK6lXD7UakbFDW+dlvqzLytpjmsIt6tawL+1Sc9fakKg5kGxjZ4vqF4R7BPMAQegUEQMXQ0SKtMygJjtsEnADBDjwP88RXEnjXii+JJCxGxHsS4Yao/uO+SAeF0HsU3QiA4sQHQZh8CQ+SJtyM6it1qJLLcu6XJb5dD4dT8+P56f7l+fn8/E4H1/avIRs3d0YMIg/6hot3OReiNecD3RCAIWoBbLJz7vAJnBz3Pr3aw/en15HKcCCPcYtuLS7vxx7L2sbUNbhO2ZwsE15FC9UYErbJnm3TZPW8ZxN6Bo9InQVbE8CjBiHrky5PrL4GdElxKOOcWmbYwKv2gInukpJGISgC8w+R4Fvz9k/w/f/Te3O9SPsi379fM+gtn6ixVjx+QEZEpiFkM6cwLFROD7cjy/zd+/v7x+ekVJTRVgJyL2atTg2Y+NiMLifl3sHTgcOoMQRDqRuGrhYkCHgCqHoQgRwQsxJAm0z46p4blycGiAhN6sxyJoakpNwLDgEJFMnwS5DNQViiDCGGB5VmYeu/Y1gRcAuPvPwLUOtjbKagQOrKiHwwNpqYuFhLG4ECELDbiqLfbjoUujFdk+vfvfb6SYff3/3/KdWj+6VSMAAUyZYwJEyIWpbVVIGQnVzM2Lp95E7ugG5swGoqnpVB6Bof6wyYtPWWGgiJDgv59t8Ow50e5OnkSqqrgUaNTVGYgF0QFXcUqVMY39HLKdzFnZAc6u1AGAWIQAGLK2wEAumlK254wpuEaGKQMixl9JIyFxba2VexjEPkiDJab0YNGJp7q7Qmo37wVujnCkNVooDA5LF6iV3Q+foQjfZd2QNWWilgzTDDUJHRGKC6uaR9XOFjwAAIOKzWFtjR0ciakhYSVHkcDcODI8vNbHevt4P4krnNC66mrGM+xu0tYCej5ck03C389mcULGZrYCckC5rXUt9WdaqhkBm2jmn0MdH0on6pgfsAZMGsfgF3D0IOiSFbv4HaLEDAEQ6C0WE6E4iRHTd6xQlD7s4G9A27d6/8PF5XPiv2kTapBwARp8PBvv89/zzzyAHAIvz6IoSx7dmJnOwPNgIzcamh9qs1rIsy6mUy+l0PJ9OL4+P5+PD6fnxcnw+P7/UtnIQJEBuBj1QAQHJIgQfYVuTrCGXh36jbivSiOMmCbj/irwEhuL9A8Ctx0pcUSHv52A/cjojAVcdf5xGwWD0FmcD+q+CTexyoH4+RP/um4CTcQPJHQCdmMwUtl3K/eq5A4eRYhOpbr/dthM5HmqYtBFBGIK1BAi6wABoy4+LFp7iccb3AG6XymJuJ+iAYDT6cfa1UJr0cuxf/IB+pBiYOW5LMSOcQb3EtQOzT0/H799/mkvj4aDGrVTKgu5IucO4Da+canMncKKebR8RzhZJtRpNsTtwxLcRp5j3EZCIdzmxoytyzsBaz36urYAAo1oDQiay1sI3h0iqYI7mquYBzQKxmSeSTkWruxkzglZHcEcSwY4raJwCnJK5JRkYhRiqGjA6GBFJZk7IDIJStWYZOeNyWs8vx+dS7x0u7+7Odzfncfe30+7bh9+vD+9H0Ga+k9EVzBsgiCA6oYAQmIMrEBC4Q3hP3JwYEVpbrAgYIKu1ypzTtKvD5JIbiJsRtK/fvAKDgUDXBlnQnQk6w9y7muZNxUhGYSQAdUNT1WbEQasiAhFSHrKqSo61pNBUCRHFi6qqhr5BVSFYkSQOMO12BOZudLMXwIFT1dbWVbKYeVUvpagWtwmYRRKyOFYkjqgW7YBpTH/xJ+j9GiEDqBsY9ciXMJMzewwOkW0ZCpMorghmlZCYsJirIfHo1jinvJfd28PrX0x1vdDFmOxm+gq0ss3DWD2T+3GZ55vDAYE/LWX/5ma6GZblRaFBEjQVSEBohkX9uK5za0JsCBKIsG3NVbRPW0EBM2Du62sdMBLrACKuJGxN4GDQEMFalEDoMbDVCAmYEIF445epQ6vo+AXDjBu4vyEJ/60ZgTotDAB9TeCfd4m+/f3rERLAUW89e34ZXE1qCAaQmFzIB3DL7bDTZuubt6XUZVkuy3y6nI+np4fj0/3p8eF8vL88PtV1iUAFwRRBDQRdQ9ntUbCB5PEIrJMGZo7gvb3umRIY28cRetFG9IBcPOiZbidBc2faUqwNzC2WASBgjyKP3b1O7khM3hS3IM7N8Ry4UH9U13SKftED4nCAvvrQOkPtATHFfwcLAf1hBNsATsjYhVLYn3xsgECUPz+cbbtmdv1kEJnb1x0+CzsdwGM/y3aUxfATfW+0KaBwtWv1OQWgZ5TGQ1FTADUE8+YEAKa1vX94ev/w2FScsxpQHtxKT+pzAGD1GEBRvZk7Uw9qjxVhPfile6bjfDRiCU6cmAEYNCLCsyAJe6tWkF6aHRtXgEjhQiKtTih9+gYE7ElCro6ZEMmtEQooIABTJIlt1hSHHnwinbHqEYaqYXSSLDGctGLg7OAiTMjuVmszEUpQzxctc12XVgzG/XcXm3dvHvN4nr769/zmW/u7fPpZvBgq9zXnGtImSOJ9Ww5hOKoMMBELx5o0d5OUzRDBUYQ5SRpUjYeM444xj1nW1l7mImPWS7s95ITg1fpienNtuq4NTGU/bBmy3qxnJ4O7GdXWHDxlWeZ1a3/cmpZ1EdkzOKAho5bKzHUuktnMUI0BzSogNGsaezVJ2trcgEVih0mtFdzPp8t02BMnQEbkXsWo7xpDB7S45UKoC06AsW4bECNRlcg1rFsE6DFDOAIzb9GUgm7mmEWEyb1ZY2bStTleXn09vvn1ze0rv/84O1naJ28oLgQNBxgPbMtZoL55983LyzyNwuOYb2/l+cl9HV7f2MsBln3m026fVMs8n9e67lKy4Pug1+Q+xPiGejo4I/blqJ3ZCD2CIULQGBsY0NNQwOFKCUezTx0qAIIITESMtRt0JZYR+06nWIbbW9XPgMEXH9E+fvGHbiXbvnnT3fyZmsiv5mMjvz66/o/uOMD+WTFyodFEd4O1QzFrpcytzOfLab4cj0/3p6f75w/vjw8fzg/PZV2tOrIQcvD5qI5d63ntrfvLucnno0z3/+znFX7xP4/G3B18u4fiOQS+5vY5lqEnRbp3kWqIEOKPEFH54HDV/YTmp7fu27DR2+Yw9AaXYxsB0f0BfTToCBWAXbVM8Vc7vbEdvubgrg0A3VziBPZo51r/Ho8QCYp/d18FhkAHYavdHqALWI+mA41Tyjc5nTuid0+8Q0zjHpAAdPmQmYGbKjA2A2Iwh8tcPn56ur9/MU2UMnKKBGxTBRYWBwt+C1ozR2IKU2ZcNzRVJgIQgBqPhzKBIbRmZrG1PIx+hDQwZjLxorWcGz7NcFFSwECavCdfM0RKMSI4ajOINwMxIDEJsXR60SMDOMyufRCMd0y/sbarC0797Yrk1hTRTNHRndzMqgaloK1AaarVTVmVss8AH4u9DLcneXN/y/9ryH+T/373+H3mKpzmyyqtMonnQYZkTrbGmSemZmCELGlQb1Cru7tWJzJQgDpOd8BgWrU2aE2rDXu5vU1rg1pbWcruRZygLA2gEQsTFFNAJ+ZWFTYVc6sNzFg4j4nB3UyYtKmbppyYWFu1psRQysopDdPgtbW2AlMahZNQEknMmLTVpa6mWksxkNPLGTMzDzyMgGTO2iJDVLWZOyEnoNUdzcw2LLi/lbSBK6EjEzFZVE8IsX8ktSgSGTgCmzfomt9OGBKyOaqqy4QUESYMLOYI5rs3N/tXyHwkeCAyzsPx8o/ulXLb3SYe7fD29iZP2XFacJRdHvbYCtZFkh+YSsZ5Prram7dfa1nn81Ori/FofTWHgmPI3+HK5wFuhcO2Uhrvcb/mO6BTnxS2HrNvV+jlOZaoxJjTf2QEnXXWmHru2QZxo1BHjbYh4L+GhTfHABjalQnszeOXB8W1GsUfvyAMroKhTbZHf3ZCABkAM7CDJRAkGMcbG/X2UEpbyteXy3K6nJ5Oj/dPHz893396/vRwOT7Xsqq6mWaWYAqFxUy7NxhZwd2cmaGH40PXUaGjb8EP4Ne9AtdhLI5L799mEbgbhA0CbMsKbLsOm/HJN9PJ55cg1qXBJl/163ngG3uyFXlHMOvmBoCwMWyQUuBOMflhBM44AlgH9/pj8OCGwF065eANYCM4Ag+CAHxCddvBo+hrYw5wiFY+cC3A7vA1VHAAC915LI6AkFP3vx1GOjVDMwSwwOdia7ebgc3n5cOnl6eHF5ZRUSJ/w5oCJ5HEScCwUtXWnFwou2kz7U8bwA001HsYeSMAREHcGPSRIF4ad81MYyLQhoy6+tqkKnv40hH7rIvoxHEtCNldESPACLuGN17CQPe35szJwzhgsW0aNk4+9ryHOgXJDbQoUqzTI04DIYEH49O0YGwTSJIdCPMw3dw6ixH/NLcLvq23Bhn/F4j5+EcAJRnbspAxgVltQMhIboauRGDeHATIyFDByVS9gDGE8h4VCU19WdplKX6QBlRbpK/SNOWcWc1zklqsr0zZdGacBBCWusQSV2FkSQDOTGnITevL6eLuQqTaam3uxpI4JSQgJuDAGo2ZiNFUAbJDHYY87IZSVgCqlzqvJcOYOaGzmZkroTkYsgA6MmESWHr8Cwq7K6i7V4icRILY6hVjdPd8ARk6kkviZgqmXVThtiWOCaiqtU33x8RMiCQ0jbvmjiMNw+7NL2+W87kNpyHLelETq5c6Dbu339yJtMPtzYB5/VjgPN/tv93dvGr2lKQRslZzaK2eJO1ub9++PP+4LqbNPOIkwAkZ0LYkyc8wCm7mI+iqlRCJBGHIvQb7VqkMupss3pydAfbOCMRi17DI9ggx7REC0YTH3kuy2HQZx8CmOIq8BezHylbAA8bp2MEXnqHr9OAbArQ9raCCKX7hhtx8ltpgx6d7ONI2f8T3JaCc8n6Xbw+71u6W9avT/Kvz6eX4fH///Pjp6f7Dy+P9+emxXI6h5lXvjKyrxyIJJL4mjTKxQddO+iapj2EaOgV7Pb88DgkFBwMlB7i225uxMLpmwA49bQg79syJ3rv3VDjEQKp6fxJXo4P420EUUBV2IAsgWtqQofWNY+hgsRYCCcwdnbFniSCRu6s5E0rf9uPg2Kl7NwMMSZ47RJZaHPlbZjRst5VuWll370FIsNne3DepaJ96+s0J7j1cVKM/h3h4Tkiuaq2ez5cP9w/nS2k2NXMglLRDBnenlEhyrQ1zQGkJvHpP6HYA8I39cnNAdq+csgEAaSyqQxevxRHAjBCTMIJTSlpqcTg1XFqYmlFVoavZyZHj3cPxhMCiDQqSCdyaAQDH0+m0hlniwEHBw5VAFGFMpso5m3ktxR1arWETQ0ThDIBlrTKJpIwAtazInHbSlHC3G+52woOqn87tI3gdvjoBr7+4+R3l109/mtZZZCRItjTIIWAnIkRvS4uFbs1UCIEQiZl59BC/uVkrRBUQMaExLbU8P80PxzMgHO5SdEZqVlttrUkA64yIOIgwk5oRs7kxk6m2WkVyykII1jBNg9VqRRt5WatZG6Y07cYsspQSnK2pATESaSt1bgaOdzQOg6d8KzLTsq6LlSJDEgI3LFVZyJUQxSByGgZlgS1CwCKUOOZmdGsGCI6uvfZp9MQIwMyO5GtzACZSrYBAzM2IKbtdEJpbNR+Bwg6cEDGLMIMPSEnK83kuS23p5fzMlhPm27f7X//ur1+9pfn8uMv75bScX2bE3bu3r8e7d/dPD8HnXl4ujlbNJ7xpNK6l1labrqEiM2txT4VqEELItPmc3A05gmN6IY7hODoRArGO3XRZIHRJYvSOFoFFUROipmycXWQfeGhMzAkMnBxIPaKsmCicCCHIRgzqFT5HmPUCGLIiBHACN4ofBtb/byuitBX0fgb0OgXXnxKlvhuOo1nwv/wGQjAEzmSZpjHd3u71zZu1fHt5OT+dj0/PT/fP9x9f7j8cHz6cn560ru5gallGRA+zlLVNkUgIsXSzt/0I8Yq6u4Op9ygp38LrusckgA2wazzDlQbuBGzMlX0LfNTRLaUxLhMikVlD7HvK4ppcf4ZvAFuMJ9FXWk936NJX6841C8bLAYNQUXM3DQTMDZjJ3cS7kW/bJNWB/F75+whH4Aax0BI2qsC8Y/0d/dDrAXVVT2loJyITK+5Sc4Nu6r7a8IKENiJgkfVyfno8frp/PJ0WhR1TQjAhA0BHRGEFkN2IjiHGsNIAQZBwYxHQCCxspCCSmDOhNfVgDbo0zJyJMvGQODMiopHMZo8FCohTIh5qbWyxzVJQONLSNRIdiKF19I2IzSGWLHtk74UIwckcYhF1JJVGTjWRmBqquVlZCgJqrW6MKaVhUC31UgsQDwNVQo5dMczDIJKYeV2rJgAES9aqvmD6o+yBf/1wW/+DDd8U2NVPh5xLnaHUnAdEdoTmCk0JmBVJDQkIBMDjEDBXbRWcWmkw7QEQiwKACIiQCA2JeWAFLdpWVbPGwLG7ZUiJGEMDMAxpHHk+z6316b3FEgKLVh3UtLa2LjNnNgJHmO5Gf4EXP6oZuqMwMzDlWgsLWGvWWIhpyrq4lqKmZlzXpYFrbTmnOOAISRLlcagnJiBAV9UtLsD7qvLIz3bV1qK1AzBGYBQFsxbMTVNVAEVxQCQY+vo2015oXIgHQXJHpYpJZKK0V9jp3MoKK+44Z755lb/6+u10My/nZb20Uzmt67qWvJNvbm//2oH1Uk3dW3M7uc+IByBtZQVuVc9NKwktp5mJEYiASJGF3cHVWZBYsM+ZEV5E3jNfCbqhbTOG4cbXWWzK3mZWD8VwByK2ljWOkihmvc64x8JXxD6UgHl40rq4lBCBiYi6pbUDA9tsEMXcNmozsNf++W01QTceff4L148rWBTHYD8wNjgBetoY/Ln8HAjYwbNkkWnc397dvnnzzbfz+XR+fnq6//h8/+Hx40/Hh4/z8bRoiXgW6arODVf3batSnIhMHjdS7+muDAtu0r5Oplz7+ajOHUIA1z4N+HYABIIUrzG6bVyu9YEDICCh/klCaqabubUfe3HCdKWoxeoD8j4TOiOZOxEBk2qLXxQUMjFrU2GSSOS59ur2+ex2BAQDBbgeUREL5deD4M/+FRD3NqMGIYvWl6ME6o7XyxZnBAL6tVgTgZu21j49Pn14eK7GnMdSmmQJB6uTmLlkzuMUwfZgoX3tIz8iYYutS92HQiTuYIAhwCFGLYsQQlAFiJPQSMjIRf3U8NRQQZCotdjd1ZASMAMlwAbWYnMSc8awtvdJEGMDblVTB2YhR7CGgSEgAjkhMnO4p5AcTAOQjSDMWGbpbqrmiO6g88VSQklx2dLtBI61GqE1be7A2WN6q8P0/YJt/6s6TP9uuP3d/f9nBy8iY6sXAY6Oz0Jk6Z6GfZZsrkbE7GCmSApGkojSsqjsEzED+5RJhacspVRzmlImYVIf8jjXZqrEOSNlRmttKQUJgVBSNiQSyknQra16PJ1zSizIiMBU25pygoSl6eBVJEs2SclNDVzQWVhSagXQQYiFpJTSoLVWzY0Yxl1a2upu++mwy6StlnlpLCLDbnf3Qg+EYcIgcEUD3hrdCH7R1ogJATVKEgFAoHQKvTwauqo7TQM3BG7UKjYE4TSwQwN0ZGhNq69EKpIA56XVRisf/OBDJv72N7c3A7T6cDwdT+/t0+X+1btvSpNxGKb9eKn3dZ2ZiQmW2YrOIhlSUSoyyUsp51KNsFkFci2WJAlxK5VEHAAhHBau1nr3QgJOrtawEWdhgQiOV4Oct1YFQQ0IbVt7Eli3gW1wTPSAnTcOUrbjO4QYwn0FAKeQTRoigBEgEZpS31CKxERCAIDWk/UcNjCYesXXXgwIiTEyBb6wkkGfBmizJ0Sx+EvOOcIgwIy2PFQPGvk6KQAhMAPuxjc5Hw7T67vbd2+++uX5dHx5un+8f//48f3x/v746VOdFycn4liwsUW3Bf1OHYmP4hsZQRz2gs8+YOgdOQECWRcMQVCuph2ji5hrIVDrK2IAVTViGSKvzTZYNQBngIih9RYrnK5DAG6kI9F2GgROFvYVA3QL8bl5X5cS0JrFM4vhwOW6MzTwfuo4TR8Jr4F2sBX1zv+6xa0TIVmd2YAO72A/I/q39/zmGBSsT0zejxtyAIM+sDjYZZ7vn14ej+daGQRRqKkShi1ICZkIGRyDHmsKvl2Epp0J6UmMKJLifFZt4I5GWDWJUGthBkf3BC6mYL4WuxRanZoRU+qcihUkAYvBFlQRwMkN3a5rSgCxKWYJ95JpNURkZtycNV0dg4woiGhqBAKq1oqBsSMzmJHVomrmjWQQJG2GoGAusT0BCYRBK1DM4UBMu5udEKPCDO17nOrNr8o+D/kiH//TAapAppRIWFsBRQdDSSw54gShqDs7urUGZEDWah1ff3WsoEbCWRWcAcCsVbQ8jpkHuqyrgeWBp5QcRLUKOuTcwC7z4tZqaTFjMSEzX+Z5yBlQUZGRgHxIyUiXVgF9t5usFnIbhmHO4rUhMyehlNDUysLCeciqdrqc53lptQySndDAbg8HoaGuS1lqFkmSlrIgCyVprca9joBoBu7qZmjM7KDI4FqJBdGtAUnE/yMbITJo5KOjNnPHYXzV8GR2RDEZBMVZXCbimTUDZeOx8eCcllqfAWehmg5+9+og4/kyH88vz/Wi5/sXzm/ntRXPb8cxjWu7PBAUbQvkESf3F3cpKmuhyXf4/nz85w8/vzq8SgCl2XKZd+OUTTjn6BxbWc0VkIpWXx3RxmHnDtiUBAGoBWwPDuBa1siLcCJA0q1f7dQlWJcD9cbSovdzvzaBDt3nihhhNd7Fi8QdaoNmSGBogEhBJHS5LUQMPBqELWMr8Z2RCKzfALCFMY22FJsoH9emPkCIL46HXvDJu3QyZoBr0e9HmAMAsIMhuJAAmch+HMvh8Obu9dfvfvGr08vT08PD/U8/PN2/f3n4NL+8tLmmJCTkVUUkfOssbLFpyi3MEnE6ONgWQkBBCMcDj1Z70x/EOAFEiMzRaSMgM5tZ6G/cFEl8i4jwz6IdMDfoa2SdWCLANmQ50GVCHZUPcfNVChSTRf+B+tnrESyvat8bKtA/q5/JpQhZ983a3I+aDcrfMCnvU5j1HxAv9zU/N86JjVyIz3y2ijtgX2JjIbMlIkcztXmtj6fTy2XlcXTKqujUDIkcyA3q6otVNVBrpVothK4AzAIABtXRe+MRRIdXdLNWEAlUAWxbEwfqygwDAzMgklI61VYcnSQoFEQgzgjkQEICiLVpHOPc1dQUWTsiRMxI1MySc1c+9c1vQMTMKewIXRNkhkReFB2BmZCckBOHpwMJOQv0FwcMWJBclSWlIQ8pIVGtpnUFYEZUbM2sOP6gtMpb/uo/WL38dr7f148ZRJjdatOmriLc3CiU8QDmVZBB0Azc3JnUnJIAitWqpWqCNORWWlvqy+Ml7cfWoBq11iSxWqVEaoaggE5oSNnV3MysFWy1kZqN4wSt6Foc0ExZhDMtZT3sd69ub3cpH+3S6SxCIHYQtU2Z5qaqCIAkwCQsss9Nax6nccrLXF8u5/llfnN3O9eirblaqyUBY08Kxlj+1T0pgFZroCNuVYCUwMIraNkcyYVRzQqSarnwTjy9SgSqn0DBWmE5WCo2rMNbfvPmXTokHwnxlKa54tnLhWClHflYT+vT6cNFF0dFHochcV0aTShDau28rI+Y6v7NdLkUB9W8GKO5Hx99f4cfnn7+7unnX7y8/frmlS3wcj6VZZWcD7c3OU8GVsq5aHIHF3o5XXzV13dC0Gxthzc7QW9grZaI1V2XGQHvbl9Jij1r4dUAJ+sRTtGhYVecGAD5pt7pNjhHA8eocGToofTv3qIQnjez4Hplc/zQxieyx6o/AERgA93EnfCl5xQ6Pf2FA+26JgQ+C458+4u4fdY/f347KuJZxeFBfXQgAAcjMCKmPOa83928vrv79s2787e//M3T08enjx+ePvz8+OHn0/3H2lbXjm5jaD7VHS1c/bD5MLYivJHU6BZQTFgpgoKLPmTT85jWiCREQEY2V6Fk3SsaC65w85DhtbdGJm+qtlUwvrLM6N292HU621GAsS2RvmjNAfyLNLrowUlUI5inDx92PVgQru06bPnY/VBwCLNVtwW4bQuGropWgE06ih6VHj/LxhwQUTvFYYjQYs03YrN2vizPx8vlom6DSFJsbkrAoIZWCb2Z6lwB0RXRDMGJO/XhamaxYI8cXJt5bWYrArgWbxUiINrUzcgxJ2Y0Jmjql1Uv1WvXRAEhSIg3kREhETuAM7amoesiznHluzLajZAzUzN1JwDvW52BiTmlZABL89AQGIAguVeOXFokJoAUa9MSEIkMwNJaU3AyFyIJhax5bepkZW1lXYcB3ZUAPSdihsRHo3+QX/rb/5iOf/gNSdaz2cIEilCbSuQ4CIOZgCELEhGymrkb8ohARcGAhIkpEVAm8XGIb4zkdAITxLIugIycgKHV1lpNhJLYmjXUvskJQ4+ubqDmZZkRfBpHIBHJOWUtLU+7xAKG5gZgJAJMrZR1LuDKpTU91VYdBQlrc2oNAQ77XWtW1lqbJxZTXaqTOSZEBBJCcGrRKqEBmLv0wD731tZWWYhFADz2n3kGGgfQBA29LmBluJUZDfyHIQmOAgbDTnZfcfMlrUou+cblxhsX4bbQubWLuoLwWio8HHWBegImMUVFp6bsFbm4LctazS/pdgVsTdqyllZmmbhyU28o++Pl/Pvv//PtPtOv/4Yqntf5cSnL6ed/+9/9x1eMT+vLx5+/y8Pu7d2b46n88Y9/Gpnfzk+Z8pQS3+ZRRLU+n0+Px1M7retyfvfm3eHmFrX3c6ZoZGKk3uPawr8T8VGdMgwrbH87f0bzLXqXqDJ9z0AHS+JIwKJbOJABEXhDIHQ0QgZBcACOv4MdsI9CCQ4hFHTY1kbhFjT95VHhBhgbHDYeeIOWA4ky71LUL/wE2xmC3fgQ3yEZBs/DkPeH27vXb766fPvr09PDw8efHz/8+OnH7x4/vl8v57qUlDKTMYqbqzaWvp14E0xuWE84rTp/ClHkAxePPhAAzaJYGjGDubqFFjwU9nitsNhN6z0FKAToHG1/HGORexiQS5A6vu0+Cdo4TiMnoiBcicjUETXsaYAIQB51AMI70jF57yFim0AJITraz+etxyARU0KcB75hPtCFToDgGk3755tjG4a242U73tUUhQFaqeXx+fjh/tM6rw6D1sJxBdsZVDuOkZJaQyb3SHdCJAFX3DgRjjwvV3BzVyKE1twqQXNTRGAWBzJzNM8gCNjUl2rn6i0eLo5uNXoGNZUkSAxamMARDJWC2UUDRIy9J53rDomubpeR0IGYgQiJExt4ZE+SuzOLq3LKm7NDp/3eiddaiVuihCxulQDJVYiAsTVrULX6Utux3b/hrxCFHA3D0po87e61/VP+1e6NEOFvlp8P66f56UXViCcAKabkJIDWjEUoCUqC6iKkxbQpjdMCMo7jyHKu5toCpMyDxPppt2rWgIjAQFtTWNdiqoEBjJMQJDU1U0QoZrVWAmgh/HVf1pURwEmNLue1lafWrGqYxbJ7l+c1QGI0N0dOOS21gdCQB+ykdDK302luTTNCrQUc2ZRJxv2NrytoaDtRiUArRySmLk7e0OWQnLjUyuBElkbQHdAACJOfj9oq8rx7/Q6JL8u9rU1EZYThK8nflKTr8ukBiejWXVRLQbRybg3UkUP+yg29dWVzNajqVHVkZ2hgVluTgQWx+JnZ8LLevUo0Ur0sZV7qehmH4cPDf/mffk/wsv7ur//mVC7PL4/DkD88Pzyu/t2P/+Pf/dPf/+KXv/7tr/72uz/907kc28vl7vD1V3ffvn51h8fcHq0u7fn89PBy70bf3Lze7V85poAS0N0IyCnUa+pbO419VkeAINF8Y4C3FSLe8zA2V6hDTxjeGkwADJHk5wpMSIHLY6giaBvKNmJRr3Tzta2PYBgLhzBgxBIH0B2kgnVOG+CaU7D5pKkfBkZEtuETvVDHAu9+Etj2HwiUQfI4TeOrm8O71+9+8fLLv/76r/7tw4efPv74/fH9z6fnJ1NzUnRMSdx60nto7QHRENz6cYihtY2DJ1Q06OEMgo2zJewYmqkFJmyRzUC9ke8P2bx70xCIBGEjjCP7KWp/vPRbIx5rhz0aUnOMwgLInXoJhU/PF4kLJ4Bg3jdQBz7U49uuO8poO1K29t79i4Tsfrp6XJx+HMSdciWWgizZaFuIbj3kQOAITsQOqqbzWu6fnz59/LjMi403gI4o4KtbwwjHQFczNWckR9oOXA7VExgRErh+wcR7R+QMvDbETsy6GSFMOSVhdla3c/FTVUcABnOFbdEYsrOwQemubuycPvbEf0fkEAebu6lVVQAgTCxsgBDHMhJzrAyLLaVq0J8QENZaKecIzW/oS1kNSCSHygvCRdKUiAGh1iVMeiPeiDAxOzgoIpESSRqR4dNc/oHvhttfJvSvat3fcoayrOvFAxsjJgVEFwA3a4WHkcx5PNSmPI7Tbt+WCrRO+1emRAmng+wOqQHM2nIeGlRhAQBHm9ei2pKQlwamu3Gn1nypiC5JEo1rxznZU1vL7EZedW2KoJ5LEqGUWmtgLjmzsGpRUyKo6twqouckxMTJkbCVJpguL5daS2C7qi4JL6eXiXBMNLz56umnn7Q1UjKIM5oc1LW5KUAbBnn3619U18eff2rHCyVnyY7KA3A7r7RUKZzcp3V3N+mnFy8zEcoOcefObTgUOs9AvKK106xz4TxOk4AQmQOzKbbV6xKaFfamvjYSZKFhTO7ldLms8AyHmoS0zVNWQCx6VqlGswNj5p8+/qnN607Sb3/5i09PPz4ej7vb/fH9+vj8f/vh/vvf/+Efvnt6+z99//+++frN8fHThz/8+Jvf/K1N+PDp4U8fvlNHWnA5Haf9+Ju/+t2vv/3tzeE1ogSeCIwQuw3Br3HN10Y+KOKgT8CjpPTKsKHSeH1XA26hBNsp8mVfD4BR/nq+QKD9plEMSSh+7BcAEEazGjX6qgsKCeJmJN5+/Gc8gTbDbvy+zUts1HnjLXHgswcawiUO28+B7bMmeRjzYb97/er1V1//4tff/vo3cQw8fnh/enzUVho69gANR2KLEoFhjFe6SmCoQ+Ww1R9i0sgjAjSz2IEe4oTQAGHATV0dA9czNcT4YZJCcArAzawbg2FTBAfH3KPiQn7Uc0bRHUlisxAxuwXlCgGciF2Z46usCMBCFtHD/jAOH9/8X7g9vq5ruoaobwNjUFW9/nc2qbMA9vmTITQOHauBGaKVpdx/erz/dO9dD0tht462HiHSOZ0yS85m1kzNjAnN3LQhIDiqFggzkYXs+cp6d+Y4glYRUMAGIkx8mddzw8XJhMGhaUVngMjIT06O4BrQeSQrwXZPmaGboTmgmjaNFVGMSLBJoGupyNzpblcAICY00FaIUtNGiK4qzMSsDrVdEGkpi6TBYrWAarUK1dyAwAFpzDzlaczCJM3d2FoFR1+9YVV1+GNNVu/wMJx8fEfLrjzDfDw/3t/cDnzIdXkechLJXhZUB5wMKU/fkMwLQhLIMkIzW8/Px3UY+fB62g+yVksABUGExiSOWK1KLF0SMCUZEiFUaLXVLLyfBje0y2VZllKtakOWnAYUZDdTH4bd4XA4LQuiiCR0z0kAcXYlgHEc9jvZDyOBU9NzLeCWRMy0AbRWRciZzItkgcbsPt0dZq2cpS5nxOzaW5VQbZsZtkZMVjXxumNeMjg0243VG7dCbEovhU5DBkgvAJc0lrvfjFBLKStIrY10dRzzuq5wrmgGiSBjc/NmMopWt2rYODJHBMSppZwTiSOYt7Wdm10qz9AWkJIHkT2v86KnZakFEzi1Rb0B3a8f//Ty+//7P8rD8f7lvMAz8Ig//PSHx/t799P9P//47ld/9fWrlx9+/G45n+Q8tTOvTy/16eXm1Vd6nMXS7377H77+xS8OX98icwMFD+apYmJXjS1An8F4JKAudek9eQ8W6x+BU4fWq0MI3fPYy6dvxQK2njvslKF/BO/1lsANzIwBiLxFCYxcX4uqhz1W8jPEEhW9H1ChzDHcXMe+ec2+YC76qXBFh+IRolEEnaHFCNRhpc/pGAAEmEWGw253ePvq9bdfffvXv/zr3316//Onn354/PTz6eHTepmrFmFGa+jQp/0NgkfsaRdqW1SnBxyuvm3qgihLGuMMRN6/bXMV9gjJbb0MEhFqgGPeI9QAnAivAeSB5rs5cainAitzJjY3VWNqkQB1Bd/6BAMoXYLbm1tHi1IdNHR/QuCdBOgIIHbHXMyJ164geG/b9AdxI2waXYeweoWHgEJXDZ0kV0Oi2trz88v9/fF8LkTZTMAYtJJZpInFuZEGSrsdEpZlBQN3VSdQBTcGUFX01tpMsmdC9+xQgMnNiMBaxAOQASKZAGRWAFsbnBcvyk4EKKYNXJnjMFMOH0BrrTPgkQNCvV1yh6j/5uyOCKbNzdWEmFAkNp1uaXxoTZE4Zk9t1UG6ZAV5Wc+Wdmm4NTU1I0FHMa1FHauKGwCNw0Ai5qi1mTs06y44bE310oouTdd1kPRH2AneXt68ehI4XH7K+H7C3QgvFeww7JiVZWjagBSsEg/khXzIsGYttVzwcFPbUpZ1v7+dl/aM5fR0aRnXeaEEVQ1QMGFGujR3x3E/EsJa2nmdzfSQJzDTZmp2mee61FZsOkyYEjGgGbBIzq03KZoSO9gwZGQy8HI+J8ZJJCG5mtZWlkq9z3ZTRYRXt7tV7bguVVutNQu//fqr75+fizYgcg3IHzDe4toAkYTXcvnwz38/3e7SkPOb3Yrrxdf9m5tM57Ie6esmyyzsLmeQJFOVYUBRGLxqKQ2sQnOf9tmtgQEhrGtZV0u7XE7mKxggVkgDJ2aqQJhIRtCESmq6tov5em4z0pzZsrC2Ulup69oumkZCZoOKoxRe/nj/x0/HD7X5cillXShjtcWhTK8HK/i0PH74f7xHtdP54o9/fJLHcUqXdt6V+1pb0jTBm//x5797O39/k95m2b26fXVII6GINvAw3hgxERKxWCD21zEetm7WQsgRtACgGV17TtuIAuzEo3cjZNS+jkxsQTGBu2BHzxWMTD7PE9sHbpX4yuv2NWRkgAANDRCM6LPcsy+FhFgvZ2GCcNjsZNtPp3740IZCbBYEglBCffEQCAASSMrj8Obm5vbN63e/fPfLXz/df/r04/cff/r+8acf23rBzWURwg6EjSBF6Nsl1TDi2TeQBTc5cn+BzZBY1ZAw1MkRawOhue2NazykKDnXSOMNVwmS2IJvC869w8xEXaiOFFBQwNY9BZmZVI0JxdxAA+zb+loPKakDOJJTqMhjNtuima5baXpVC1go0i/7lUfsqbvm3tcHxUhzhf633wldSlXwfFwePr7U2YWGtRhixPfHM5LALXkcc8qtVlDsTH9t5sBM4EYGDkJ5j5jdm2uL05nBHVpfjtZpGx8zs7XaeG566XYBB66xRJA7oOaU8jDeAPJyOYZWFwBAgZOguyoAAzmRKzm6KTqgGWJDyoiGLM0aAxPluq4YXk0ERCbExAkJLawiDWQcUh5MzcyZyRkjC1iroSMKuRIlysRLs1Kqa3VgHsVCeWmqVo/z+e5wg2n/YRCR/SwjmaSavxr3AMdkz1jnyeelgfF+zwNphVIbzGBZJbW0e/Orv3rLr8XkcHAeZS7F1jaNQkJ2q82aN7Na0IGFxjzOZQUzTpHYWxGwNSWAeV1KWVQLi9zt7ygjMCp4AhyG1BB0WcpyIXNrilmU2FWrNjVLyt4MEZa1tmLE1EpRcxYG0jGPGUW5euKidRizMKxtRSJtFUGZUtMCIV/xhqhEDuZaikGFy0vLu/HVPiGulzOkeRicptO096mCrvb627t2XmtSrWcEak2BhfPAyXcj67LMJ2NDR0jESm4vZtVJM5JgivwoV1ZCAYScJoZ9c1+Xk3pRMS/qc3NUh7IsaymapkTCQErClBChLevp/PyMInVpWmEgkcxJJq+ASOtaW1Mmm17n83zW+3JJzpI/PhxVlYrjj/7j0+8zpAzjzfDuV1//9qvDm7/97X/89t036OjWvDUSImJxIO9dNTq5x2pRAFcW7MnGvY6AqXfhPlz5AiBC7yoS6J0doBME3ReFahO4b5GSDbSfJX0S6ZrTDgJdC7JB13pSmJzQIPKDwfoecgJzEwK7SoD+3IT2xQcZXvPp4Tpk9HkevxBAxb8YEssrud1Nh1ev3v7i62//+tOHnz98/8dP3//x+eFDWxYzjdQWdCdh0wh8DPgrTGAQmzP6bzUFAFO/wv0hlieRsGUAgpviVvYxqlCXY6GbozuxRJByVxx1EprCvEu9xkZhDpqS4xd1lRHFjgkwd9nGBQ/3HnSat88g1qDbWreWAK9T4fW1w56m7WrgQEjW1Vfu1ustEai5u3u7sgYx7PXWmBC1+PF5fXi41FUsMSWq60oOLM2drE+KTkQW2H7TLMmbGRhyMnTwGqovZmnWzK0F7AtuUarJwY04Q73kxEk4hESnhnO0REBOaEAQzizqdj4EyGlqfJnt4mG2BCbOAGTewmwCjv2VMguI082iDiAAE5oZOqIpMMT6L5aEMlQtzESJx91oWQCURMAg8i8NWsIwnFMthm45Y05sKReI0dABfUzikaaQ8+HuDhxQZGa5R1+cACeZvjrDznb2cbk/nI9y+ihaD+Pwzu02NaFZCI2gToej7FahYZS7dLt8fMpQb95OaEWJXkohGE4V17KaNVEg4pTyPNeX08V3O2ZEY3ebl7JuF343TNkPY8ov87l45cFHEXV4/9PHaUzC6L4gFsnSbDYHtZr3DA4VEXSpVvPIKefzjK6WSFiaMIuSrkWXwpkBJHMCGCC27Xhl2Jmregs7RUpCaJyFxnl5OsqQx9eUbiFR5cPqdm7cdneQ97KebREjXDn7smp9KQBESD4kQ84jEmo1TUPihq26KqgBFIWloohLc5AGqyIgoXllokSYOCeRpaxmDchLXeplARoko+QskztQYgBiMGuLIzQytqqmLkIIrrVCARpBjQpIaWatndd1PBCJYMgEfAFQZOcJP95/99OPfzLjSdKOdv/4T6//zS9/9+7NL7755dftUufTAt6GYWQRi7WmLN6AmTvlFxvYo45uoZ7W+8KNVHSA2G4RJBxsWqBY/BmA9ZXe9ZAMhq6w4w4dpQ7lu2+ywY5pb1QDbuvlemfZtvg6C4wkACOnDhBZ55n7MXA9AzyohS+a/f7VYBG6NJYiQiFOHQMDMIFRKKfD7mb3+u7V119//etPf/Xbn777w8cf/vT4809aF4yMf4gtXQAEvV4hBocKm8EYACz2y/ZXzomx7zEGNOuZluChjIfNiRXWAgJy34LO+omJ25EJnauxEGACbAbr8N8aQBTPuHRGhAgg4TLol+lK8/ZoCwOgFi4HM1cgjLDXYCs2TDC4/s86IANAifWV3r0Z1+mibw/tWyDQWgNwayaDrHP9eP/y/v68VFLmis0pTHigWoIMUTUtDatja9Y8AgsJBiIBX2oEHg8ZwYWH1iojwlrD5cCCqm5ggLXnnTtLSmpQgIpyawAiJLtWjdCbNlNIoxCNPdkTiFKONYdu6H29DpoqcjgBsrUCAEBELIAMgARESQLpsuqMhMTNCiWWlCxiXsxbqQQ4HW6g1nleJQ1OYmoMEgMCCQuSJEHnqsDCIwyzrgQwSEKzWhWTpAPgDNYMmAvxo9ensk4sE93yMNoAQKIL3O33vjzvzH6J5d9P01s7LY/Pu9e7haZj5YdP89NhOaPe35/znvS01qb7Qx6nDNaGnLRUlhETZaa5NLN40zUHSkKSJrN1rS1J2u1fgZmtCjKDOGHa71CELk2HSQWKz8fl/mfJecCsNp/mmtJk0Jbz2almTm0pw93BKoAWq6WyO+XEEokPuySOIJhwOJxfHtbnC3gF0Gaz2qWUWQYjEobXyI+GJ5pgnKa0h3RncqilzV4qpEo50IOCAIl8Pc9mUN0MIAtSFko5CWTB9XSqywqYrJBWqk6O7G5KKff3aezNcANQx0zggooty5AM1lrdNaVcfS3VXFAS5TG14pREW7OijuzYEBtJQrVWLULT12qXlSiJoyGTERrAcMi7fU45vfrFuJ6X5blIQhZcX7yoToc8ZtbTuWD2CWoql/m0HC9Pn55e3d6Mw9BaU2vDmCGWEBMQcNizAEyrUSLcdrgK8ZbvAgHWx3KmHpYeGUS9+sBnRMlto2I6m7w5d3vPiyFo9whW2yLsHMLl2Vf5eOxu8bDmbNgDAHDEEIFBrLyhHlv9Z3LQz/6yHicXM8c1j2YbEDZfHHS+87rWhgiMSXg37na3h1dv3n3zVx9+/f37P/7zxx/+8PzxJy2rc+w5boyChk4GyFdQJcSX1xGnl2HvKI2pecf7HcKjhxH2YRs50dcDIPZz08wcPTbbIFIvy9sLB4RMqL0Fj8jNiGzqiJI1FWEx63KZgIqu4s5YYhAnq3ZjFCkYbjw9AAU0495VoAa4EdORhhb7BwKjCjlZN3rEfgszBCFbCwK1oi8v84enx+fzpfEA4SVnNIWqZmBaTjQMPI2YB7PWzq22BoldizGSAaMbEo3ZEJEBERMOaM2Lu9ZWCgQUSQTeGJ0dx2EgTuaITs1AiZJM5gOiK2m0FOhYVwdFaxUMySENI0lqLVbONFcDb0iJUNyjuwd3QxJwBiBGchBvYLWQqxuqFmZIOYMBIqXhti6fvBRdlonZiB0IkNM02LIsa0Hh1hASjLsxk5hZsSZO48Ctkhqg6ZQnRJzLiswgbGBGsHhbrDIQIo1DXiV9QFXcn2l3oFV2r0WXRxkawt/y7X68yXkyQkIdBc7z8qLlONedJE2N0QypNHO3l9NCgE7ESZbWamtE7Ig0jK7O4tZa0waupg7OJGKuD+dnRZF0WNqSLbuvVj6u9cGefqgvz/LqLWFSP6wLsCCBpwTDQNlhOGRFR1akgrycji/Tq1eIvpRHZk6+tkaV/P7+59xe1M7jHuaXUutcuOg4E16MavUfJA2vvtrzNFi+haEMB9J2Aj+t5ThORIzNoJycDc2AkVwVzHiCfJA8TUOakLy+vKyPtZi5FWpskFGSS660QHLMSAg5Z2verJopIDElc1BtNJAk97WCG5gnGcq6eni7IzjFqBWARsAODOBUwb2aKWpVRKrVwCyxpInyJOJ4uBnevNuPO7g8z/PDiQ2huitqJlUlMqI23fHLiktZ6DV/d/+H+8en+f6cHH797a+N9NWrN0zi2IoakbSmRJYpA2AphYXKWok4Z+6GsI4FhH4FAuY2Ate+eQ0xDL3aWz8k4h6SG39xOx4AiCPfDq/og0UF8W45Agffttr7lRS2wCJC6QfmwNwFNr3GbFtvoGcWXc3DQf8GkP259H+Gm7rMCGwbC+L0sC8NZeRAh93radzf3r569+6bT7/61Q///A/vv/vnpw8/gxsTowEzdd27heJRYyAgBCcEC9l+mK7VLcLmu+W3ZzWFJtMdwWhznIX0hsKEHHwJIkS+wEbzQj9s4xRAjeGsM9J9qnIIxtilr6RBbGqIm9zTrbVYiBAxMm6uwtQ943E09QsZpwVa32aGTZ3QiYInwNB6WlNAIA43RHdOtFaICBLrUluzp+PL+48Pl6ViGg2z8AFsNWhEUKuDDMPtQdKEeTjff2zU0g5Tyuh1WWZOGRB2hz1Sms+X0lpOA3lCf3FsRDTsb5G41drmB7RGAPsxoSo4rLWVWrWxNsRsRiaUHZNSQ9VWL+qsckBScGNhYlbENA1EyRxQ1FsDd/UGPbvc3VytEaI7MhKamZO2hu6UMrgyj8yDQlVX10sedq0WbbWVddgfUNLaKpClaZcbQF0oZ3VTazmPrc3Hy2V32I1pGvP48nxR0hnOQMRJainuxmCMamalrm4qkk7cFm91Lgl9RoKc92mX0u0J2h9Pp9Tob27v4LCD+rAbx2E3+pp4t7sZlEBzqpnNvJ5WlTQByjyXQbxaW2tbS8nM+eZmWWZUGzNbqcOIhBldW5lL02L1cpmnw521s66f5nYxX8r5j7B+j5cXTo5yLuuM69dfHX61LOfSUNhfvbrBdTWAj0/PWpq32bRlrAMthsvLp+cdj+B1ni8y3oKX9emPss67t6PC41I/ph0nN2tVaVYru7u7fMPjfl8AChcWRygA/iqPhKqtVlVJ4ohE3rQZNCIdRmHGcRKoSzm388tlObcglj0hS6YxlaWZ4zjINI1jGoVkLWW+mLuBpJRSdsFCklBglJWKVhdIiUmH0qoBmMG6NptVm6HDeEiSBkCwEv5PYSB1b6ZMSJmYaT/tayssZO6X+3k5l8efFxaadjtPos++KibegeHxoaWclnX5z//w//rT73+/Xhpc0v/yt3+72AkHyIdxzDv1psXEQSRRFUImBEnSai1WwavwTpIAYTAAPVKGANzQI84HNwdZz3/0DuI4qHnYxyJs369IjG1Syauc6BqXvJF/3WF6zcHcgiSg2/UBzNQxelaICHyCaN6IDACNeqDedafalpzaf9qXdrHtw694SJ8DujLJN3Ap0ci7NObd4fbV63ffvvvVv/3hD//48Y9/OB8fAdEw4u4JHQhJPQK9CQAMFXvUdgeFfNPd4qbLCqV5CJywT1YRz/N5wUDkEbl3Ljx6/ABvoINtWzwdAvQQWYyrEMejmYv2KCXrO8PckFC3oKzAU0spRABEGzKH4AoAiGyh5glZK3TEyRG0hcchOHErTcchNeuXDgCatbhMoEbMbakPDy/v7x/MVfJQGhBVcjWriC5DxiR5GJC9nD9kXOXglDMA7Hi82eGiMB32nO9eHo7eKrEQDF4WKg1a4STDfiJKpfhSX9pyAp0RcMp7dbWmpXoFxyRVjRmER9CGyNXmlBLC2c3dOSE4Uq2NxoEkO4A2a1WRUgvLhRmzACG4AhoKxPIyAgCrHIwDBY2lgG7EpgjuBCjCrdW12mE61DJ7qaDNgfMwAhs6OXnzZXZZl4s1dbXICxr32d3Wog6aB5Eh1VqQqLXSKqjhUtrEHDl2S113g4zTgazu8u3NyJfy9Aza3kyOb3bf/PLnn5bnFX5+eD4J++4mU2JYKas3a4CnCw7WsqRGdpkvIKSGowwTNa6VsaZJ2uVMdRFiAxWQ5mDLMVM73A0Fl2W51NP3Uz5zKrD8J9IP/Jp3r26Wy8/ry/vD4X+FZeeXF62QdvvL88ckSSu0crL5makmYKfi1S8P96DWWvJaAY3lRucfbPmHLMMvfvcffsbvwOZ02F3mM1MZBiq1Dd+2Zvdzu28R+FBgHJiJWkVtTYtiSloRk9S2mBkzRBzfJOTz3Fabq80rmLAIMJBM+yGNZlihpZvdNI1vbve8wGWtIAnhLAA5D69vDzt/pQXVKrrV4utS2B0HljRZjTBuBXVtqs0lAQELMYBXsDAdceKcIHtGIwY6jNMgCUHXZZnP1WtRMwBB59a8mV+eVyBId2lIo6kyDbudvP/hY11/srok3r3+ZsBnuC239cP8y3f/ZmntdHzUdX138+5m92YYJuEECJdlLq3eHm66DwlAzcnMo7ohceLWNIBuMwRrsUoGupal6zkjHiwW9vX9Zr412b7pioLfxf6Nm2QoFixtZdk3ublDB42C5v9cLrscqRPWvbcmB9TQ31sA9bSRAQSwRUdsZEAcDNczoH/yz/YQBDPCJHJ7O+x2N3ev3n317hc//fKvv/unv//005/m4zFlTsKoRkgI4lehjSN2m5q7aWBESByFGgEI2aKCM7l23sKtD11BLgKia2yUQuu7DfppcjVxw8YdUxD4RJ+fTVhoEaSZhUbdrWGfdcDMQlIKEdOOKiTAfc9A90AjaXd6OJiFV6+pxkPsu8MQtCmAN/OmBgDIqKatNc4cuF9bm5mVeX3/8f7h6RNIZV6oGpO6FyIdph2zGMxsz261Pn/af/V6OPzCGeaXT7wsy7zc/OqbfNitzwu0c6IGiKAvoCevT4izjPvbt7dW0K1erAAUAhCAceRh4uWiKyHuM7cB0t06Exm5urY17gxAQ1YkcSRdZpSJWNTUqtWiJBLZA+4tgqCJ0ByJmGVStarOWNAU+5C3ct6pG3ll4vPzs5Ck/WBay+XSzJ9Ol1Mp6+l4dzM4qDQAK4pcliouF0ckzRMBKAHNZeEMIgmKl9W0lkFEZHItmAF2w7quWfD162lg1OqSBq8n4ZVZm7w8lnVtc/768APMgz9T290P9cImed5nf2mlLtawgjbzlXa5arNLFRYCEiRFc2u2FBoSeZ2waX2pL58yNrIKvqhZJaknl0x8+Mbm7+ZP/yXBJ52s1qd1/aMtH+7271Cs6rxWxwUJV2+gS235bj2fcZxaOeHyCZYL7RgMEg+2PsxPH6a73Th+Y8sJCdR+D/iztQ9leKU07/br8not2HgP082Q2VWn508/v3q9c1/dBFZzl9KIEJrpWtUTsTBjUofpZjRtUBo4cEJ3VVNjqoutzYaUQzEJam2d56qYp3GaJNN+Pwm1x+NxNiQBabAbeMqJGjZY1OXyclpO84s+7OCQMnmu6tqqVVNtDhVBNQ2DAHjVMMTJjslxGNMgXFYVlGm/m/Kw6Kx1mU+ny1KJSKsmTGlMZcVVa13o7iv53W9v0G9eXub7xxcidCY55HEYUsqP9jPM5t/ZuOT//f/u25fTd7//+//01au3+92Il/zm7itgXNb1NJ9kEBmEiHsiW2Ax6EAOBKrKQt4aoAAYMQUr3O1O3ElK78IQAEQD4A7gb+eAbaCFdxTDr0EDhJt16Yt0Cr/u7O3px3SlQkMVj9f8yYDeOdzN4YXYOOTP1fy/xoL+7PMhFb0ut9kcUrD9eZCDvBp3u9vbN++++vbXP/7pn7///T88ffzBdTHDBivFowdUdRYGDFIgFPXOyPHU1RQRNRD1QKwZwXzbBRqnZQ/5974nEjdXX7imQtqLDmDaD4ZOpG/BDQBOTIjoalJKSSxEYGoiZGaCFKdK7M02UEJA7oYLJOxhnmAApNa826oM1UwdBWu12Roz1KWBWRJBBDXXUtKYam1alYVC3lCWkjMs6+XD/U/PLx9hVCJGBee1eiGgJoqS3M7gq9kKk9nBcKySvj4//3gi/ev/zb+pmc8fn0/Lonoi8rUWIpJUqa2lrsPuBmWHi5XLzMymRmy7YZiEJMnFyglkgYycEn5b7MPSjtCK1oVGKY6c8jTtWjMtlVOGlHjYoTthkyl8LepIptjKwoiI4oBAOYxbcPkvln7FzEbU1guBWjUe8u2rN1Y00d7dAEopl2G6TfmlYUrDONBwl4iciz7N9V4bE3Kiw91+37Sul4vRsDYYx+xWa23LurSmOQ+MjVygWR6JmGWQYoauiM6y1vqwlE8NXoo9X9rKDM9zfWjT82H38Yx/d/5HmOcTp0/8tIwnyO98gXVWuTvsb1PKdJa6vCwA2YxRmldtD8/DLdd1AVpoqrD+bPRDWR+anaCd14o1vTLbk9wY1v0t15cf2/rDejrp5dn0hUei3ObjPZLnPc3nP4zjxUlwOCK+qbavx+pVvbaUNDE6IeedrpYPZ6AM43F/91qfPiIDpxP9gqA+L5d/mG7nu1GeXh4Od3ecFapOYxqm1+vTc9P66m6vYmUpPI48Ynk5NaOUklC22piYAXaTn4u7o62ogspeS7XVdoIpYa3aDHQpDDRMw/4wqDk1Wy/zIAKwzIvuBUAQmEorSddSaikzE4uhrLfGWBZz9Uqtuqo1rT4wE4gXX7E5qCTKhzweUltNRrDaiq1A2SkXpnWZX07n5bzGhD+QMDthwUT73VBTvR1dH2bJ43qup6cZMiLVYco5o2M51Yflcvn9//Off3Xz7dtf/9UPf/r7n3784W9+/bfjbvqP//7XBlpqPZ6OH9///Lv/7m8lCRi4mlrzcG66p9iAq0qYjDAyImJLmMceCLCQh4SqMAoubkUz5PA9C9567e/Z9hC++40J2BrPyF1A3/r/ALPD6eHdfmQhK4JubIo5gtDAGNxiFQZQrFcn/LLkd407IH0OyN8mhH5aeHw2yIguXd3oDJCU92/eDYfD7d3rd6/effvz97//8Q//+Xj/oc6aBgn2jjk6ZkDoexdiOd3m+Sd15/4164rzK/a1MchxQDo49h0CwXNEqlLohBDgizXE5sHZbfJO7K8PoTioKQFCbW2phYgGy24Wxommqt6EGAHWsgKiMAtTU3UER21al2U1w2nIaNrMd3l8nk8Ppyciqpc5Ob/96l3Ok5qWUhtoWRoHS1nbZbks8wIkL5eHDy9/WvkTHUaHlSdCo0kMlAxPRZWgSobdgHpjOL4ogp4/vpyOt9/cLlNaPz2eHp9aWRTWZgUowcToK8ZE47Uun5bj2ersWhB9kJTTwM7lYi+FXtahteScUMNZ1zgBkKQxl3AVELI3REpJFFiXGUDBmISYWN3IzYGImciQYeShqhk1aA7wJnNGGRwUx4paqs0pixgawN3u64rPRU88IMvF1p8Or6eE5n4WP5XlUs6nqpd11nzzOpMkK2bPvjytL1Sm2+HwZsjTdPta8f2+1UHvcW1thUx5SHcgePNmuJwr1TOU07K+L8v7Wj6dLj+iPo435F5PavWxnk9v79L+9WFM+/k427neyPQx+y+b8rD7Znc77V6nuT6W48cJ1Juk9NqwMVwQHnB+adYAnxmtvPyhzd/D/JRSAWiUbxjuySdedv54U9Pa1h/IjgMtvm/53Z2peV1IKKEjO98AwgdUdD2KnzIMPEBRqGgEyu6UUn5z0OfK68mW0/xx9Ne3wDQmFFbHZXm+vHy8pDHJDe1scL1A4URyc/Pq6em9DzzKtHgxc94PTbAt69o0jZwHIUUEYcJa15fjwgRInJGbQWnaqiIjMStaczWHokrEgglbI3IyWmd84OY0gJYKKrsJkFdb11lzm7ASM796/aY+GnJ1b1paAVN0YMrCu5GgwWVuL/PKCYfEh4l4zBVbAyva1rUoa32uxR2sCSXJQ3ZMObHQkNi0TDcDc348l+OnM63C+bK4loYEnji3pVkzYBQTp6VVeJlf/of/9H99Oj7s8vT++eOr5+c0jZCkzXo8PcEoNCRKUs5FOHJszJrmlENdAoRKYOZrmZtZznmUDMSgSiRhhlfT1io5C4mwRDo9AJjbJswMbpnAwjyECD16OcYHcATo8v7P7C1tWlIw/Ow+AnUAVe9S0B5YFAQBOnsDQ40ML7smj8KXyaObTGgTC0FIRbuoaMOO/owojhMCEXgcb+mrPB1ubt++ffX2Fz/+8Z8+/PGf1uNDYwf1sAT0xfSOgVxFIEK47dgw2OxrHlwQIFcgLPRSVxBM3XkL/WfmgIagiz2BECNwwbpJowNr4KjqBC5C0OYVGl6Wy/3Dw7BLO5nGw6DNeKNV0kjqdrycTsfj7avbXZ6sOTFdyqW2EkKl4/HBobx98+Z0aT/+/KdPTw+Pj/evb+9eH97c6S0VWqzNpxNk3I2DAjw9zV7bkBO4revl/dNPj8v3tJuHt6yoftIk1NZaTpe1ftqNvwRMw9vbcYTTw9Lmn3hatNrN2zTdrnr8Yb5/ouNzHqlNaslQwLmYXfLob/YHvbzM89HN1suSKO7R/y9Zf9ZcyZJlaWJ7UlUzO+cA8OGOMWTkUFlZ1UwWJ+l+aTaFIvwH/NEUYZPCqsrKyIiMvPPg1yc4cAYzU9U98MEOPKJIvMUNdzjgAlfdutda38KBBgE5ntc3j/rh4ksAjoXhDcPiYCzy7PmL1VfCBKZRm60zIYIMGOytBmc3Ek95mgRAW49trgkmSELPwlfHBlEZG8ax5C/Mu3BqldA753Sp70l7pgceh8A23qW1vode9P6bvH9hy9z6L0v9oLoC9MQO/Z2tJ5cz6ype29Lr5WdbP5HnXzTqevo52gX6Qhih5mhOn4a/qC5eF9Uz0n2tr1r9ZZlftflnyawNeUfgjutc1dPfjpUefvjj10uPHjzSK6RPAF/c7f9+P5A9uq09t3eDhGtlGwLd+8XjIlbRj6jvgGuhx7R7nOO0Xpby7KY805gfL2/fk9yo3pvVMs16PK1LHQ45CRjF8bgIUx4SssvIFFWrDSWjN4JazwogN4epm2lfMej8+hgiLI6+wlr7oHybCAHBzLXHom3OeedHHW6yILa5EvbjQ6uturu6Q2DHKMLEWKsr4pRKJuxzMBRXv/RWO4rEeMBVFTCcoIMPU4nO2iO2fxo9FFR7PTvQkAicuOA4HfKBxhVajfDu3vtaoKCUHQ+1tsT55tlh6UcufFkrhHNhSZKBAtzQZC8ZwsIkbY9xSFl6a7WHNu9goyBzcEpFUrnJutiyrNZgrZ4SzkcLX4FhKHe3z18CSV9mx4vPjQfql4ojBZGUGAKB5f704fjtIsMwH+eF/He/6qfH5fDs+bGd3zy8zZlbbQ07Ebk1ZwT0Dn1IAo6t9TzlDnpeL99/89X9+4e/+/u/+/zF57lMhgRhqFsrDVxnT6ANPO3bCXftI3PYusS2KddxK4zZqGp+pQls8/9/v6TxjUMTALCtoyHAr2+APxv+NwzBNeq89Xlshnpgul4h1xT0E1nIP278P2rEH9VgeHIK4Udf0oatfkogA0CWSQ45lfHm9vbZJ5/cPf/k1bd/fPfqG0cDZOHrHsu2vwNE5O31siGZLRye8Ml/3jVtKV/4qIZvCWCPKxGaaAObbyz8eKoViydxABn/fCVs14lGEAiJUwJVr60d59Nyf9kP5ctf/8oDel0Z5fmzOxLu2s/z+efXP5J82VMdy84dHk733dvusK+1fvfTt9OUbl8cfn7z3Tevv/vu+2+X8+Xf//0/3L140dFqP17W9cOb+93NgHzXzdfzgr0fbvenh8cZLt//8scZH+lWIc3rpZYXewiVpTtJyb8DjTKMwVR9jqLr42weWfYRcH71k7nlMlluskvujtlwbJfjcSd+98VufDG9+eo0HUrHpTz0MLXWyu7Zs7sXw17npZ0IanZCUFy8nwxlVU+H2yaVRT797X+4//b3qKuutddOozoITrsyPa/zOZP3+x/lMKFQikQsfb44ebe3CgKggBeKJcBRlsyGFDkJJ2ZZhb0uH+61Z76LnIdxQDqCuV7eUTyj7sSg9irtlPRcLzO0hPtjrT/Hquy3BTHUBCgjY7T64StqH5waU6BXEtd24tT0glbn2k9Gb6u+qvY+4p6KQwoUCtLhhhSH9dhe/fhzq+d1OZ8el3Gf7KIsD+Xw2Dx6u92BzMsP9vhzzzTlPrK09tDXOuQ8PPtNf/ylx4dlPcrowF0F6O6mA/QPJypcnhXrZx6mwNrrGhI0Fi7YbFlW70SBLI55oGGficKV1tW16lJ9WaxMuWrrqq1ZmEEAVO/uw8s7RIFo0y3ApWrHTlE+PRQkt2DIQBCMGoYAthpuPmsic3cjGkopQu4jJxborXWLcDLzag4kPAAQTndDHtJyqYwmktDdWyeghEFTqt2CUCHADAnJYuDd7c3zm0O/HB+W9WJ9CW2LOiLsJE+73fF8QWahFOxlKrZCSTnC3MJAzV0KJoSBuWTWtRNi3qeMvJxrbeEAB8bdftoETV3mwIho3kLN1yX2N0UhsIy7u2fPX/x2FFrefMcJl8sKK5UhY5YgCe+X42KKutQUPO0POA4Bl+O63B/fCpQfX/30+6/+8Fe/+506KAUBtnlefZ4v8+r2q199gerg0FUvy/wvX//hn/7L/3uk/OVffdlqTwUCwMxga8SKIKJMiZBj87NDQOCmFG5WFUB62qc4bR70TYS96rVXuBo9rTae7gCADS50PZM3o71fxeHNmfJ0gto2/TJAbKK0O9jmDnInAv5zIuwvtlV/ebj/pTXo43boY74prv9piy5LyVN6XnLZ7ffPbl5+sv/q5evv/m1+eGcI4UqO11UNEYRfqVVb0ctmD0e+suKu4z4GOBO5P9U2IuHHPszYoKKIAJuRZyNRe8RVjw/YVAGgLbUGkthMpbUmnFy16frqzY+n+QO7g3TJ5fH4eLPf7w9TMam6qi/d2w8//fD5F5+lXTqeL1//+NV0mz35199/8+MP37x4eQPcvnv9409vX331/dc3N3cP7XE3P07t9nS+f3w81uV8aqnDedjvv/npm9uxON799OHVd2+/+ur979twHgdEihhEvbZj7ctahpG4oXC5lTLtL2/eQq9O5wBRRPzik/an10Cw/8fPj1/PwCt7l/0UQ6/D480wcTm/f/WBb4qH6n1XaYgABnzLS1nOe14LRDUsrHXmEtA6wLBL+eXnhaTvXj5TfoTdCeZzviMujEN0SPvnn8i4G45L9oDnw+W05nEYdxnNT29bQzf03f4OJNqi4DDtMqeHUhIiNuBLnHd/9amcXrW8zuc3nmrZDTEWoMsS9wUxln1bWteKB3Y4LfU9ZpC0U+29RT+G8E15/sWAnwCr+dtkfbptAmbLh7jcr/2o7vScUslidjq/7/WxHGouVYjK8MzrpbZa63k991oRLbeqaqt3I+Hnv71jZkQ/n9/Pp/U4Pz7297fl5fl01ni8vflM5Hx8+GmtH6BMlA487Jb3v8z17NLsrF4t7cp4t6uXtbdaTz2olXFcl2O3br0DYBYkj7VGbWEgIiIFMbweGzFyztq8Va8t1h62tGNVAC2FAQSRwd17wA5pH7BaO6svXldjYgFnCgtyD07cFILoPGsWyom9KYoFomSJCA5F4lSkW63da+vawjqA2HQr00gMOuWcUuGUl24R0NoSCgwgSTihUW+uDAjmFDQmylTEJZQCkho1M7cG4eGhNNyWiUuBdRFMDhbIQ8pkGx5FzAwxmFMmAlAKPh+7GHUHHnG8yefLyoH7u/3z3bA0bbVqj16dEGVkrQroi5u57wsLwro8EA3gjj2SxzTydBiB+XzRZVFGH5PQ3X6Xh3G3ozxczvrNmz/hf0k35ebd/fu3r1/xLn3x4Vcz3B3vj8fTL25tnhco8mZ+dzmd/vrXvwWEf/vuD//6r7+fz4//8e//ExHMbRYdBHjrXiaClGQjDoFvlif/qGfilvJCfMKhbYaXLXXJH1mTCGLoTyf7U151MyRuGatNFo6r2IngERQYmzH1aYGEANfKGYpQ29yaiI5IQuRyHa2vY/71hvnvMEF//ojrUQ9B/z9G0o14RgggQPvdXS7DeNjd3Nwd7l7++K//fP/mO61KTEkEwpAwzDz8Y0AMn4qc44kwhEgeBlfV9ymBjRta4goeJd5k7s0aG/DUFhlbANhtw+5dv3gINyMkOR0vZRgAkDMo9J/e/nz/y4806aef//rt47vzPO2mgfHLc50pU7ktD2/fvoTntc3f/vinf/rDPz3//CA/ffenb/+4LpcKn0bWN5dfvn311UkfRYbvfv4WJZ8vHx4eXjftQ84EUJ7/VW39/vyT4+2xvf7+1df/+tPvv3/7w5kajNAvHQlt7b1ZnS+ASkzI4/q+jqd7tw7Z93gb2lF6e/Xjqg/PP3/pDw/TzsDX8db3n95d9CGhXt7+CIdn403RsH5cYui+6xk4Z4HnVp/Z/TM7ni7LsxWUd2mXMyQaUYZ0m47ffbXf78tnqf30/bL+a/Z09/nLPI6PDuMkN9NS0nLWN205GcD0xeHF331ZP5z08bG05cNp3h1e3D47VHvEUTG36cZjeQ+9CQGNBnlw/wPuTraeMV3GmyHhCbpGaB6K1Yr8CGjTDe1f3i2nR94FjsWbzQ9vw6EuZvzgx1O5/dBXAN33ZUD06Eew9+ZHTgFIy+nHpZ7P96c+H4HWyZ0FYECRYnXW3qxbPa8k7Np0NksZAMmCcjhE094NJPnl/ffL+1/eBO92d33pp4d7x0X7SbJrW/p8f37zrRtYptptqfXmbsr7or12Xda1bcNUm1cCdfVunnLSaiuCBgZyKkIES+u4HX+J7bxqjwheq5X93lx7rTkxUGIEGaQvjQd27/Do4NosvMFaLRdwcBKIEGRui3WL2lTBkpTqNSRcDFRKGRLE/LgQEZZxXqoHmsXlvKREY5FpLIkjc4oOl9a8UaveKrQFzJFJQoTQp7F0a2Dh4WZYxnIzTtB0WS/Hx/t5vWi0LMJZeo+Ans3c1axpaGgEkzv1umqvu+mzmzIpLcu6xEjEeDzXXgNbxfMCI0rBaSregSxqU6trn7Wtah2IMcDKoQTyUluvXrnVvry/r7MM63LMCZ3y7e3LJHRZl8vDufWYBv7k189evHhJFm1t74/LvMyv77/5/us/cQBoHG7uvnqd4PfBufz403ePp9cpx1huhv30y08/I9o/Pv6DteWX1+8e3t9/+evfjM/2x37Z+13OyS3Cka8wFSLmbey3J5QxbZRk34ZsJ3jShiEQQN1JrqP/duAhYFyNRPAX+IMNTUABttWOXW+Gq+sfr6jrePp92/+7bUu2dwcBAJIEgth18N/OTbl6hJ7IEPDRDwofT/wnrNlHLyn9xf98qskqMvAt5TTk6TDubn/6+vmb779aHj/0cFTjeGKxRbAkhM2M6h6OyAhPHti45mx9k4y3Jf9TgVdcLVK0cZGuCvnGc7rW7SJABIGbBdCWTSJC+eXVzy/unq+mmH26HWmk79/9mP9I/+n54fXDm6Okw2HMVO6e3bZ1ndtyf3r/efv89Ob8x6//5ef77941fpzf//z2l2d3t/3x/MPlh/PxfP/hXe9tft363VqON//tX/5X0zrt0zSMaRj8l348zx9ev9lfpulF/tfT73+cv6njypnTlEnow+v3yCmxyy052HC7G29vd/uxP8yXswtykZHWCMGl1xfPP/31b78oaTx9+HCeH6Jfkt3ffro7jc/q7oCOp+Opr5dhmNodHigLcS4YUZfd+S1a2yntaGx4sxf3LqYpRfVj2s+Hl3tb3/b11fPnOh1e1N5qOpf981Ganr4GiXHSdvl69/w34wE4vW/+IKml9LDb17tPD8P0BtvM3NOB0X9q67HPr/nwaRJ8OPUxDoorxsyBiTzsInbeM5RpBztejo8GNS7Lo/1ISZAG0nCC8I5AAdZP98DV1iOkaCZl92y425Oduj8eXqbIWWss5w+pRD7UVt9w8m5hgRHc5jO5S+Yk5fY2t6aQcI4+z0sqqV/aUiGAAIkHEWiIPs/HxNxavf3yUB9/bla1N6zQZhiGnYOtLVLKvM93z0shhh4ETpDKCNs+1s2bhnYFglhrGbMCYpaM7KZE1KsDIAueT01XkJJs8wozJorx5S4RI6I7tGoGCER2DTyKQThHYNRqxAWdgCmQfftpRxiGbKDEweKBSBxEpt2RWIGs1m4OxKv5cEO7/ciMyFRX6+FAphZdyTr06m3twIHIbkQE5CpEAYyODDykDFGPl8ub9x+Oj++bXsbdRDwMw8hT5piitlXPy3JmSYGups1bP65SxiHvx11aDC6q6oaBc4WUEiWO6KDOmaQQZe61zo+rLqvaBl1DSkLBIgOPYoi6rtbs8cOZwsPcwSTx/tnOHcJwPa2xNGJBo3CxCkHDu+P58f3DcDOs9fRw/w47DPthSvD1t//5h5/+bfH1dP/e+jLsp8/++rcfvr788u23L14eDH8Z0/Ts5tPfvvirkva19T7Xw69vAMDNNiYlAfD2VxUY4IQYGITERFe6j8PmXNxWKe5oW8W8B9CGEIBr+hdi+4zu8MSLQNgW6VuCFZ5g9Ff0/LW1jK5LIL/W2W6Lc3iy1BO52/ZmAEAi3q6XraMYCP683vko/17vg7+c/DcuKQT9Wad40qhRoPCUPv+s7Ibd/vZmOhx+/vZfP/zyw/YM4qdbzSIIOcA2+2o8kSncN86d45UEgYjbPmcLvF1vvIhwu3qH4onWCrDp49u3jrDJ0A5EiEQCvHqcrYaZ7XZl92wHxV5/eP1f//n/s5znLDLk9Ozwicx8P9//2/f/9v13X3EZzvPxD9//86yXD7+8+emXn8/r+rfjbx5/eZBR5g+X9f0pEMYDdV1+evXPP3z3tRd6ESOfJe3G7x7+NF/6fP94czfGD+s5Lpc4d9Pp8EyEtXUZUmB69sXOe69LLbtCArfJ14Lao3krwtPzASXzBcdpejEMBWK62y/t/ri83U8vveeM2MNb9+l2WnmGaNPzVO5Ee9/fTazqMT/2amohNH5yePjwIZc83o55IK+Yano83w9lyAcyHms9Hn73edMMMtn5bbMP3SyPlO6eUaLI9uH+VXM16JEu+0Pu85tLZCqeBpEi0LpMmPKXrtaAb58NiaF1Onz6XEryVQWzzeSgXdvNy0NK+fywzucwQBFOiLVprbVpvbk7lN1gOsxzM1h97Ugp1ugfzoTrbidYzL1iST7Xx+PP7kY3wOxT4QBYV+1qjBCuIoxJwkC7djTaixvwyBaYZdC1gUa9VBrzUFIp2WrHHbbjqoQwFK1x+GQo01Dn1mNpvY2BN2UiQDM9Hi/dmDINuWtH9WgegMwYnMDdJEli793CwwicMU8ZDH1xknDrSZgYmDwnYTFmb4t3xMiMwSiAgAZupm64RW4ZHDA8nGIrhEBEHKZMELV3zpwzrs0dvPbFKwQlQA5tYEECZZ/Hwjc3xS3qsiyXHiYQGkQAoobejQyZIKFHM8fAhIAEjEA8UDZbXr/9/uHhclz6sswQXcbOkVm1SCrI5pfaz8tyERHKqQcwpqAx4ZRl9/z5541O/d0Psz1yibWqZCwDqwaQCkIWkoHauS+Xup4dAUrKw5SnXalNtdtqZu7AANGXtbtZRLDIzVTKYQyP0/Fo555zAmLs/d2Pj/Oh4X54eDiCNj53SnTz2ehrPP/krtzIm29+vtyvPZwTT2P26D98/Z0FEFTHYbH1+fNP/+Zv/i7vb77+w9ePx9N//NsbzGza3F1VMyQhDiAN21IyWz6AabN/hrkJUWy2zIDNxumxeUo33PpmM//zDuY6+8ITC3MjlG3CwFMz2cdulo3Cfu03/Ji9JXw6NiEiGDbfqT9dJPHElA4ER39anPw5OOYfC4evW6A/C8X/3en/FzeEE/BQxvTyMx7KuL8Z71789Ke79z991dcHAwjrTAmCPCyeLK30VCq5xa2YeTPEPjlPaftL2zDTGrZBX8yMmTZMx0c1GAE4sXlA+NZuv63P5Nzn04/3X/7ub3755dWb5X2jNe15epF+ePOHFlZXix2Uu5fP17s/vf/TV6+//vanP72rrx7m+6BQaG8eX7++f1sGeTzfE5C+WaH58+H2+c1z3h2mQ/rpw6sf335z+PTF8vaopuNNal0B0Oblp/e1ZKYdSioYab1fCVwJ0yARUS+n3YjpkOf5XCis482taMhce5ooC6y2WGppHHqc3f2nD+/O6zunAGqXy4fLcVaNVCaPLpOMuz1C9KWSo6RAsHltBn2tNUn2xw8llSTe27Kezq1Zr73kkQbgQ4kCtkZfzq2FxXHItHu5q6djsPGQVtXT+9fIECzBzs8IEy3v58yAyRGtzSulGG6LV6ynVVvbHdhVg4DQQVuQnecTuTEzOZ3uZ47emjWj5AKdA3Bt3pZK5rq2zz47vL/MGaVZUPB8vAzs4VIEe23rrABCWW4+2cN5OT2u3lQGdrXera7WW3eHnDCIIhDBwigxyjgMQ0H0830lQObSrdnaCUUyqXdJ+PDqYVlMDTLzLtM4Ag/WGjc1YspM8/GM7tW8K0Q4hfXVmoMCEedcNImCXkWp9dI8gFlsVRQGhba2cWBCQKJdTqoOEhYQgbWaKYApMghROLjZ1g2XiKRQIsKA+bJaUAslRhbJhctB6mnhoCwEHhRb+y0S+TLPoA5E4z6XqVCSMTGGuWld23xSIcw5hDlYsANzgtQAVwCvXRMTITsLBJAkEfGArrrWBoFMQARoDXSofXGMkMBwi46DLm1hm7TbNGRUck6l3OzKJ15hlJ32D4xxe5vdncHMNWUMxUBcF+1LM3VGSANN+zyVkYukMgX4xVqf12EshODaras6AImF67w4yVINJhmA67LKlPKuJMm1uzu4mnIQkaMgxfFyKVHTQWRNAFH2JdQrWPfoGlSGdBgN2XN6mB/e//zt+XSZhmnW5d3j2zGlBImCShZ3V9deW0rCJHKl9mx6bgiRhktsYLE/T/CwFR/itUkK4+oAQg/fIk0UtMVmt43L1lbwpBA8dWv5k3D73+/vn7Yk10fAVS7Ga4EfBQCZBxKTEwKFWwA+dRRvxiWPv/yUT2uiLWb2RJrbbLIf/3ACACbZ7+9Eci7TOBzKuPv5T/95PT8GACcmwiuEAZ9A0AHhQUTMVwxPuCOSu19bKbd6DCICNDMAYOGIYLjS/zd3UUSYOjEFyyazA4Cby5v7V5fHx0taf/zpx2/f/dxkfTi9e+hvyk2a9vnxcv/TvbR/vkzlcD+//3B6mPH++//234Y9TbspT8O0lxcv9mHUz52ECQUBbu9e/h//3X+699Or05vXv/zIgPZ4mrstSzt8MkzPJmDiQqdTc5EJc++OGOY955JHkh7rZe1uzWQacD/m2uovP33wgPFmL1kc+6XXWnUcy+n0y3IiC1+aNu3DuG/hdW1mlofSa6NMXCQIAbBHW3svw8AJ0Zm8JOAAlCzMQojT89vlfG4PFxKkjNqbzdXCCDQhpSkLUUkU2nFACOgRlhQRdG3jninjs2efrn3Z7XJbFhkwtHvvwkOYe8BirWu3hxoSMuzGwjr39XypvUvhwsmcvNZFbQXDSThTBNTVu4UGchLnuP9w9IjAMPUwHW8mFhByQKtN26rdcZjGCm5dW18isDWz5l3dgxCFEKLR0rxbT4XLmIZdKWkgAK/NuzVtANxWlYkYPBq22mFM3tCVIAIZUuJ16fWxA6SUMyY0Qu1oi64tUIQgHi992AnnjAmoGRhgEkqk1Wp3a42Y0yhlNwCEaVCQ5BIWDFFy6a3VS+sGuWBrFhECVMqIgefHs/VwAxpoS9AzoUNotXl1mgjVh0QpMYlb72CuDRAMw0kIHXtTAmrd0g2l3bi7u6HT7KZr7740WyMugqXwKMAQgUNKJVIP2t8ekPG8Lpd5BiVTo4TQGwjPy7meanT1GrBaPsjAsp4vvllgZGCWshurzoNwq4FVz3PXCl++fPHJ3RdefZkv2mtbak44lmlZLr31vrhI6lV9dSzQWjSFMuTdbhqmoo5eWyqILAmTUhORItTNi+fWMIKJWFtX701bkZQYw6Uc9oe724mG+/P75eG0qgJnocIAGLqeF6/ZwGRIhAkzUIZ2qgZsHhpUO3by14+v37z7YZn7YTx8Qb/6+uc/fg/fjyx/+8W//+TmEyRuzcj6Nia7O7HgtfzR3R0JkMDBHcy7kQgAQQQ62FZUvuVan5w3cX0BbGks/PhwiOuiH/+iqz08ADzw2ldDT9Lxli/YljwYsTn68QoqBgANIkZydAMSIHkyi2JsjViw4ZOfjELXr2sb+f0Kk7t21l8vjKs7yMHJhWiadkRU8lDKLsn48ze/P93/HCAWBlekjxNKXCEXcO0Z3ribAQhBxIjgVyTTJn0HMSKQuyOhg8PTo+FJNYYwfwrPXSn38s//8s95Et3hq4cf3z78dMH1vDy0U9tbefG7T+zH+nB6/faXn7V6upHlcl7nxfDSVFK18VlME5weUKTU4wK5MCEitYLfrj+2Xr//5UfXnjOhuCQpjCmlCIDQYB/2WcGqa84UQL25u8GpuZlZbE0Uc63RvRtIGW6fHRA3Ba3aWhHAzsuyKhBKFgDIScJjVa9g+8PIIpg1ugKB9YUR8yhpZJbIpciUlvOMTECSUwJw77Wej+s8194QxcCsV9UGxJTZoYM7S1YLsL79mKDpMA4aNr4YJEHqhG35JN+8Xn8pO+nLmcIhvDdEEieHghsokwHJmq6IDmpuDYCZBFAVHTwASNDJOqjFWqshAgQzIrFus7wCMI7jjhP03nQ1GgCQgBOAAUFOoCh5ZDBiJgrNjN0wALR1EM5FmCiPKSdJhGZ1mX29VAsH4rQr44sp3BOBdnVk7RgMAVBX5eB73ZqPMrHvXgy2+untJZgIC7FTwmEEixpu5I2beY9AsCYh7MLR3awT2dacG7IhEj1xrJfmGh8MIEkL6U3DmnpLOaUhZ4lWOyVsrQORG6lCI/Mc2s2BUEwSu0dQRNhy6ubhFjFv72eQbIFo4EYBCdIgBOi9l5Qe59pWQwMMnm4SweBtG5VapKhqKe8+f/mblOjn92+svekM4pqEQG0DzwCAzS2cejeqkccEBACm2IP7UAqKGQynhyOnEkWW13Muz3/zq7/961/99tX9w8PPx1N7j6AEPO54ujt8OC7zuj4+KJqPI5VBkpDsYiy7kidC+XD6sNZ5GNOQByxRsri5ruoWgJwcQVLrroa1r6HuhM44HAY170utQ6+1AmnZw3BbDoebwoBAVttyXnTl3qkcmFwxB6LrsjGs6Xg+L1+dOCEx7MZ9Hse358f7y5/W0+VvPvnr57sXh7ITSltSKYnMtTERB+U0AIBqg3BXCKC6rmubcy7oMe73AdTns6oNQxlyQRJwALkewQB/NrtcPf7bmujaSHb9RdclPxKCh4ejX1NkT9bS61F4zUYBusNftJx8xNtsax2iK9o9Np4xAUBc4WjbIXzdC10tSh/riP+8E3rShwOcgYY85mcpSR6HYdxP3/3hPz+8/wkg+MpOvZpXr3w9R6Aw96tH1iAIttIhRAhHdyPi8EBGIHA3iBAWN8MrmX9z0dLWXWwWiMjM8vVP/5IP5Vzqu+O3r49vzvViZPN5Hu/Sz1+96TavS9Sl96VhDRmIJxzLyMwKdl4vl7fdnUwWGXC+XMp+yCy/vP3hzetvVb2Jma75WS4HaSDP9vv57bH1NRQggkZKRJyId1gfbG7NT4urSkppyLmkjT5UvWlX5xHQCQOwu9feNSfiQgm5d5fENy+eebNlXbrpbpQkZL2FqfZVZLz7ZBpFtNt8WVTXtbkTODglYebelpy4TGXrJ0DycT8Ji2kMh9tNfScK7733FUg8vJtzIplEBCQjswNA89U9zetFo2tfEdQjIA3nuY4pJZab29tSxpRTqDEHFV77qgDdVE9qnTVDGXLKRSj1Veval7W11gOJU8pJmMF6X9a26FZRhaFh5t28L8EpuEhqDNbnUw9mJCQDaE0hzEIdTAEBm2owJmHEEIHe6zq3SzUI3u+ncTcyITI4hOlKKBniclxqVUciQrwldxAha86Ckmu9BAqGh3uTjLtBVnMiYiRoPQhYBBOmlIPIqgKhIQ+lTM8Kepg27xHgy2WNENf47OWLTz/75Jtvf3zfHlMZdmVHFCJk7nkq5TAc+YJK7ri02ptZMydHxGGfuZCkco39Z1F1YrUKrUHKSQ2ZScM9XEZMFOB1Xbr16M2WhyZJhiHlmyEs2gVMW1urlYGlgBTMzyShpFOz5kRlYvLeTaETChkjDgI9GNglmjkOCcENesOmsohgCVk0a+MUlCQ+//w3X7789TTswt+arQ7t5sUexda1CnK5KXLytmpKSQoTUkmQx1zyzhqtqg/nc6DdDi9ePn8uI86X+8vpfD7P2kxEOEnKsVxarc3CwKAMDIJB4ev6eF9PwkZ+eL4vU7q92yEitt5b5Ux3z4a3b+a2qC8u5u2irfvaO4YJJGsOoZdTpyLMZdX1guvx3c/1bJ9/8ul9exiW8np5u6ztcLgliHU9nh9Pd3d3z29egsfleASkLz798tzaH/7pv6Lb3//D/1DSlMxb6Nv37z98ePUPf/cP0zButUvkV8ZcXI0wT5nWeMoLBCKgxxUWjRuO/uPbYevhe9oIEW6tBR8LrjAAwy2u8SrYslgRQeKE7OYEtNU/wrV3d5Non4IBfq16jGtu7ckt9KQTbJuhTRjYrJlE+XB3l3KSknMZv//6n+9ffRPaFRq6wdWxv1VLBWJs6gTANeK2kaY31h0C01P+C6/f4vW94upAyMQRjlecdTDxVm8rx/oTkLz74w/N6mWtra3lMNxMU20Xfetrt/IsN3a54fGGXVdo1qqZCaW0vlMGBvMADSSUMO/V6nzRVruwpJEpkTL3GVngfP8QAN0sDRkRJW14pB5rRACTAGgqg2SUIbNgKtzWaKsBhnk7Pt7nJAGoQJBSXBF3knOExvn4gCjhRuC+9k5EWfKOzG1Zqr36kCUBkaqbe4Q7OEveEkDWzWqXLCkxBA7TkBMxRaIhl4zhbuYRtDVvqqFvP1juIXWurs7OrZt2T1kRoK5NEgknQTBLAKqKg5Rh+vVIr/N025bj8fTgS23qrZmqSpg3VErshYkTs3sYQd6VNDkLSCoEbj1qM+tKTNb7fDJOvHVstMXp7DxUQNSOQRvi0KFZmF7fikQirGrWlEboTd0EPcCx68b74iyc2E19XaouyompYF+NgsokeUjAAB4oYDWghVp/+BEtNBDcFAKkY4tkidyCyMPDVsp7YCevHoy6OiW52U9SUC/GQrW6VbXVJCcPJKIMgqve3u6PxxM77PNoXlftRAyBhWiaBlu6m2jW1ZsREqMQApHkxES8Ydw63d6W9YJrBFio0zTlXjUcpQyJyJrp0oDZWuvdhYSQORVO4tkFIJQgyDKWqYS1+/s3ksvpMnNKCM2aWvQQcQ5Ep0wxBqcglWE/pJwCsWkN9e5tsSg5uTgVMrUsw8sXt588+2zg4Xic17YQhwTKbmCsy2LrGWIiIkmgOdFGWSdngYTK8+Vc2YZdKWWXcu66tnNba22mrRsgWJB363PVcICQ7coPAozedF56BIrG/m6kRIymy9p7v5waGu1vcit8unQFVJe0EhADGbYohJIhsQG7JFK0djk/vBeLXldjzD+fvi8P6bvTn5hRqwfIMJb5PJ/v33169+Xz6cVlXo9v3v/qi1/zfv/N7//l//X//H/8j/+n/8nUZZ8fT8fH8/H7b7/97MuXQsmRwK+IyVDb+h63tcdGVEaka63g09zsvoFJn0rZt18UsJ2g28G/seqBnurHNr9kPAnHYddYMThoBDsCAjEAEcnTLaII8hdU0adp/6PtiK6rqqcXwEcR2QF4k7mZaNzvP//y16Xkcb/7Ng+vf/hqnTsjC5G7h9vG2b5ukxyQ0AAgnIIwABx4a4NxuN5PeGXMWQR7IF+ttIgcboRXMN12YUhDXe7fxjAguCvkMYcGuJtCrQBI68nYqLAk7V3BOwtyWw16oNO6AhIBEaKPt9MmRXhLaUp9aUSchhF2Y+9VgXKWIckN34Q6uDmaWjVwvThRKiUFMg0siZFEGPvSW/O0n7wurdaSSvNAiB6OIH1pzd2FOXMmaWpg6tBdNedxzBMmDF0ACoJbZ09Dq21pnYVSyimLu5uZq7k5eI9mEZxKCgeCoAhXXdsC4N1JPSyQUW5307LMdZ3NcoLI4zDd5tYaQ2hza+7uqj6MJbGg2lIXZBKG2i7+7g8IMPczEJn2a9NTtejOE1PmQPaI9bKuBK0aArGImZvHkBO4z/NlngEhSRhJYUxBICX66lG9ngwyYHLKiCic0CC8KmFgcERgIgRCwlIEE21tz2abIxoHGjlR19beLdq7I1FHTySO1lGVd/spF/Bu1Vtbra9AGcdMgmhLdm+SU2K+ui2qRXMoFAZXWgkjBdS5AWLaXtQCxGG9g0EEy46F0J3Q8cPj+1WXS59TYVY7L0dttW2FewAfzIBIiErOh+EGeyq7Ioih5qrMoNabWaiRp0sQ6hYLEkYhlmlfiFGSAAQY1jqDhSCHIGQpeTdOGQA9Ok7OKgMBsERd6rq8mc+ekopS6KYdUp7SMHBA7S0gZMzMnCUn5DyUqg0urXZtfSZJRgkEDY2JEOlwePbs7lZbXas2XR2Mc9G1pz1JwvOxtQu6YsoZGbR1igBTtHBdUHgahjLumFHYluXcaq2t9e7EXMYMANpUuwZQHoskFwPrtp4thE1gXfrI2LVJ4l7rfF5VqVZIJMvalwc3p+kw5UOKauG6zxx57CcVYdNOIvmG2kKmtC51/uk1JxKRf/7jm+9++C/TIZeSx0Ek75bFTIlWfPf2DfH+7av37b7+X/6X6fmr7//wx3/KQ/7i8y8+/fSTedF/+dd/nfW0n3a/+vKvch4gPCy2HyAADPOnSponY/zT/mUz03tsVWybNX7bySBAXKXia3XWla6J/tRZANeZf7NLejg8ha3cFa72IwGWzZ60lSGSAxBtDAkCCKKn496v5/xf2kT/nCegp+UOBAQBljK++OQzTimn3TDe/fz1H9fTO4+GBiy8LbKeOnHdti0QYIQjBiHB9hW5AZK5ESMgBRhtAgBgYAA4RkCEhV0xeQERISFA+72HefMIDINwb92BGYcsmbUFe2D1tYcTIbF7cEqn+4UBPWGQ7HLK4yiciKxkH0ZZzv34cFbjYNR57eHOMdBwc7jLazP2x/XR2QACUbgwBfiqEB6GMkpvWldjliDQ6N1t2A1lLAFsVqlZhKEkU916dszDjLVXIvQgThn2kx2P53PFlMcpLXOrazfXiEDgAHbbetGuiNemyh7IE4UTi4W33qN21QgIRwFhACZKK9Nc28VjR0g9ZIhdpsNw93C6gFXrjSR4KsFsYW6GBGjW1lUoupp2hwtTIk6pGejcAHw65LIfWCTYvVurzRQsQCSgzt0ileLgvFVfZJKcMsK427UlXLRHA4iRYR2idQxGUMgDIhi758QEAd21RYDzNJSBRQiI1LyuPZecE2rrUkpvNp97KWxm7KSLI4JW782GQ6EUVqO3ZoheUQYenvFIuHxYGaGMwpIYuXU3NWtuDqhBmc3RNPTciWkT9cx6qPfOPARoRA/Z+IQALADmDw8PDx9OwV5G6eH1XB3AFBGoN9emaeCcBKAC9oTwye3NYSzny/ru/Yda59ieAqMgcBisq3qEMMcVTADTVNSsNu21a4+InjNIYmDCbJIDAs3JFcCNicI0AEJotTMEu0dCQyamLJRki+hsrdtCjDhkEUhtbc1r16ah3tfA0l1565D2IErDOKQs5jUVytnGEa2DVe/CSQTR12UZhuJuujpsTagGocGSdmMmph6tLstFa+g2fyAn5MyAMe0LeJnXFsERihjBaH1zTbpkGoAZo/W1qUWAV9NuqoAp5mMA4f5wuPv8bj0fMY8hQjpjqWQABCxCHHYOcqJ8o9pJvNa6zHOYnY736T5Slpub8e7lJ29+eOQYLq/nm7svPvvy3/3ub//dun98+eLlfDrf3H767NmL5y8+NcfHx+NXX/3pxSc3f/cf/9N+3BNn1U4bxpk5fCvEAggEjrAr3Nk3o/tWreubz/NaWgkAHlu3N0X4UyXitiW67n6uZV9XORkdHDyAAgPdHMCfih2D3Z0EaUtSXWNiW5bsGgre2sf+Qgz4uPd/+o8UTyoxAgAwApAgEt49e1FSKWmQSD9+9c+n4yt+gl9sFxoSbw+IQA+/OpeuircHILkHEW2rHnzqB4vtNfNklN2+cXdDIoiQeWl4S31Ra6GqSEDIjhTIPIm6VeoZSZHdqZ8xD8IZeo/hxQABRJSHKRcWFnWty4JGxMg5Tc8PrXUHCK3ebEglen34/ickWFQ5eWJhRGSz1bptiz2cpkTgJUWrFsLjbTqdOoukPOSSIbT1sAZddbEYxpIKobv16OZu0aoChI6d4/Gi7bzUkRkR1br1hoBEOE5MDl1bqLs7J+SMGAkMJQsQAWFb6lo7dNem4ZB3lFg26aLNizPevXzBgc9uxzZfzsdT16MGYQow5cxG2Fpl4rIvpF5n7dbdzLqpBeeMamttKAAUFFimoRB7mIUBRZhbD3fwcLOgnAKgLnXI+e7TOwisl/WyrjvJTrXXVd1YZHxeYmY/d97lUkgXC3VUYASw6D3ajKRgpR9uUu9uam6xrmtHGyznLJRQEOUS9dRAMTQkCwa02QOQJ7eu0cMANBwYppssrOeHFoHllsdCvlqfdb3oJkkxCgRoR0fQ2hCIEo47IQskDgJrTatDDwgfpoQYPcDWRtcyCcWIOndAYBFhXsza2r25sAgzMDdXWz0zeVu94Lwe5/m0LHUYU94xsqBhX6xfFDMHbK9n6wHEYN7NtbVeOwhAMAQ7YVPF8wqSGIPdTNem5gyGKckonIr2KuhDEQTW1dqq1qkMo24XPIURj5TN6DQfj/MFAYgDiT1AVd3ADIjTWIb9zX6Y0v5mMFiHgF2XfqZgrOfmgL0bEwBAIu4G7iSMqoaIOYt5b0ut3lqr5oBAJJiFAbxrnXYHIbbwzLjOVSFIMMiwYHQHpAhnDHBzjUBTBUJmuXIZMILRY10fvv/RxCnt9s9u8VAu97Pl3teaSYSIOUOIqzZVWyqKWXhv9XDIgYQ09Iv88uFheWixLtFvv/jdv/+//S//95vBf/q3H764+2JIu5f/4TfTzWHYP+8Ol3a5e377u9/+7fO7l4lL05WDECmu8/Qm4+PGBtpopPBx7N6wOQToVz4+ErnHx45iCgS+zvpPxV4bjw7DYANGbLSJrTyAtgBxRNgVEGEYgEGGwURBgA4oBE9dvI4OiPT0nvCnlc9TqSX4VTj+mA/Aq2cImCTJIDcZf5PYM1H58d/+63x63WNFVWZxCDW7puFic/4E/Rnzf/34WJ2z5cK2D3oqI3uKhV01YUSQ8NbeW4+wZgGul0jDmMep9a6XHujuVhdFkS2N7Q7jMPDOL0cHpZtnh2HcCaS1nZfT4m0dhxt1LiUn7L23UGeAIpgBDyyz1ft3J5N0+3xkJHDVqqawKR6C4XMnFoeAkOjGjSXIGJmSt968m1ozbc080FQJ0DXW2YBZhFJmU7eqj2+sNsVgVHMNkUiY3IEwYulBhAHRXMiZxIOAyzhlzhThl0trFaxeX4UY1OfelpZKMdEkcnv7Io8ZejOI2nk9z5JkCyWljKbRups5MmHGciim3JbFnGLtwWRdg596wxDzmCRHq7X1DgnHsfiQPDo0A6Iy8PYXssw9DBPn5ze3b2rrrZ2Whar7GuARIzZmJh/GZECuER7r2RjdFMLRiDyj7IEz9LU1DbNwCyQQye5gpuupqRqoC3KZIO2TGdZVowEStg+GHD3C1cMUQPp58RfETEtTmjlahw5tCTVj4WHKyBwRRlGbQscwCyNbG2cB6AGoPUIdzCIFGAtANwv3xESZx6m0dTXAPEhKuCzdAObuBSEXALDerAO0RW/2w3JctFnT2puGAVR1gM4WIdqVEpCEsDClVjsqEAWyY4Qw0Z7CXT2a0TAmVUAnwZJZer30vmgHloKZiQWpJwwRkozWqVmb55VQSlcZEwD23rvp6bKk0DDszVk4TwUzFSTAQAPvJpKGoTy7vd2PE0kcT/en4+vWLstaLdQ9QiAI717ejnkCwOWyzqcToQ9DEUZO3K0v9bL0gABmQHEUCmjrBcrI48DWo62+rB0TJJFwRUIRVuStXjjURFjNzLaCLicmFgILDOtLD0/jMI5lJJChoRcrhWokbd4uQGUgEFde3ZbWMQWRh2nr0ACH6S6XZ9Sgaw/uh0/v/sNv/9P//D//X//68xfz8fgPv/vdZ/tPh7I3JgsjgOP5fH58/Jvf/vbXn/9WQEJRgAiRCTd2jW/nr+BWFebhm4n/6by7LnOejsZNL0WI2Pr6Ajfb6DWDhg7xVG6/Jaq3SXnr4t3IDuixWYYcAsAR+YkPSsgSARjqwExPyivQnzmkBE8k0W2Chyff0tNi6EkeIKIAYJFw2B1uv/wtAwmY//CNHR9+ZhIgCDPfyM+EhNeqeo+tWmeTO5Domv7d1A9CdNzKJhE/VuRsXwbzpoIICPoa6gYI5Iw9IupcvbWedokcWYTBoRkVadBKSswMY6ZzkylD+OXDu3Bcw1vXaSi3N3dLmy/nY7NLuJGTaYQB0HqeuxpxHggFqjdXd1LDp5wDETmYrYsZAJWEgHNVSRlC27oYxNqqqvceaiHB3XsLrd1TKYlgGCQ4vHMoegQnSeDWVIZhSsN4I/O8ruf1Mq+9m0eMQ9lP4h517a2CByQTHrOHDeOu45oyeLjOtsy1rhYcwwQpDaG2HBcsVM8VnGXYMwcgYN1QfGDa6gooHCOs5y4Z4xy6as5pKCkcDVyKOGi7KIbjal2dM6ecAoALsEIg5yGViTGgL14SCgFGP18+mLVusGMa7/ZTlzZXAxzygAiLt6Fgm/u6NjOwDhBAjMhYRp5uJQmFmlu3FjykErvdOPX1bJfaq4YTMe32pYwCYJdqvcYw5TJNp4fTelojCNjLjiQxcKQaLYAC26yG6Oppl8chI6cXX76ItXezakrSLDE0ZQRBBwt0b4u7siQGRiRmAOtu2pEJhYhY3TGlRFGG5KYyJEafCKAqCSK4WbiiB7jhfFm5rrVWVShZhDm6gyJyCDOn9OnnL8Oxdzt9eFxqVa1bNy0LIQEgShJwRMXdMNQZalvTMEIDxkEGJmQKDFUL28Drvdmy9HWNrhjaVZW75THllFiEkciojLjTodaNCZOIAsOBNTnbuYOYWb0sD+d1uT///O789rQ8YGK9rEaQ8/hXv/vyr37zV+3cv/32u8uqCL71LQ47qr01ba2HOZQE404AyQlCfbpjRq5Vo8bjsQZFoaRkTMEkAMxMy3kxVRYUBMlZwlmIGBEZhbFZWGCi3bB/Pt0iRmtuHvWhWW3glDGFoECGyK4qwtNQUkYZ1HrdTXF4eVNkiMbny4UTr53+wz/+D//bf/zfT7f87Ztvfvj2m9+8/Dyg7/zF7d1zCmmtzfP65v7hH/7276ZphyCmXRK6RXdFQIetW3GTetHi2neF8XGnjoGx8eaevJ2B11xxIF0b7WOzCUXE1hS8GfU3fOhHLvRWtbUV7CJsT4EIADD0bZ5nCApwIkbYbJm0AeGfEgDb/eEBQgRBAXBtIkPw6z1xlQ384yWBBGUQocPn8IWpKnb7qtX5g/lCCERbfSxtuvZ2afm13AavN+HmbfUAwo2hEQARzoHuEeFboGyrjgF3kTG5kF7MwTABhJ3nStlLTlvODhUScGQ6VnNOY0oMYJf5cCg5jUyyQH98WJoGi3eBN6dfwqC2FZkFwN3MybouDiIJmCKTIHbrDhAEEWQenNC6b/J7a32z0XJKKQkQEaNVtwg1BBEKZ7dhl7F77RDAxFQyRdPmHsSugR7IcDlXweAypZRO92u3qEYGQglECCjmtUE4gqRE3r2aYoPePbiLFIwAAcoteQ90JLTuJtZ6t256DhECB+udE1vvhKHqtrgDKjgPPJ97QARvq7toBje7wiRtbq5OJQFHWxuUlPMwPp8IoM8rIGxDeWJJALVqVyspTwOZqrVmveeBEGK6E2g5F1qr9t5scV29XyqCZwHJWHu0DlgwseQkFESUGzRtK2YRKZRYz6e2NiJmBI0ujGawzOFI87lN05gHmedaNaikMhIhphzgVtXXxdwpzGQQCM+7vL8bYAHrKdd8Wtd5PQP4tMsGBagQAoN76HppQRYc7jTspjJl19pajZQQQzImpk2/okByyqkMTMOIda26dFBDStwsReSpIPnx8YxIKDEkFAYwDYcIooBhSKWMQyQq8uby0LW7qjdHBCAfJpEhA0BddTm3yzFa6oy5q9ZR0yD7ux0h1dUCCLFtynkzhwgKBIHduNfweqmhyN1TJkTQphqQhabDELyGYZgjhFkPCwzgMXfob+df0iLNzqs/Pi738+UUDMOuuPYy8G8+e/l82H+4nC+Py/l4BvbIHsHzuXeLuXUHSAk4IzBShFaHIGQyg8XQVnVzCPZwimt09om7QKFYDiWPjD16C+sRipQQEVGRKd2+3N29fMHL7nR8u+jJus6+zosKkODI4Jj5dFxb1f3dizGVVhciEsjlMN3lT1Z9PM+n9fiAgvnm9v3Dj//67e6Pun73ww9g/fXpbw68/9Vnf/u/G/7H/XCzdn083rv1Ugpdw7fh7qHhETl9ZLSBBURcAargjixXB8/1HA0g+HO1/Laj3363f3R94vWIvg7tTyM6gEVcUwTXUPFHR2dsidwrXMivF4dfD/yt9J2dth7ea/grCAA0ngJjsWGFnJ628U+/f2MbbS8EdxSebvZf/PpL9dna+suP/7LOBmDwtPS6lifHtStm0zwwrnXA4Q5PlQMfF14RAQS02RevobhAQlne9h6KJeWbpIuCRBnzzWfPcsJ1WfvqyIEEEDYgceKMHr0vVksa0pRsPdW5rbV17ANm9mi9a2umMYxEsn1zERqtuieSAkwMCcfxoK3mwsc3s4drsFbdzgbzEALrVluboigzJQRAd+SUCYJYh1HGfc6cLsd1rZGYrJlpq82QhYhAgYVSFtCQHBgtENQcEPOY0bivtbsRY4RyhpRKeFhTCzcFlh45FKF2CICyI2ZaVrXApV00wsPQyTTMgjJK9sM+97mr+7I6FMxJulpymVdj3JiziEjrSQm7JHYnWwKBO8SYU5nGBGRqrZkTEOI4ZiHo3btbt8jJIapbrGtbuyJnCl0eHwkdE2D3eqnzw9pnywT753naMwkuFmvrlLMgYQ8I6g4BjJRyKjmjtrW2VT1a1ZyojGXal8xy/2Zd1uoiutq529LMyTM5YaSE3VQvqhiB6BGYkRJRglBdzwuAJEnvH9+eT3OwCUMcK1CmlABQtSGiZXLXQpJzYQb36qCYUZCFcEjiPepajWAYipSchIgC1BQDMiJyq2rqWxUMuIOgORIAb/9QEIMwOoQbTImF5vViq11Op8vlQokcARxSAnPztUIgMUnO6AEGLbSGG1gepmHco9VLvfQeqOHeAyLnlJnNY0wpHyYcy+X+ZK2LAEK4Vw2M5l1hGPNYSjfbTJxhrt3AKYnN7eFyfAAATGY6W/SyTwBo0DG8jEJgr3758etvf/lw/75rQwMgSLusrmvvZpBHFsFUwNy0g3UkIK29rppKmfZlPwzuTqC5QKjZak2t15A04JTLPk+78A5xir6oo9u8uKdwSrt4OY3RdKHzY31/OT9a9TmgGybId/ucEj++O+9efmZL/eQ3/5vk8v7tDzmvWo/a2/uHb1rX+bICaZjZY/8F1+Prf2t1nc9V0uH9w3s2+j/8o//N7/4jgBDCq8e3w64AQ/VmTqY9SWIWsG1I9riqs1eDJ16R0Zu9mcId3DczPgZes8FbtcD1QN8IENcrAP2aHYAnvs9TfQA+eTbpmh0GeErh+mYjVULa1tbusdGgiACcgQAotkJM4id63FPe7GkxFP+dQ+hqJH2KDkCAs/DN3e0X/Xe2qKm9+fH3tZ5MZ8YrBPsj3eJ6l0RcW8NwI0RgRGyaMADS9k7wDQ56bcLceuFlmRdKLOI+qxDvnh/WpWcgbI3dt/YeEiYgJptuGVp7PLXORNiW4z2FIkGZMikfhvLi2TMGf/t4AlQgZIJqVZhwwDp3NEYHpMBIZRiiq1YvJWkABpYxIUM3T4hJOABab6pOwWYqmUPNqwMDBuWU0Kj2LaNnBMCJAiUzafNwA4tyMyQr4GhV19m7WVdDjACSnIZpiLDeluWo0sW0O2o3oMiCA1z1IHLVIPaGEcDC5ZCZmcysRSymSyVgkZwpMfOK1oFI2DxyJlDsi44pI5g2pMyEZN3mS5NMMmQgCrNETMQ5cVvqstSleSo8TkPOGA61GpJLRlU/NQs336wwrG51niupo4BQQiQCzwRSgJAyp4bIZIcbVnBXRcV17j57kDOLufVVraupowGBixRTWU7mhSgze5oOk2Rc5zVPFBG+tN6hNW89iJAYtEYaAjMBm63qcygWGsll0XVlQSkkDAFu3uvSwOBaryHoCjwy8ma53QacEEZm7lUBsAX01tNQmjnGFVvX1gYWGTmMdDHJuMF4OEsSZCJEDzAO7mbeIgDUbb7MEADo2ioJBDgnIEYz6BcjcgagQizMmVUZloam0DHqpVIH96bNA1nIV2Agnc3cqFApJTtToOdcQ8Hdmpk5BIahaUOzcczuYasyQRiAAjJhor52jd7bzB6JsOwZLLpF6wEIpvD1d28f3h/v35/apeUdAbokQImEAgyHnCHCVNulI0NbKUEKcwJMRkKUcnlxd/v+/qE10GbMSdUCIO/zuBt6XaK3qAnDShEEqnMHRQygJEL58rj6Ho7LsnbrzbuBB1qNPOL4XGK1Z58/e/niNz/8cjqMtyMN8+VUzz+sevHHKBME4rhPRENAa3VZjh92L2+e3R7SVALy+/evpnH/ob37+vXvp3j+q89+e7gdW+v3x3fbuTXkHGgFBiHZFvVu18X51V1zhTwAEdlWnhVXtihcCclP47572HVmpoinIrHYXg50rWTcpnJDxPiopV5neb+ezE9dNhC2/TgFXaNVBAhIgNuhj0BIV+MzXFnUjh/3PPCxgfLpFvh4+n+seMEh3Tx79uWvf9dqXU/Hd2+/gWjXWuIIJN5WVE+7qdhKYJDA47rtJ4DwYKYtjHPVR65XBREDAIiqAZpVuH05llwCINDXy6zWmRkJmQpDOFoELKcGJC4pPCi8z43A98+mdlrdiCkPAfeXRcKH/ejeL7X21Qhpd5imAr2qg5qa27J8AJY07NL4YjifLt1snJJqI+KUsiqaW1ObzyeAyJLM1FY1NxcuKbfmx4cz5zLc5P0uj0IP9zMIAiJnAY9UGImgu5uDgRBKJodoVTl5RHQPdPAQ4rSxMwB43EuiYpU1/O7ZnWKe53leFgt2VxkJQMGUATCsO+IwUDhn6WuPgB6wqPLAyQx5s6s5gGZOkWI4cDisJxVEX0OjU8YwY8Ek0NZ57XWZNYKImAyiew9XdVPYTDVL7eaeCxymZMTdDAIUI3EJ5jR4HAoNFoBNoz7WyJgGYlOIDhoACVFUHYnCg5NkAUMEJK8ug5SbETteThfjQMMXL+5owPl8AbVyk5bjYpsRIUANEIINJEMqwOyq4RpmQIyMZNZRMBfe7ZOrmhGYqTvnJMPIDAS+IAGwr6rWtp/dlIkLcWyFBGpukpMwR9Nz612bI5kDKAIBsBC7m7tFMOaMQoRESAhArg6GHohh5/ctoGLCVBIPw0hAjJuXzqpqDXQId2sAEohBJSQQgUTIm1euzClljm6EgE7Yo1eVRLJLEd7qItEzYhC7OqNgRASqudZGRCbEwfPSiEiYCRkCtYe6BQYxgIYx9tUioiMEcR6z7Mrx4fjhw2m9NCYmxpQpIsKiqcsgw5CsW2gQWO8OHtv3TMTjmMs43U03l+MZETpG706OvEtD5pw4Sde5tVOHtaVBJDFY2Kq9exHKhTjgcl4fTkuLqJdOMAQYCqWMtzd7B2ttnQ67y3l9/uz5vrywuobrUpegcAxJwzCgRL9cVqXY/hWrBokMAy+9HV7sz8flu1/+zXsc+LP/8+GAtLx/fP319//1i89+d5duP/3000+ffZk4rgwdczBHQqBthMArF/oJFRpwHY3NnDc0PsSGxLnuXZ5U0I8R4vhojnFAdHg63fHaw+XXI/ZabX8d45/aCMIByMNg45gAhgW4EyNeF1ZPxv9t4n9KLVznfvizCvz//0GIAHkqd5+8/FJ1nddm/nj/TeiMYExsDkDXWnmACAvkTfGGqxN0kzi278gD4Sm1HLA9UbcNv4x7aVvcwDHc1lXPp7lvIf7EKRUawCC0WdkXc9RgSMytTsnPS2Up0pUD0LTj+s0vD73r4fbAB5zfW6vNzCkFIecMRF6raL8AFAe6O+zcVvSg8LEka1pyLlnUeteotS/Vck4k5LVDhFkQyzRNxKRNc0lAhBXXpZ3mOYjyIadEqoGEeUitqivUVTHAE+xuiySebkYWNEcPt0A048ySaCut291mTvnh1RqQWnDOYx7wVNc6W5LEHnpRZiez7eeGhbGUYAaB6W5v8wUujsiIAegaHhHuXLE7OVDOY+rLxZwwoaliB2YgBu8NCFA9EwEhep9PDhiQJUgMDNRTYrZkveYspaTmbB5UeNN5PBzJeSTMpBbWTU31BBhM2SCCGR2RCBEcwlMRBCwDAxpxRBaHFB3n8wUwyIOZhcnB+rxi4rZ2c8eRmTHAoEcqkAtGJ4IIhdCIwDQKM5prr+vhk93hJve5qppbdOvuEMaHfUIDtIiRF3WraooAmBKJCLpZdeubUoZk2Ofq3bSrBQGGEyGKbnnmp59wYbKIaAoBmNgBXMEaQDgTm/lWDSUBsv2FQ4SHteg13IgBPcg1EIASBbmzitAwMCBhYXAURA/wbl49FHwF67ZSzSOG9jYTCoADuvVwwQSJe7fNmZk4uzOB9gqGjkKSCHpHAMnMeQTcojxq7t6UmLDCellbVXMjCiEKiwAGj76qu3MSQpIsvVqtFlsQga6FB2nMw0Dz+bK6hRBLmi9rGZN4xKrzw8WbghMEW6Mw9AG6++pOCIbu3qsaRhiRuiG5JBkPOQy00c1uH6Sn5lLh8GzqNDyezjbPy/Fhs7Yxg9Zqrs1s6b698NLAAfThfuVEfW0OkYby/vx+WddM38dwvrvdv331+sPbN7Mu//63/5ge7vfT3VSGAHR3CA9CgqANRnJl8IBHkCNc+dJbumKzOG6Onuvm41ossHkwtzRvbJ7/bZMP7r59UiTcEDt/uV+BAA/fgmK4uSo3dRUcN5vnNZ+w8QmecKdPnxyeyrme7gD/eIEg4LZ82l4d8QQT2rxNZRpefPZpW+plPdXlcTl39w1r7Rsb4gq63i4ahK04AAMRcCMkXS2jcP2USJvRdQOLhqDA3e1YV7fWL72dL+syKxDux1xyGsahta5mZRwO41AD3j9cmGQ4SIfeIapWq5ESetB6njEUXNz78UNdLn1ZrIiMY8pi2qKbKiFxSpKGsRzPj3055zIgoNe1Vxv3eV29davV1MnVtepuX5BQq7GkcRqZQ7tK5mHK9dSW88U8KDDlwoSJMbq6xXru7hgWbVUzGybwc5QiiUmttmrWTBuam9l68+kNu9RTqyOEaqtW525wevGpjAmLUOMAwL72NleKyEkoQT6kwgOV8nh/3k23u/2h9kYQVjsxhHsYISJYdHUHNwVik4GqBmdGZsaQQiljTgnCUZUwFNENZMhlP1CShKnXXtsqZMYRZ0OWtaOFAbC7YZB3X3vLUjjzeDeIlLbWdr6c3s8+u1agzJKYgpsaIpVBSkm96uOHlSKogAi6+qY/3N7d7LL0brVVNXNv7qJuss+UvZ0rAErCPKAwAGJbwPVatYFkga6rB5FbPN7P4WYavXcwcAMmW09rEsYktdUWaq2FAhcGQu+qs5t5bOXZTB7e3QM1AEmYmAJIG9STq6uap1GGQQiQhUSoV+9r79XVPWfJE0sis3A3DACK2uoW33WH3rAtSswkTEJE5GS+xUPQkrCRo7utigHUUSt6NwjAAHULYlhB6xwR4I6JmJEFyzSM4+BIFiSFdyWPIu64smnXQEDz5g3BOGViCBIkj1CLML1OuxFxfrMAAifx6luWv1VFQDMLAE62njUw1m3hBMEEjkAJIKEUD+yzhWP67JNPzsejty6ASSisSpYQDtcwIqKqvZ+wuQvJRsCfH1d1yENKQxJydRszf7LPp9XTs+mL5190e7DeAny1D6fjhwXfSoAtD+qL3Ja+zrbMp7YGIYyFB0zCLMU01r7asQXFNI2f/vo363xP5svxp9//65thP+Q08jR5Ch7xw/vXv/78NwCkauFKAEJCIpvwSUiBoaYivNVIGgSApSRh8JQJ8+uqCAERt2N+G5M3UgJgMBIgXksln94EHo5/MaxvZy4B47Wf8uPH9VXwVGIAYAogH8WGbeZGkM0Sitf7aiu+v256nhJhT5rzR18SAJCjUJmGT7/84rLOl/Px9U/a1/feK2IQRCBdoddXuRqRcFNdty91SwPA9SsERvbtz0a8uoDAcL40C/SlB7iaDRmRmTjc2lKdEBnx5Yvbz17cNRWKN+vSl/vL2borohm6WgoWfHY7MuE8d1+7LWEtSpKSB4JYF6vdDQEYJUlKVucHjY6EigYRvVlbmlrjRObgSJJRzdrcYicBxMRh4NG9Kbh7UG/sBmFaxsRcuGC71HVLBRpwLrubnaoGU2AIckoMFPNxDXZr5t27RnQ1BFvp9rc7UmzvuyJGwG4/EpEvOt7ckr0XV8YNAhrI7CQYwC5loPOllVwGEeqxL8OlpLlW3HaMjJ7Qe7g5ERABGHgHRAJiychgwrTb5ZSyqvXWe1hTQwQ0E8QIrK16a9YaiaVEKdO86FA4EN07QLgBOYA9+dyRU05lSFU8WldV7aAVVB0Jckm7cUyJAIGCTverOWRkY+MgBBh3Yyllt9stvV3ev7XAILHeKHHK2LsCAQsJEbH35tbcKgQSMVGEm4YBCo3DYKsihSn0HoDEjHmXmEVbB/f6uHQzkwi3lJAYTM07qDsE8UDEpN6dnInykHFAQjYHbQBENNCUrnALZoRm4BjdGbEHgscgWBIAWHjgNvO7baQUAATDUAeNCCRBHulqlcHNOE1SRhGOHr1qawrqQgmAETNREHnKuPbKkkydyMGBw61HyZkI3aw2yyw4DCJs5l1bLrmbA2C4ARhsMEwZ3AHU3N00rAc6EwAmYaZU0HqPJH6Ny3gEmAEJ9O5hLe8yS8ojBJCwAwU6cGIl1N6a8W6Xnu1Gbmq7buqmq/e+24+DsEWaz20+z8HoSOMhCacAYhFVXD0GpCGs9Q7guSBS5JHHQVh7A50Ow8PDZb6v2mLu4FVvb5K758TDeHN8f6kLeEYWw5wh536tgQRKUkpOOR8fX+tllsRmy9khho6Jg6Yq9qdvfv+r3W+J2CFq72097nc3AwsRuTkgGBkDMDMEgoO5bS6fDWqiau7qARRBkraqgG3FHrEtwq80oU0v3byhW87YMeKJDL0JCHD9xU/7HLiWi12P7es+5Wl2d91UgSubmvB6zG9FAkTw8R3wlEr+uBL6i5th+9wEBDLIdLP/5NNfnb58XObz+1cXQCOG6yMmAIlgw2U8hZsBnuJh+PSC2SQTvGYltgADEUqtFUuRUfIwWtdDTh7uZqqtVfNWyzSMJVvv69I8wpuS+37K50dFkdb6OrfpNqUsgLCuDRhlyL0GsTPGMECbVYHW1UIgJw6vToiASdJ4V9rS69J7MyQOR3csoyBHnZUppl2x7imxupnacr9kJoyQnFwjEFPKu31hoctZWzczKBmRWaaRc5n2u1xNe3/58iWEfXj/LtB67WCGiGVKLCVJEUR47ImxcYQa51RG9qZ1fvz5/L55R1PaCM1GeRpDjQQd+mWO3sIJexfttCyzEHJiwEi7hMGu2CHIPRXZ3e1ubva/fPsqqntVTK7WUpoSpUBce5yrKwTKVmfX6nLpSKrAiJyIC758uecPfHxc3V1D66WzYCImxCycCLStndGgSyIHNeqcBZAAKQCBeDpMN7f7sQyqdlzW9bK6RRh27YpKwPtdQYz3D495GJa105DSNGZkDwXt4eYKHO4Yy8kD0R2m/QgG69zMIxdOmTmTCLlGr+pqrkEUw5hSwTYbGCl0baZmiSVIcsKwWJuCAheWUYTRPIgpCyaRDevuHm7eVZtC2o3IGOoY4epo5goWEQoQhB6JZdoNZVeael2adYXuyEFbc58FGQgg5DTeTLfPJzRhJnNd13VdZkKKMLOtw9zBAIUxJLr1ZhTmBJzQXfOUiCMlMVMpkgUBYlnqNO4dycLMISJ6N8hpmMq6rENJyDCfGjEBQl9bbH+OWigQIQt7uFACVXDY6EqqcQ2wJkhjTpQkCwsxoEIHwJylm7MgExMSpVQISuHj6WS9JeF1XVuvCA7IeTycT/PqDYaE7qVIGUd3LLtRMWANUTWv1Vtd1sUB+lo/NBlSyvjm8oOpXua+rD6UA2GyaOe5XZpNIz8+PnCGnDPccTUHlOjYwx3FbbWljbtpv99J4q7LeZ7zVDz6OI0ePl+OgvBP//l//XT41W/+p78PorX1+w/vwG0/3HhGBNAI7Z2IOrjkLAQGZu5ZCJAc0M2aN1BAABDZ9vBhDnFFR3Tdzk1ApGvjTGyZsatg7GGE1253uEI8r2JDPLk3t5ox9Hi6GiKuxP5N9L0GhdGR6GpPAsDtiI/tcL+q2dt94ACETn9RIPN02zikId99cvf55Xfny2k5PS7nV+ZnQYwI4q3QDP6sXG+vgS364A7hiJs8dO3cAQhi2hZJAoI6d/BoUEFoo9shYwBGsEfU7oe7YWn9lzfv0XNbLZWttNOJUQpSoGzoyupu5AhO4eRhzQwWJtkqdiSAYj3PwyCEkodBhGJRbxpAkohEcpJhKmrt8rCE0FT24y5dLitLBl4f5sqIpkCIbemSUs7CjGBgGwqPyFTDkQoTRT1fVrVelSW34byudT0t5rquzQDKNAzTJMKMTAanD4+1WdNIJSFEXWfQ0GYG5hDMLAkCrCBiNEwMDBbY3J0w3M/rvNQLMbgpoW6ZwzIxoLgvDiQlRzerfToM87r0ZY2gPEoaS9M4z3Nt3g1QIiXKkhIVVfSI6Druh7sbaavW2ZIICZtakkIFvDkRCkGSQOhu2hWmXLCT1YoKYV7GAiRNnZjVrC5rAnHkVjsyiCuCuSqmcdyPhOHg5/kI85mISuYyICKsc+sdiDGXYCBw8EStAWMBYE5OTMTAhYdp4uRaa11duwIEWCQCNutHxSQyUlcPCSAn1DJyoBlAGkUNOOdxcAdEQ0wCrl19m9NRkEGShxTg7NEBMayaLeZdaRsGgUjiisNiARAgVbV2qaCRCm6y4ObSSTnlXBIzGz8/3HrEXGuDbt1ab1tJiAj1RoCEjtq6rt09RBgHnHJB4rJLiQk3yY1iPS+9mxSGZIzU1Yi5lMQJ1h4ihTGKQDjHgIEQ1UHBDdzM3dC3BZUQhavj5mTtAEGM4QzIAELIW7FJaFdXjaYK5k6YKE/jUEpJyQP7fDmdH9fLmQIXtXa5BPo4DSntKN3SEMvDaTekaVcQk3b31SGTQ6/Ho4W6uIVGgVSAkrn70sAeT2ER3cIMcShDPHuxvxzX8N5aPXflIst5iVE++eLWgNeGTCiDWE5tqWmoz/eHm92vLvXdspwExBYwwxhxnntfV6tHPQt/8fLDY33/+uEy9rbOh93e3E3NCLp2NSUngiAmdSC6LlE9QGtd1tVdx2lApy1kG+7M5A7uvvXrugO6E/O2HPJ4UkyvLLktHBaImxT1FyfyZrcngM1CGhAbAmKL4VK4R5gRohORG9L2DvDr7O20MZ7RtyKaCP9zZvgvXwMA27VB2zxYxvzyk5fn01+dHt790s/WFMnIfSsIg42Q93H5gwQU4b7xIGA71T+6p7aOAQQEFGJqzbj4WpUzVEYgo8SIhBHCDEiP50vBkER9PvcWJRIi8Jh6rZxySRzu1kObBW4ACidh1xBO6Gl3d/v58/H9u/vL+QxJkOBwt0dT0365VHdIw5DG6eH95ZOXd4x0XrxDjMNIiat2SrzWjoBJxICmkrQ2cHD31lumwkORhBarxRVbhkx9nsN8XVbvIZJbW8IDB0ZER0IOJFRt2sG6u0Z0d3TOwkI6r9s6FglJZOsWiKaQkSV6t8AIAwDszXu3klIFJYxwy6OUnAJBe8yXOtwCD2Tah7vUrD2cO4YOOwIQJD68OOz3O+8+5eTnNQJSilT49m4qmOdTff/m3qv6po4Knc6rGTARcOSS8q60RbsqF1Drpt08Js67RL1HD0xDCgNC7F1bt65zbxyhXZs7KjhxRKsAkErKQ+FMXVdVd+8eKEOadjlLX7v1bqqQMhnYuJcwqgb+qLq02RQ5LGJ3GMfD4K2Dt76uaqG6MXTBHeqqJEwczVdDiAToQIUCI5zSxKPwfGqmfTl7TpIG6ebaYft3CzUwEAkSIyP0pQWIqYU6NOsNkI0YEyOFAUYqWQotl3WpW2Y2wgAkyCEMIiAPmZJwRtVmlnqbm8XxspznWT3CnIIJMSAYKQCjeagpRL5JZSjTviRh1whQomBOnNCgU2GNygkAau/qiEAylISlpB5rq8Eb1TlYUnW1aqGGAWSBFr4lK3pDTojhzayFV4gATCSJAh2vLASrzUJd3V0jiJjSUIZciiQxtGVez5eL9ZaJ3cO6h/tQyuFwl6Qs5/l4uZQyEGJKJQJVw3r/8ObBopEDF9rv8jiG9t578wAk7P9ftv6sya4lS9LEdA1me5/jDtz53sjIyKxqVlc32fVEIf//O1soTRFKk80easrKIYY7AHD3c/Y2W2spH+zgRmR2+wMEcDgAd8fZNuhS/VT09jYQicLm2roV5k8f/0Qq9aSUbl7gtityvHx6++Kbr1P93bvnbvppHD+/3Dvmb764Ej+/HT+/HW/enUypOt+OFJn3JNTafv36+8vlC1qnyuXdte09Ua/HbYyTYPPmcNubilSmqsMQgeO4vb6+vLy8fPPVl66m4lCQtaCfi/VffCTAijA+zv0qXLpI1iMqwNWZ9bmCUR6i0EMYWikzeXAjVo4YtgwWKEBKpCTXu2Rpj1jJ48dh//Pk+M+O/rXySylX2zAet4F1lVDVd18+f//9b19+/nB7/fDxl7Pmi6suAHpVYaEuqlQVIvUwSFGgEKo6V2YM/Nw3CYX4uCcdpPSrPr17VtUxS0xq0l3Edc50N3fJuKdBNoER0lDlrt1VIPc7WJXJxFDA3HKOFL7/5l3H5Yun9z3x3buvjturdofqOUuYxzFK5fK0Y8o84t2759626/bUW1fdIoeUHLc5ZmWmm/atZZEMa3qOIsuA1qyZqj9CF34xa64mNaMY3gTdKhnIft22a4txtlSoqEvcj3MEqSOqibYu3iRnZGbcZ6ns7zZrlsxxH0KFsSmoOoNgRVKKtmIplTPTu5Kppo89F5Ij3ICLjPN13/bzmN219zZPXVyaOAqF7hevkQlr+rQ9aUpwjppnzN773vvxyk8vL/e3u7fuu7ubmbTNRC3e1motZ7C56b5F6Rwzg+ril55pIpSkmTBqvJ3n2wFRUbSG53fb2z0vz+9VNAOgWFMRdqBvbh7ncT/HWJfKvKcJZMI7j3ssndFc9GJdfHvehDDh8Tq44jEOCtoVthJ3U2SSKmtoZoCQEGwXM6nzrSqKhu3dpe0aM/PGOUtEalbMBNFMfEMp6uQEUVzTv1KIQg3SsorWW79Y3qPB/HKNzVx4HkNNRBVNFO5bo2BUMvM4zh/PcY48BwuFKFMjgPUlJkjSShTX5355f9l662YicpvH/bj1vb+/bt5snkM0W5dizJHnkDLPZjHn1S/lehwRSXPrfeOoLM0sQpvlcn5kokAEY4RMQTEjWSTEFCICtYKoNVaOOXMEBeayb3657CIS9+M8mHOcc2aGSYkUIP19U7YmW9+vyPr44eNxHE/Pu7f29jYzI5HAkpt6xQQcJ7S3Bpkrlxs1OasoCVV4121TlKxCjZEVRrhD2Bi72ft3e51znHlae//V73765T++fnxxs58+vKi9HDmLCdOmoPcYJe79av1ykbzIfvw8bn/38ffy6bhs2zdffPeuP798+PTy6edvv/r2u2++p0BUM1JMgBoxj/P85eef//jjH99dr73vQCsSVVjz4UTE5IpDUxSPRrC1NazW3Kp6TImWFQgULpYyVsvwn9n+y9m/qBJFikpxFUWuEJqJVoUISlY0+5Fck4cv6DEDkH9x9F/u/T+7Q3+dXACAdv/y6/c//PC7jx/+dH/9YzFMA6AQ7lZVIBeazlz4q/QPeThXxdYVR1T00Q8jTop160/Xy7W15sdx5jGX5rdfGqquT6KWM0M02rWZdYcx9PXjpNS758u8jZxznPn0/gLb4aHEzNVGlef49DqAo17u90BZVxPcX45znOra3aoiUqx3I15f3t7uR3Nzt7f7qHOeb8c5qGJh3N71pW5lhrtV8jzy6V2NYzi8SEL25x2IiEgVaba9833fxq0yeNlsHMccQ1WBQgRIFQtWb+Yi+zvbN59H1QwW1HR/133vr2/3vAMBd0cTTwrrPGoxp9rmzEqWuY2uLgABAABJREFUdCUqSytISBW8OaviDCncjnHew9RAWxlyVYz7cRACmWNUSVQp22AyebzcPr1ONn9+vvzw7fsZ8eNPI4jNTCEsuvfrZT81zphjRqqoet+eCDuOOM+MUdY8KUmUSNv8yXqe5zhmJvvTtSRzZjN5/uJp9SLc7hOkd6+o/bop6+3n26wRRRG6AAFRIGvcRSGFko3W4Qb1OT4dRyJH+MUh5p3VCkHdTEzixpRSMRQzi6vjj0SwwDklpvjm1ozJGojBMUKbSGVm1qgasItGkMIIlHNpPnC0xtZdhaKFVDCOl7u69365bP3aWje79SNygmLuLEnwPM5gsmqZ93JS4JURI1ozdUeRo+KcJWrbcrI0bzBnzMgZ5zjHHH65pPI87uc4RGkmTD3POSdTc6gd5bJ7AhHzyOzcxijXrW8+5lkxk/BmFereMiipMoCmGRWzTNfUUsV0IV8e3U+ikWgd+7btm7tWBY7ziGJWiqRkbU8Kor/b9nfv5stk+TFOxNvteHt6ur57umTl8Xrez0nB5XLZe397uR9vtfX0TV8/DiiDLSVEuDAnBul7u1x7U4sb55lHQeB9b19895Up6ni7SHLWPI/7WaIJzMnz3funJjYmLVNc+tabWObMZBJ1DFPJPATxx9f/8X/4z5/++/+ldWxffPX1f/Ov/93X/u71p0+d+s0331I1Kj+9ftqbPT8/l+pxnB8+ffr5w4fe+vsvv3HfP5+upXQh7meOCeXWd1+d7w9Jh4/q9scSXZ/HAUQhkKudvhZ4p6C6AmD4Z/vB51M8UVh5YSWqdJGsMwh9ECkeH75wEfJrmvfPb/8sG7BCu2tCAVH0a//6u6+/+/Q3Lx/+8e1DCl9MPrPu3IX88/AZgEBNpdbR/0GPXv1hFACmEL++2/vz7mh7t5jMwfN2zmITVov9ehHDjDPGFJFL0+tl66WvmaDEwAy9fPkcuKse7kZUVWWkq/il3V9vBTl+/FOxZoRI9YRsLmYx0ggSx0hpHTlub285S5pf323JmpmZg0LfFSnWurspUJFI8SYzl1W2wfqomiMWAjsHY0qpXDZv28Yis7p7xby/HSXYnppTM0qa66ZyP5fnV1JriAuMvFy6XXpH67rpu/bpyJBS3+A0YTFdVWztq0usW1xyFYpQMheHNCsz5xQVSpHJYpyFKoGd5xkxBiEirpLJzJhDbjdoVQ5edodqb24dZ4x28e39vtkOwaiCWYRVfmbpibUdWvXpD5/qHjTz5gQiZc4pYlChqu6XqgVA59bbOVAqwTrfZmSNM7a9d6v9abs898yoUlXnOZQQhXapE+eNaEro9eos9t0gcxxZZ9ZEBbSl7eouhAaQQZA08c3AygkTSpNmgFadOAeRIopuEjHgvl+2qvQmnIEoKbYGNOFkLAONLpynqipgrbupCLIqSpZHUI5zTKb16kVlNdcUg4CKRxmPwlvbt64irjJ1ZmXNM4u9Sb8oU6ipLBHx3dyUjDzzHIHEzJrzFDUBjtvMzEi59G5WMWv5zF30uB2aXrSERIwOr4SLXq+XGp9oGlMWoMVVC05klaj6HIxIgXmzQrFKpVVFVJpRVdre+ma992YGyYpxHDOyMqIK2wbpUC2ztl8vppJUQu/3M2q8/+b6/umdJI/Xl8xTMsxb68acTKbg9W1mirKK1S6qTUXMKcct9eJ23cVwvIwx+HYO9e35qy+etue9P5npp7fbEYya5zGKjGHA/u75aW9bnmfFccbZ1i18Vsw6z8yU8Xb0bscR/bIf8eMf/uEP5702f/7ND3/7+uGX/b7/5t1v/+//1//b5XIt1k8//uyueP/cK0H+6ccf/+kP//T+3fP3X393vb7HAkaLsGqF3cc8ydUwLxSwcskgn12SWNvqI0ygIiWFksJyH3zmwv3K1/nsptdVLbNuyPzzALZqCTIrfosKhQmEDwlo3QLWTvAv1X955H0/l0quYrICAO96/eLpm2+++/jtX8f4mOd0TWSaizz6ij/fGFYKAFjFumuK8biorHBylbm7ivmumpMV58zjHOdZrprB7BU6UTKDFaImcWho3c8xJrd3vrG3rYNw9+k6Y6qtF69o9yLKEDMhiJoMuou5vfuyvXzM3l1K4mBVYU7xdSORp6eLNZUcTEDUn61t2/lWIDRLUDWzAlNB1cvlerk8xagzxjlnZWggs6rcuhQx3saMHLdhbkwW0LonEoVKZBUlyUrKu2+u130/X85xMotwtN2u799XkOMOGAxSUaMyqxR+dTWhkiwJlfnIErZuAOfMIioqpSLonb559w1QJJDMOUR0dde1fcusjMio4BAXfVSPAJKzzl9eXsi6PF9hqtQ5zua9wI8fX5EQyu5bKVgY4zhfE2KbNy3wrAlEYO9i3nPkyEzotrdmGMeEWJLnyxjHzKjWvV3afmlzxsDom/u+88NLkorlTsHtpuNIacIml83chDnJolCaqNPV1Lm9h2sedwZEm3oXhXJUkkCpwRpVEJMjUAmHmHKSMPe9ucmlGzLPwTloHdrAKXDGJInLOxWodstZFLVNV44KoBjZUMfMqjrylWlZGTkFpQ7XtikubQercL30y2U/Xu6sZAVQdPSu+7vL0/PFVDPl9jbmeVoTE2FKHnPGzMp7pHRs3vu+mUq+JUoF1sSTWbOgSq7XRc45ZmIV90mWNs86DDDFjCjSXEUaziGhHLD2bL1PnBFvcUbfOyWQdZ4Hmrfd96du2iClJpwjY57HPQZnQhXbpoISoBJts8qa58hpM+d93OnRdj/jNe7j7fUtKCrazS7N7WraLkNebz++jE+y7Du8qG9qjqlE6XjjXQ92qckUwW773n/zr759b8/niJ9fPn58e+GMqimiujV3larIYJys43b/xCLcMZg3zIlVKuvX3Uz2y6W/3xjxNoce963vMX76+3//6bdf/Nvv/+1ff/v9X/m+f7q9qVtW3o5T+9vrp5f/8B/+4zzP77//Ybs+afesXKEwVI3MOc7Mqc1cm4kA6u5LDse6AayDuApKSMoKjy3u2roorMDV0lUe/SxYEiagvyIZPtfRLwadRYXCzaBQqFEpC2b0aAf7i+nyr6v9Z+cPsOJj9TlUjPVp9q198dVXX3//17e3n94+nMabmvkjsmaP9psHkVRF9HGZWeqWLU0IFEGlqzoQ919S3e4x7wfPQQG84Jtg3Xng1jZrsl9a73beckwet2O/7H1rUbPEZswZBSIF/WrNHcZ5n2NSt1VxL3aR3s0c99dAQlTXuCWy3IZA2967X5spYyoPjskT3i9b8/3LftzOTx9ea5H0YVLVt75t/TzvNeLtdm/NMrJGlTUIVa3o8zzGyEqUsW26q7KSZ50LWqBSTBa1uUmTgrkdb2M5u2bUfR4ovHx4Gec0a/M8wYS7N1GlVM2RuYKIFKSo6dY9QxivWaUF2gP75GrNFOqzAlr78+5q8x4ckxmVFTkB6W1DZE4aYpJlSFe+Rd86IBV5xuTMjFfqNo9w6Zd9c+lHnJUxj7LWkxlZcZYkaHm9XK9PF2k+48jzXlmMEWhlamarX853bYzeuznHOHOESh+SNc/KlKI0heK445gFCEeo9cpIybxXCdgAgypEkgSHnJNRMBdVaJUqYjKDIhAxRBURc43TQWMEGNKfGoLxNluHdIs3pEBXKWqXSqXEtptfmj5y8CIqGWOeM6Kaq7vkyBiMARXUTElEoQzaontnsW+CqObaDLfX23g7Z6RKwap3096WoKQMUx9GugrgbgmlgZUUPDW+/+5LlV2g533kPdQ1zhgzZ9W0aqZU2/dtsxaTOSYUbo0aUOkmY67yJs0oN0WxTgLu/m7ffpDWnG8f3+5QG56+MWbsly57e3p3ebpsCYzzGMeYY0QNUMVz31VIUVbqGWUCmYQltEvbTMLD1C1HvB3zfDvngG/Stk3V5xEs872///qdnENmxcG8M8Ir03a5c25PKqrHLyMa2pPp3rr407V71ZgvP7+9/fjTz6+fTo7YL2obezNKfvz0cUDrvJswM811vEWSSlM3XTwnyL5dtr1X6RzRmm9fv9sue+SYjMs3T9//8M153HBcv3/6op7jx59//PnjL3//x//0+3/4pxjxX/1X/1qB4zgyYtY01b1vQlaVCM1cRNytikQI1trIYhVLqVBIPWpmYL4MmljXhZUb11qE6SXMy+dOsYf0s6LCUqIoFkoSEDwS6IVErRz+A18tj2bgx5vg0Sb/50P/P++T/DwMUO1+ebp++fV3nz78EOcvmjAJZYqoFGVRK+QRh86CLs2nAFJcwPUPCegCumhmImbkmXNQxDWgnb6bX/3py8s58u2cApkjClURnz6+ZUF8xEjkgU1hMkcCiKq+mVrOIzOZMylqXuY0MTVEiJjBpTXGUWembuou2uVy2dzkPM6IkXPME1noySqKFlniararMSN606fn3TcH5T5yu2wqWSeOYr+guQE8Xm9FEXVrtW3eHahAcIQwYa6LKN+ufds2ybwfY55HEZFlboV6+/CS5BkRZ8oOUZ2s67V5EylZ7cQV5EILFbS3vGcWVAxJNijYXNRFqjJmxpyzzO26OUuJlEIdSQ5TeBe3nLPmSOmWVcgSeja9n1klMK1A3o5r7+cZ9/u8umPj8XYc510hRqpBKFXprhV53f2LL7/sbq/nbdxPYSiSpdCy5gpIxQpA9b61DRUVM1UFledbVqYkKN62LsI5M7TapjJTMnJWFjOgDU2BBgJasL7OSp99dYlSlCITNWlNa6R1yRCBWFNRVmY5hGICzhllANWptsCPUlNEMCttU1VExuWyra51KcmoMQoEjKyqEzkghJgyKgEaRKU1d4Uw8j5F1dDO+3ne8/XjQcN+gRNq5o7MeXuZjBkJqrmb0DIUUSOybb3Lfrlu795flfXLx9uYA4o68x7jHOMM+uZ9v1wv27Y3pWQFDFt3k3UkOo8jHTwR+XCmMDLKfG/v9/bD9fr9uy+/Om8fmfP19Y+ok4Ltabu+2xN16dadL7f7/fY2RmXWogPsW/NuUjWzzpkjwARtyiYarJg5ilkSSuc8ZySgcJe9QxW3Y1hC8tTN+3Pf1CDy6Y/H/JSRyMzLV33bbJyDDgoyk0dq8/OtXhvE6qefXu4fh7wlCyHy9L6ZmVAmayYrQJe2b1y1VFlifn23ierbhztLsFsVxu0YEQo+Pff+XOMt03HTn/6nv/8fjvFvvv3yfVb/8e3H/+Xv/rdfPn445+vxdv7w/V99+ZuvR82f//D34xyu8v76xW//+odunTVL2wJIKBRCJqqCKp9BDFgEzYdOvk7QjxAxPi/vRH2O7C7bjz5wQ/hzjfAKmpWu8gGsoG6hEqUQopLAn2vJHmmt0gXqWQ4i/bNf8zNs+nN5mT7oEdr0+vT07svv3z78gSMNp1QIoKaPDUweiDvVR3JBDYtx9JhDEDAB6Vnpvb99OgXISdOJRJIhbKrHMbNYkcxCaH5aCHWIYLv2fe/nW0YQurgUVCEiz8wUZEIMkgDFXQVgCokKcdNmgoZxy741Me67P123IiJwv+WcK8SB85ykWcP9fkLlet28oWZ4933TEhmZ8AWz4TxNIg2QKoJCoqpfRYPKZLLmohSYupZKRrnr++dr842F19vbOaa7qiUWScMEwt5M96BVVNjW3MQh3n2OwUwmVWw1TYz7rGFKEUdzp9DbIyLIYszKYBW7+n65ZHAeWbXGxaZGmGRVrVoEoXVXLSRuL0OMKSKtccK021fP9cv9noNqdVctxu2oe7T3T2pqJDJFxff98vwVzV9vb/f7CyQFsTVPgrlgKSt8lKoqxDyysiBqQsSc91SqqG3P/fp+NzHqgS2hABMVtpgIAeqSHSH+uH4ikQ9pFMzlG3z0UOeZQoxJ2bTtlsGsx7Nm4HgbSIZWg9QtI1lYjRskdHu/X58kz6K7ml3fayXjPrj8lgLF6kAVlcfci00Vqq4UiNCcMVNJbTpiZnKcn5O5DnZANaIqZ95jnOGbiZIuppbIcRwQ7M331r23ccSc98BsG1SNXhYq3nfQsG/aum3Gx3fkerk0N2ZVRkREZESRhIuQs4IlZrvR9+35my9++Prb747Xy/H648e3P4C5tbbv+/PznqzIebvdPn74dL/TFGrLqqpZdDiVUHGjcuZIFPJtpEQlKiCq2pqC0mVzZdLA8zar5hkSeofyctn35lvrCMQ7eRkvFdy+6PvVMqc325+3RBoKVQmU4uePHyvn/SXqDhSaC4GI2p5EtZV7jjmnwLB/vdcZFZkzmgKZMYOVBkFUzJgZ5nbZ3ZTzmMmTzv/t7/5fP/38x1+O1+h20e2PH3789//+f76/vezP/fn6xQ9Pzx8+5dblP/yX/3Tcbv/qd3/z9RdfuipJVevWlgL+oHMxkyUJVTPRB4Bz9QCrrLYW1gJ84GH9LC5NTQDwM1pnMYJW29ZnMNEjjctF51or/2cMdZUoDAtX9GvS+FHv9eso+S9/+et7+NgzCqpt88vT07vnL9+++P54OSWhaKzHP8WqB9N6/dnkmmOvLUxIVWVRzUm6UN5eDmsOo6Mq2Db47v26qaqKpbFvMu9Rs+YIbdIcrl0Crx8P30wp58spYtKgZCUjQpolKeJcAOO1uxarEDGa2ASo0jcfY/bd9n4hGTnvb7cxEks4dtOpInq8nZHYLr3tZgp0o/A+j6KMQUYp5ZxU0/7UVVQd59uRJ8WEM1HC0nLJXFA/QpEjxbS31lSJfH29HWNQRRr37jFEUDxLVMSLHdSSQu8y3s5y4x7H25xHqpooDVggl0qYrlPEurEZUQDVkFHrE1B3E3fHafMtIhNQdQdTgpgKaSKrc1QkC4SOI9l0a2bQ3i7HT+G5fbkJw677u5xDOkrmSpyLFMn9sj89fQe1Tx8+nMcNMk0FJiwRMM5BimihqM23bRNjRKIAYc5kpqnlLN0UojMqFNr1suk4R5EIiqau9MmyhgiKqAmVR1lGW/yGUyo5s7JqnT+kwAltkmeRKCVBVZijiqISFeMFTDIxJpxoTz0Fl+frtiu7HiMIKYH6omLQDVCIqW8ybmW2TiXASv4rChUjGAKFmlWSxRg1gxDZGtXV1FgVwQyJkRHQBlfEOSdzZJ5zPF2vvfftcskc9+M4Y4iiGTQztCBlWOV9EDFOnnOMGSZum1EqK5KTkXNkRC74WD6sep7JgvXtXdue3l+/vrb+x5/+Xj92v2zezFxLqpAzx/3tdn8jC+1ZzJSSTN33y9Pzk7iOjPvxlsFIxjmPgrXFCYPvXS0zQ016UzWLEcc5ZwHqNTOFT1tp44yToeWhT/ncrV8k5/BNzXz1dtxfh7jZxd/uhyZqlqnpc+qTzrNorJJxTnWKqnCMOTfbzDbrSpt2bVoqqrf7AJAVkbOKUexNMu4ROZkxl0d2jtf/8vofX//x5X+x3I8jfvyH38ft/P5f/fZ7/E7/7t9/+MPPvbe311++++rr9+/fv//iy6Kgpqt/PswLpCqryM9H5KXoyLLly4rqPmbBpZ/LvfD5B8WjcAuAPFq+Hh/06xqNR4oAAKoKAThkdUSZa5Gaq4zg1/X982YAPCrj/xwF+xcmUejaj9h7vz6/vzx9M44PIiF5ilHXE6iLTLT4RqUuj0ADKaR85hmRIqTPkf7UWnOi3DluYRffrhcRkcJMtt0u23bq/bgdTvXNLpdN1e63Oc/II8xEMii2X1q7+vHLUQNaVAcrKKwQoNZuSwBZo8Dk9tQmqiT363uTxsLt08txT35uZpACwBlnEJfr9t0PX5oii/fzvL3exjljFhJSNDF1EPSrGxlHVayFhpxMVVdWKSB0RCYH1cTEmjYS5+0+zgkVEzUX6yBkzsgJdympNbBwk5yBUefJ8VolkISQQiUIE8nHLl+gFEw1ATevrMrMQjGlgKz72w2UqrDd84xcr66mmVTIMiloSUoBOmeIuYhIFrOe2/Wry2++8C/f4v5yju369dCPP87/EpCoTBQKl6d3799/b9vTzz/9+Hp7NSlrzLX7FQoyx3BfVZzS3IVrur6aR4lKKWaVtQaXkSNvc5WsFYqBOKcotT1sxtZN9VFkzwQcMdA2Ad2V6nIGOdcBi0XQgG4h4lQAykCpGisplHlOyBJKxHfZnpvvjbD77cgQ1/0M1kgK570YJYSLCKkm3iQHdfkcKKomVTGLUokQQwjEjdBMFbBEfGO7qGmpAyWRnJNIqkkXVFZUghVVWVATVYmZp40Z9+O8F7PZ5y5wSbJmTjE1l4Ldj/OcJ4itrxK/onAlc7SLUGQQWS4GIJIyIW1jWAVYpuy9P237tb8zbTFzxtt95qisErm+sxlpnU3mmFCTbe9UqYRrc9/mfVZJlqiSCgikQa2yUol5EKpPu6UgIkrQu/Qykq41YxZQKcna3tneFFLvv3yi5dvLyLWQuo7INri55CzfrXWds1pX3JjJUtzHlAgRkjRVU73/fBPN1nXFaTMJCmiMGhJFimlVnGdkZSET1Ta3hpH383X8/u/+Mw61bZvnrMl+Xr/sv/n09lO9DUu9XPbvf/jNF++/VPeK9GaVhcK6BK7UVkWoapG1at3U9mYwB0tFHlXDS6JfgsqSh/hZPIGy6lfJft3+P5/c+Tj9PwbEyyMoqCwR1aIoK/A4k9dnoPNfGn7+YrH/dSb861sBUNOy5vt+vVy/ePv0LuJNjLouLAKgRBTFJXc90gckVB+f0mMQoUR5FjSYq/ZLoXujMGKwOGc2vzxf9/dfvv/DH/6eTOvqvaGxrKxYA0wWU5u4qbnEfYqqdjN8hq8qZjEFrmQSpmIiAsIKqV2v/bpdbRz38ziP+xkBBayDM4qYhdZb21rbjBUQud2O19dbnHXczjzQ3J+fjIE8isqUhDICcxAiqlBV2Ap0Q5sDVQhrvjXbvAlq3o/znMvY27emTMkV+uA8gzC1UgMoCtZSmIIMiJlY9e4oKaWJlhkf+ZGqWXOGNEGuVWgJgBBBnHFK9Otle9+gKnpURQxwZnvaxFgjOTPnmpdSgRn19HzVmNt2+eHpm3/7/f/5b979n/7h5//0d3/6h9d5+zR/6nvjiHkfdYztcv3q6++/vH75x59/+vj6MTSfLrY/9QqOXHxP6taWddKQzLnW25qxXsS5zA0ObEEw7jkStsxlj9IjUzAnAfpm4owbKx/+iFx/tjTf6iREAqtlQ0RdTVSbxqis4qrJEa9MTWawqmoijfver621rUFEuo4J771rh/q8vxz3o+bSRkMSKjBXVQNLVvG6SMWvER67Xq/vv3w24etxP44TRAYqJ5StaVMIAdOZOaLm5LZ7d6lRc2ZViou6Xratbc2tScUYcT9u5xzNTKwtBA2pIwvm4gaTMc8x55zDVD0Vxdbag/EltS4MNSEppsiEFVq/7NYw4vZyu98OUVZ533bwDo0572Cwi0BRIgBnjoA0MbXL05XCjx9+LnoprTdopwalYGFdBTS3ZV3OYBwU14NVIuvcYQ8ShbGEXiqmHZICszFyu2wCGcc8b8ectE0FaF0tkZltcxETFe+gYHuysS5NWeLWunvTb95tBM/7ayTinmaKMm9bBCqpaigRQZ6TsUaXxoesIPNM0pHUU3PSrrDL9spjgP15+82X/0bHwMvb03Wvqttx23vz/QL4yFCmqpUCQOYD/hxRI8fW/LK35FJ3Vt/v41K7dgLRx2yAWELPimT8Sov7PBjAsniB+cD+yK/XBjxagJcl7C/uE/98cf/1jP8Xiz0+DwB+3QnWMEBNW+/bdm39XR4fwFzGVGCZWCn6GQFKUh7en88JNIFIJSHwyxeNZiwKkKYCiTliziwKZN/dILfXjyDEyps6ZhzISTABcJCq7Qv1plKw7nEG00Lo6uIVI5NEQpUoirBfuojN2+km++bmet6OcRuRYwYgUF/+LYyA7ZAm++buej+OOc/jiPM266gKCMEzbpWmlplsosZSo5lfOgp9N5MkgCmkwMGs7vb8/mqQnDFHRI6l1JmpUtQ6TNsVPIcGzVysiCyyaKAtVAis2g6Ig4CpKby7LPAIeJ7jrAAzB7Rj3TqhVOrl2t26KOYcu5tZ7BebU56uu7klCzLvUQHEkbJ56wqjFWqOup/vnvsP1/3ffPfVOx5/P/4U958+vP0JlxJr3MyqCez9N9/37enDxw/n/eV6oTS3TczFmxs8k/MsEZgJWfMYTY2oec51TOF6Ch3iksXMXIXGVSmPWgslc4yiwl0zCoGi5mo7NREHVLNk1ZCJAUqKiKr2BmixiqxQFZqZF2fpnEVwJoOAQcz7dVfVcc46qNqaCKfE4fM44jwjCiWmfFAhIZE0XQherPWDs6AQ8Kund//1b/6GVX/30x/+8PL7laxhVXNpHSiwlOboJkir8uaQgomWZiYAN+vNXM0glXGOGZkG2/uO5Ij0ZvOsDBVz27d5Jmuup6p3UaOC87xbdzUXipRkSSURBTMp9H5x2eY9bvO+73h9HWYUa9b0GPfAQS/kZLhBGZXJnDDB5f3z++t7b9vHl9siAylq3E8RK4F3a016996sUDl5ziQp3oo2JkbNElEHRG139y5qIrPIqpkoa7Zdtq11Gpmypjtx1HmiCtcNYphR3sGkNXVvRvPusWFF+aWwt35t/na/VwgLGXKeZaa555ypJr53N6PUPFlVKVpj1iio5L0oRi1JuvanH65f/+sfjk/3byjPz/unP/3hf/zDL038nV7/uv/uPm4fbh/MMWJ29e6uS5WMBERV3KzIMe+R87pthUKhMsx8TWoFUshMqEKKpsJf3T7rQF31eSFda7JwBYO5+oIFi/SDP/s8P+tFj3c9xJq/WNzl84//4k3++c/XIq+qrfm2XcwvlA6eREAel5BHzf1qfxEs4Xt1KKyTrqlAtXJ63cuegaZMIDNGETB3FRFqcw2E5Ig5QUFFHAgCqixKQQ0UTUpD5cgcShGum41IlpDoXWFghrmKiJrGTNns/dfPqybh7e12MqoQAXOooQozkAJTCCpqxHFmcJ4zBhkQ4LJLBeYbY7C8tMliK9qmWDwYUKWkKUZlVJXkLCi23ptbb56bxcscY/oyjXZtm0NsnRsVCkpmtqbgsrAAxQqQIk1b9949JiIyR0WGb+a9QRTdrBWQEglUybqOlTbXvTXX437mxDjKu4h7b23fupmN4z5iPZyQKZUPDEEFx9uZb/neo2+KPF/PX3768Pf/8Pv/FM+iekmt52+//PSHj3u/2Oa38fbp5ac47k9fNDbqptrMfNtEinLexrjfq5CzCJuR+RYorjGpmIgQj7pTSIEKNDGIObRQmXMChJjY1lu3iNAgOR/tjBdv1z7fMpJ5EhR0sc373rx5EfM+XDTjpKDOecSMs4IQ1UjqDgi0iahE5jhjQvoOJef5dr9/qAphdQFMuEpGpVgwMU7GwroTK8eTZylEDBlnziNvrznOINHVXHtvrZusgBg0o1SyHis6SFIBV1XtzTMmMlKMlCJMzDdj5P0ezRohNchS65tmuzzvVRzHXVjqpsV5BpOSVNOtt+PlnCMqE1FNqNYVYOXid8/jPO737eKq4s1jRs3RmqoLWHHMSkmBNn16vl76k5uPcbvfPrLy0nvznsYsAvO8RU20q7cmx5HzzKT13lxs3sd5zmSKY7/uW/fWvYiYY8wzc1ZBTC/b3lpjLaLU6U2Ok3NADK6YAUmok2QJL02sqVHyePCKMyIr9oGyc4wxK1VUSlzUfY+zILLvly+/eFbTT68v7fn69nqvMzllvpa59WtjaSZFrfft+f07D73uT19989zLjpeXj2/H6+32zbtvn7/68s3j40//5Q8//9O7/nxtl7/67jdPl+dGoaAi1dWbCwUml+2q3jLyiCHgJiqAqS+dE6wsmupn2yceoduHTk8BFfp54efnFVwXcEhQUFtyELiuH4+bwK80CP0zEZr/nAX9L7T/x97wYIgq1KCm3fvWr2/SsoRVKo8s2ANuTQiEj7siMqse0At5NCJRfR45I6upGWOQpZSCyr5tKj6PMwfP81jhw3mHzEWoLe9QF+0WZ0LkvKcUUKTZYmIki1Esii4HrCbZXCISwNY3Idz0OM5aJVWFz8Nq0MAGE4hLZU6gCnnEGjCqw7tJIkXqQgra1asAZLvauosjC+Bqm61gEiDVVVS2vTkEtQpPZuvWgCyoeAagEjFFGbNiWQZOQKRKuWqJuJJ1jhJOdZH7MTMSOeddzaZdGknv6lePA3Ok2roUulqfY8YxMqqKjMihl3duKjmjuWTGcTtE1JtWglEJpXFpeWW4j/zPH/503rBf5Q/5ic96SBh4/e45AOleZNzvt9dPY8y+r4ZyyYAoRp1dGBN1ZMzk4/UmOUlBBVShu4iKusyCCJcnCSxTN4d+DsSoQw2+e3dnlauGlQoy0Lo2c08TExrZIgXe1ZqJ0MyatRplOzKiIu9j5qw4IAbfql9Nd1MTb06RZJ0jKZoYGZFzqBQWVovG5MykEBu8aVFi8nMPH2RN44MUvr69/uOfas7z06cXVZEoN1ETW80gwgpGTmF5s1TMkNXisKhO7l1F6swzJmWaO8zMdGbO+wHpxZJbjKPMt933p6cLXGJOapmRwsygOZFiDdBz5FKL5zwVYuu+whJmSTSryMhMitBwzjPyaJtCtJIxWZOA+6am3rwL9e3l9nJ7+XQ7dCXNHiNMBlKM7p2It4/LhWCuhqozwhxP111KUkqVrk0Fc84xx8gpCQBt65KYVXGMMQYjl5LRryjBccCItqs3jVEQYzkHZ+U5M0qWAyyTb7fb+SYJ5uewVWtWcs4o37f37983bXOciIwJhZkL0vxLJ3i9dO/2dhss++Krpy++uhy3Y87xQT4qOE++fHqLSj/y95/+zv6hRaSc+aW/+/6r7/bL3nrvbauMKmghswS4brt7i6zb7S2rttaL5eJqmGdU1TJ+LgsQKlW1VrNc1gPrL59Lv6oUgpVM+YySkIfh4fPkoB4aPWWpOvx1sMzPGIq/OO7/5Sx4xcHW9AECMFAsdbXevG2ie5UtWWrtTQSVEKCI1Q2/OrdFNNZCWRRVCF0bciqLI1iEavmuklXHvDz323G+fDoiCENr0EJOiKJt5puqcBZKGR/ndnUIxA2FtnsgsyIzK9AbeMJ2ZSBreuumasgYaK7jPMd98NGUAiikARQDzGCmCohJRWYAAbvAu9VkFjLYnrRdelXmLNu693VezWVHiRFiAkoJfTMV6d2BmjHnbdzndMO+K6ZYl3MGBwR55rSm4h4YTY2iOWKxlmALXQ2DKZXM857zyIpksKraJTCGiPpuMwSKUpiKLVh5RM083sYa5ruiSiQkM2eM45SMnGcJ6LtDBASzVEV3zaRdOCr+/o+//6P8XMA4jzTiutF83Mftdq/Bwoy326p6T/OZHhkCmefB4jmpahESgzR1s1WZMlNUQSMWqhKUZNZjZewXMxWFmJBNOQmDqbpozRAKGQQz11FJ98umJSOggiBgfDTiiYIyiSS23f2wNf+uz4ea7alfr116E5eyihxjTAU5E9A6F/oSvsPVtGmphMAd+7u+NR9nVc5xRNtUTQqVEWQJ8PbzS769QniOWquLi0OQo3hUxMxCsbZrd12w5WA80gMChxgjc9aMFFNr7q1TyaJfVajMGifef/3lb3/72/eXp5nxp5/+NN5u8xjlSdESwEzVsGTCqIxUsKln1ozSDhEFKZjH/eXd01SXmSNraFccaqpIjqgZQKG1au4mmhGv56d5jk/3kUQXiKc2KX+IwmYm5P3trACh29YqmDMLuL67fPnlU2V9+viaIyPGeeSZc0bo+sZu3btBcHu5H/eZxTwfOq2aALpt2C6971LjZIqoIiUG58i3M1bAS0VVxFZ9lVpTRYk1qM4stKt318w5hLe323k/1ByTl6fr1HnpTQ2bK1BjzDFqzNvHP523t/N23Lcd6iiKQueMebG3+MP/+h/+9Kc/fOrY/tvf/Tdd/PzNYapr0xFdhfBQaClgjJj3MVTlYqpiIhqxplWaWVpQNxVEsTJX3Twe67o+TDm1NCQ+qr3k89wVy0okjyO5/qVdKPHoBtDPaQASqvhLvedfXgL+PDJ43Caobm27eNvHaBXjMZ8ACEQ9mr8yhVkC1GMO8BhpLDHLRSHQ8RbNUcu5H5CGiPnp5a1tzS99vo6+qVgKH4f0YkUCKklqt7apqKihu6HoTd31HjLuMeZqgkXNcte+d9+aUMbrHdJ++tORQEw8WhkE4gqWQdx0bbmZkaMY5Cp0klalVRFRYmrWQLhr29rna9rKAJDUYnKwX7ypScJMVTRHJhICV5HKCkBlJI8R795fm7We7e12V4utK1axLwSq6+bRvEGgKXEiZsTMOAIK3+DNpK0wOcZMTVDRNid0xS4qsyIf7dMBGvxixxFEzHPs180M20XiKMx6qAzQ/bKZ2owxNTD1021I3SNSgPauN/Eyefn0kqMipim8hmq1rmSMI8pEo0zBSSYiGYkAm3oFxYXdpVY0kEKwJKvWUisb+q5m4gu0RObMyrLNt72bGVLGudpdCgVxiEgc2XsXZkYxUrHcsW59p3mOqBmp6oohBWB7Ml4A4eVp662VSoEROUdWpCqkgfMUcrnZKkWbiCJnqKtuve8Xd0mGnJWVMqtRKlFZmxtZMSJczHTf9ByRwIzIOxiskCoWSs3crG/tHENQjOXS0EwyYkZUxpoUtt6e3j0nOTKZMc4zjnnZ+ndff/P9uy+abT++fDjvx/F2U1Zk2Lb3radqnVGkSo5juNr0NJfMYIokuKwRtfptK/NkZXDGjLZfsw5wMiEPazqrSoAIcM5gdRc1uBpWI+nEJKEwZ4KFmolmxiqSMNndnvZNAhmRmceYiIqcM+AdfUc3AXNOnjeOmYv9wA0CWPd2UYe5N/Vk1Ih8eMxI2tav3TbGggTPar6rl6qWSK7nXWcJ+lN37TXs9fUtkvM4WNEkvDXkaarqsnd3ICqtGzDfXm9vWcmqiuMO7d67psIu7cjz7//4n406zjy5f5ofZRPfPMljjuN2XPt28R1raRaPKBCXvgNw7aqe9Qh2ARIRblasOYIoNwdYsQ7z0DVWwEMfEj7CvPWrd3NJQyJqn+kOXqwUtc/xMX6mEP2qBP3lVLj+8id8BMdK1tVWF5pF1UxtqzKG1CokFmEVH3gLYjUDFMV0adjrJaQqhDgmIqJ3iGqzB2owD9JV13HJfXsW62hbY6xWeC2Rc6a4XrfeNmPkvJ+kVKVSzcR6/+6L7572y8vt/vJ6u7/dZs79ar0BghEhHffjyAndoA1VMIUkGBWAd7amFaxZEUAJguQirSpnxWSJbpetP1+2bqZSrON+P4+Zq7VZTBUMe1R/JipLXM45UBXB1qS5ADIis5Bk3zfvtpuewf3qEbM3AziT4kvCwb43K5TImBmsCMZIBdoV27tu+0bVZOUc8y3zDCGSJp51Lue0UGW7NmRJ0kVQnDlTpyqa+ftn+3S/r546UEYJTN2suSuoYlMyomqUddXm5Z7CMe4ZGWcwYr/2trlB5qzMOu6hHYvL72ZqIkTNVBPdpG0torx5HFmZ4tQik6hEwda2rsvcWAQYVUSJbua7txJERM4Z58Ng2jtYI6ccMXNk5UBRClIUlcxZzKosrclhmr4JRFQkCXfVTbJqzkzmyKooh6oJq0oelEVtUsWaNShqtl2v1lykb/tWMngLuK4cMqq8iRnmoG/S9k0doJhpjYjBcWTN5fMT7bSm5qaTETOXQ58NlIo5g+rStq4qvfvT86U11WTMmDMRGGftT+7dzvP+lq8//fzjp4+fKmfrru7eVZA5IyLNWol49/OY3nyOMLWaOu9sKOl925+vX/7VDz/89v3799oyjo9+tvM1TQGxbgwgZvKo1MzKKlax9963zVSqMitzxCiUW7+6G8jMUu0ikHnOEdG3rt6qYpzx6fXl7XZEiRRrAoatw4DzYDLZMgVsZt3UUoISAJbdNqGixXMEV5AvM6Oqu6uq+sV9LlCjllo+CPoVMXIw9ksTdRaqxnEb5xgglKDBNgWsALqngJgjByXVTZCz5pzlrrqZmIpL75aIjGEAWrt++/S8fcuLzSe5YYwceZ6//PzzN19+vffN3EUtmXPOEaNi7vulgKwUoB4XJ+nbvgiewcpMEVWzBOUBWyNW+4nUQ/B5nFE/z3nx2BqwGNTyoMCVli4pCgLY53nAryXE//tJ8J8lIHkYSx9wUIiaubcNaAlZqwqIh4vv8zxCuKYCS3XSJWyvL9PVtF/UXGHIyXmWdq8Z1q31Xqs7ZjPbXLX6rt1ViPt92Obb0+XS+6y486RJZYE65nk/Z2tR7/ydX75o72zfMCvf0rVBakbF5JyZJ1Qxj8fX9BhYJ0i4gY1M5FgmpgXx/pwylSoT31t/2lozcxXKmHFOnidba1ZSkTJCU+DSvLXmc4xlcs9IFJRQGIwBS4X55qIV8z5CDE3q4XC/up15Zq7rW0bt+3ZkiksGkKM5tEl/37enTkhUSaWrptW90B/pVGHkmk26qpmJOSINFskZo4ini7cNx5hKVpaUQgWTNGSG0bQcx4ksAZOUJrp7QmOQEczEmB3QnNZ9XV2PUaWoQk40Q67XgMJ34RqrVkpJHuFAChBc3SdW61VGMDkhqpkP0i0M7uKiCEAzI1aHxkKc1AlBcUsmKlNBd6ynGpHLlpkZGakEFb5j9SZVJIExUrWWHSAntMS6qgGiMz1HrjMWUZU0kbZ1cxUYU4C2Ne39lnuiqjLghGmJb9fNe3MXiMyZIUIXJmgASiiqbE23S0fVuM24TwH6bplZKRBse7++vzTbs0BW0sZ9xDEzUgAktQjRjz9/esHHc56vb2/n27HtBqM3V1ERU0k3MWu7t2SEPWaSWNUDb9NdM2S7XL7+7jdff/vt+y+/mPXyIc1aa9uT1r138VZv4xifVs7SWDEim6G5b32viHnOSo4DaVAROUubQKUoJkJgxozJ1rMq317fmBwZ5qKmKKKLegI44nF/henqGpGii4ojk3GfEwhDzNi6P2zDWkVoMc5Jye4XsdabIFAipLCQOVl5nuENvnX1Bkrcb8UUwUICqSM1RVMEI0cUWRNjQLB1tYtfn23cA1K+hl4xxzm1a7C0hhS3vlsHNvtwvP14vH5xvNVZv//5Txmxte3981dEjXF+evkAkefLk4hGTK2y7osTlxEuKk3ncXyO56PAjHBfAjuqYoHkVkN9LuVdRbjqealYGUQUqGIPXf9XaOifF318nn7++su/BIL+ZThgnewXdAEQNW2qG2krtiHFx4BXkCBKqlL4eJS57HH64NyR5UqTZtqsUNYxI88ztqfum5gRUIrOMe+vN+smUFNWyqwiXdQM7ay8H8HK3qxmxajzNXuvun9Arqe6zjiz8rwd9iTjzHlmxvKRQtvyk0AMKMSUGmRKHqyqzKqCNy1VcfZLa4qgdrG+bUCdxzlPlYYUDRUx0wCKMpkT6t1NpKwCqrZCTt6aQGRdtRcYDyKaVfX6mm7SNhEQoPvWHMfBzKTAYFC7z9Ju421U0a7uorpDd0lNAhGJIkeWQUWYsjVnMlcIxrn8XwDVJLAmajB1EYuTBMfMJGqKN9tV4syBk04UCXdpVHVnJjmojclSloLevZGSmUNKEcliegfc2pNJrfQRwBLWAtsakOu1XZQsQGJtxKUQYRYmtT+QuVkEYEvMzJpIKHUhqNar1JAJKcx7qOLhE5PPk6uZDJYASnWYCqUAJrPWxWOMrHTf3JsiZV3yz7JNtSukYvIR5zQRI8QEEpnMcq2jeTEcvG6eGeMEIb656+ZmEMlgCUfVyFmEd3+3vyNRY2aMfrVuEOYZkSMVqiKiUlDr7f2X777+9qtI/fDpJY7j5dPHOM+8xfVpT8ocp6nWjLf56X4755gF0UdbX0sARaEYnMJuTUSbOgVHUSm2laRtac5WZd/+5l/91Q9/88O3P2zX/dPb7G2T1HnM7mJmhchj8aYkRhWSBbv49Wkz9WOMOEeezAAImZEH8mLYYSqVJCvX7Sh5HqcIMsu3drlcVVFSnHnOs1htE2um1KoFvUqhmkjf+fap5oFUmEIpsygGqCz4gaRWsJhoJ7PMYdAZozJKQGHF6hZ1AOd9YFXtmLqpbdK6NXMVmecszpiVWSJQJoBtRQ1EsCtKqKjgyFJVzdp6R4IRZ9w+5S/79cuP+eH3n/7xMqjRPn781FS/un/t1lTsOO8//fJh2/v767OZRc5ZsyZTsrdGYMYweFSNWcX0LLLM/Vdaz/JUriN/ZALLZfqY5hLk5wDBMtMtjYjkZwpcLcllbQd/uR/8H+QDHoPitdU+ssPrhr6OF5VcD5NAKMLH5/ZA1z3o1SBEmcWlCCW9oEyFahZn1IjVPVqy7EAPt32Za+Y43mTcQFV0a03jHJ/u86zBKhNnZY5AsjkgOI/4+cdPZalJmIjpPPK8FQVrIV5LoQvEFEQFYz58qjUhk0WaPABO3oUibTMlG9V7N9UYNccsQA5J1py18HeoylmZMFeThrmaz5mkOGyZomINcowsJPOoEaxZc1SkmZRvvVq9vuWMzCh1N3NAxpw1KaqCFBftqp3FEGqxLpee95pIOalFBedtMBLNsPjhLpmgLiSfpMu2XS/b1l3JGmPMHGWrlkhzkpX3e5wt3F13897a5dKROVe1WyWDKCHdavmUs8iSIKWtNtpGE86KzKUNWjdfidZcbZgwKMkoLr65qUWWqKqt7mtmUQ1rMNjMmZiZ5l5VeZLLEAyIQ1yrqGoKFlIAEWMoReAUFVlpOSHAojCZs6ooQKX0azPzDBkVcVa7uHuvWsLpXAqOd7eGxXKMW8yEYs7zMCkD3REsMbVuunUAVQsOGjMiKtNqv/T3T0+b9Yz4NCeKJu7eYiZTgdUEsGwd7L1f+0XDGTmO83i7cR51Jk/cx1Es3RXmmeOMItI7ImhNqY8tbjKsiUs3Faw4RaUpDBUGBCRzIVO//83vfve7f/U33/32229+ExnkbXvbQKmSMr2PkfMc91ouQO+y+KvX69XNkTXOGHfWgBWCYMK+kHGkQ0upjop0hW1QLQIZSrW2dXftzaPifp+gejNTQTAHx8l5RiV6E+0ySitKgF21P3U1tB3Ha8SgbKJZyMoUjhqjNrKhR4zMAQYE1tVML9KMOF9OAra3DAQgwia6qe/W4FYp59tLWUNRbA0zQIl5ciZMRd0icNyjVJtvKlWTKMkKQ57n/WX8PD/Y24ePf6y//67/5puvv/XWfv7wiwBP1+dZk1JjnrfzFmDlzEpJAOLerFtG6BRVd7cKzDl7b6v2Q0Vr3Q0BELUkFVGsCgCVYCmE+pkT/ZgrEJ+Jc5+X9YdhlJ9nxFikCf3LAcBnV9uv14OiQtbcDp+74COTGcv9udqKfx0I1+NpW00AsZx8AqmiP3/zpJszKqPu56QSmTXiGGVGoS3UcclyO6y+HMkIkWQ5AkEISqRmZE0q0HYZIxWIlxCHbMvuopWskAJ1e8SfewMJiWKhAgSaaxPVBAdQ2lzSRR1wNVO1YkhGiEvFinrKgncw6YCVazFTsmR181QOVs1ZUwpS+6U96E+iK79UuRyZJEXscU+jWlSdn0LIIt2buqEYr2Nk2tYemUBDWkELzBjRTHUts/GQ3lCImUxqK8C0hFPOKmm2X7Tvm2nvjwhZxBw5zxnizfzSLI2e41WlPA0iXaxh66QwyjsrClL2yJJXgWsVzmRwBdtMvYspmfWYgwQhAgeRs1gkaCUipWJiFFdVi6ztSRkFYbGI1EYVGsRdbY0fRfMcOaKCJLSDVFWhmmRWFihFgWuVcApRldBOVYqvuxfXzDC5Tk6yX65934WZxsxS9355en7/dPt0e7udsmwkQvdmAlFURs6cZ5AIk02kdVMVMVXpIihUZuWY857nUbl8BLu4NZOadR632+3tYLBvmdCKAsXEM5NSqaBAFTGP2y1vt+P+8WNFNJVmls7jdUHoStsMaBbWC0yOgIlBihkRFOnmppTCrCxNh8Lk8rwFFbdjTBLc37377m//1Q+/+evf/vZvL9Y/vHyceTtur/M8RXTcb5F3LMuyo3W0Bu0b1mGEOY9jvN3PV7giADhGwkK2q48xoZgBU7iLbV5R4iphEIuZq3nt/nY/7uGbo1AzI2reax7yUGsnawAti+jN96u2C0chme2qWeQpFMDKvElTb5en90/3cZ/nmSN6k9ZMLEUrk2OkWWu7z4j7bS67jFsOiEFU4na/j1moQbKbr1hVzGSQIVTVzPvJLKg7ExmMOQKMmcVqIz4Y8OnTH97kR/yA7/2//jf/1lv7/Z/+tHtr3c3t3fP75PynX/4oVXEeyXL1ht68ades+fWXX/XtuTIISJSauDiE2gRcd+G19D8O8IuvsDT/WiwqleVLJksWFnCNGJaW+aup6C91/z/nw/7yTT7Xyi+BRx4MO/k8TU6yqiopqFo9B6sgfj0qy8JRJKFgUQlWuRQkAsUaI0YIi1IMFhl3uD8yZbJr0fsmNSJmRVXca9uoMFCUxcycpSpmJoXWqStBqtAOCBB13oDG1pGEKdYpUMRqZBUq4F231txYUyrovipzSKa7MmscWZCcM7OYApS3LqwKmLA9YBekihbmiYwEpCTgYihz23rv3UXqHFlZn+mbEFPkygtqEbKg2Q1CW+l5W3XASm+aWcmo8aCs1pwsIuG9RkQclSGyImmLzBRipr6bKmYyTl52354u131XGINzZMYcb/cxToc5zF0riKKqmYi67BfrlyZiUYxzFFJVxUp0lZ4JVKQQQFIoyhJ14TIDF2PWfMw1sNplVbUUm7B08Zd0Mxfoaq1VcWk4jmMFGmWhEtVQWLDBOIOQiCoiE0hBW9W/qEIOwCGiq487WUJCiFLW55aLECwqXEE3ba21Zlo5x4g5RdK9e9vizpo831JEi2mOYiGFM1msqCU0cZJtJeCdIJMlyBEizFFM+KOMnSzJUW81cuR5P48bu8q8Rd5TuipUVrVrrSomzhmf4gbleTtiRG+mKubr6LAmn5BOgr57qTBKuy7AzYxJgalBEBEk5wxT62iX5m7mCZom08x+99vf/u23f/1X3/zmosbMjx9+/vHDLx9ffq4Yz8/XOWNmRg7r2DdvbYOAkIicLLIYsSIaIlADHfsm4oxIKkaib1C33lqBETSqqUXynAlItUTKtru6CblGR/NODjhl8f5YicC2uzXVBiDznDGAJm3TallJ72CVb+3Lr99/8+U3H28vf6q8zXPF8wjkrDlAg7oc9zFnZkC7qCFTaDZT4hy1ejmK5u4i3lchYBFakUctHwvcuqqqQ4ITMiOtKSvNNY63t08f45NMjv/L3/x3e982ae8uT2/HW31K9fb++f3bp7dj3v/wj//4d//pP373zbe/++3vvvvqW7+0n3/+6R//6R+/fPf+66+/b82v+9O7p6cixxxuIrY65FlRVaUKeTTNllaVCapc13OzymUUnw2ieAg+nzuBHyrQ58nAX7bE/MVAeG0a8nmUWwkACKwKgypGRuWsKlHNh6tyCU9i0MWxWIX1a8ydQQH8eHuDa0Qlq2D9yb1svkVmphAFM1W1KmJwCpWqikbELLvq83XPs/LkzDSBuUKxVGW6QFnQGGa9MjMSpjDDKizTxcRLciIGVM1KURbJTE5SWL0JUWoqROs6ifMWDE4EBqRbzVNNhCIqWdWairQLESHmXAE4cVVl6301PqwhSlWOCFW1vqiboFQOlFrb3RyqtXIgi2Tkm1ai9TYJCre931+hrBzBpAiUyISAMSWD0pttTa1VBDO9iTfAUffar+4qyDyPe5XWyfM2isVjmpu7gIwRoi4uvlvmeHq3f/XlO4XfRs55ZqRUSqMLciYLhEZJCRbirgrMYjBkLmphFtcd0KmLVw6sQZIuGeFyvRgENE+OGcFIFiMjw0yaQc2FiLkAqLL2NqrrRkVy5Tacq+KEygfcimTxkX4pkQkFVoM2KCzVKhFVVV9U5BkzMqu8m5lWjHvk68sxq2xpRg+ec5FrOShTqEAbzJCsuM2Vn2IlZ5FQQHNR0qVUYS5pGYtPrlvnddvOt4Bj34xFMZoJFWYAdI5kzZmZI3xzqBaKWRD4Bhr6e/dNVXRp7I+nWVikb+atuxqmVHGekTPRwGaiUqyIysoiN9++uL7/6vLkM8e8vd7H7//xn/744R9ePv7RZJDTNjO7ZDhQvngUiZiVIzkyN1ez91+/E7nNo2wT7WJdrUlljbPajrb5vm9CPc9A1WRJDRa5RFc1JQtsKmZ+zsRBP0VC2iblqZugifY1RAtRYYiKwauAqpQG7zABhMw56hg8vC2pSnLyZAaRAd90f24xq1R8c9sfmRdGpaVI3W5HJMxwuW77tmtT5EnFeNSuu6ymj97cuzDak8RxJnqJNsOmZCKPmWPMgdoDclSO7/72r764P/38y09/9w//6ayJ0u7tv/13/93/73/6/7RL+6vf/uabr75+/8UXvvXjPD++fPjxj3/63TG+/+H7r56/vFz3++0EMyhuzR/XjoXPKlsZAUJl5cVYKClZo9QquKjaOrP/avp8JBIA5f/Bkf9fKEC/jokXT/Exe9B1+K+Fm4wVM14uoDWZE5HJR1IBy70qK80KgfgxzjobBSKlInXnjMecYduNAhOjaGRhljxGz7SiCnSyzvHe21taZalYoCpBRe/eN2NiTj1vjCNJqKUAkWpeQRiRATF6NxFp3UWkua/TW6IQUGe/tN41QwqmkkRQYaayyX7ZDbIOwOP8rEVoHRSo9SdzRYxDfd09JUUTnJnMGXNKJkB1BYoiUEWT3i9tN5FEzsqqLEKFIKu5IEKiOhRl2+bjfjzELIUCHBjJIkrERUyamSY0VaCECbN6N4ppVLwcBxAUBiUX1Bq+YCSThJmmm9QKbPnWtMfEMc63+6GZXUvAHLlcCBCN4uWytW4sjjMrKUTeziDFaH0NNWEmNatynTUUSppt+956d2gmiiVWMlFjIlOW51y89CHXNVNX861nlsqa4goZNNDgTaGMARbQzBQOyeDC4Zl/rl9fll5goYJkCNfkNzBnycqsR97H25ycI91FJVeMvY4lTIkIREVVtNEUUcwSMQclJ5mUWXXA7EHypamupq8BmFDk+tTNpEJw5LLPBmjuBLWJGCqZkVS4CmC+tWV2qRGUxEZxwBVqpeIuMWqORJSatK13783VatHCB7KaQCtAOUcJGkfmDFC8mZbkPY5PrzT98OGXn3/+6fXt5+SLtYM4C2VKMYmoccg45pzFCXmQy/LpeROqtwZJbartEQ6dmdqwdVf1rV8qWF1iMs8hJazqz23rEscZ54LHKx0xmZNqpsYz5tbdewuWmdVaVwSX9/smco5xHqvsAqZwwwxCcT9uv3ykuvVul6dt3M8MRiELkjxeQly8q5cEC6xKMpDbjFNKKA/gAUWjzlV4zkrJIMWywFLX5m7zHOenbB1NREQZSZGcJZs+X/qn+0jO//Xv/9e//eG/aXv76Zdf/uEPf/zpw+9vbx/nbfy7/+7f5THeP33xww8/fPftd18/fwP6uFcKv/32+5pxfff8/O493M8Rq01A1QGpYmbmcuUUHKxH0SNA0MBINue6PT/q4D9TY//3q/sjWYZ/Lv48iOa/uj8/lwgogAxWMjJjRsz5iN0SDJZwWasFwpVlKcojKqwiJWpkseiRiCMf477KXB3z3X3bxcTdRPD4u+9lg2Y0ZxFOYOZMfmyx8gYFsHRmuWr3rYmWSRaapUTMWslWmQcnYA408a5u7ubVOVP6tn3x5XtJvt2n2+3+dkOJWxMVVs0xklVV2rTt3VW8CVJoFFMNz5kZlYKCorl3KxlgaZO2qUBJjohkEVlVYupuagJVUCtgzbfWE8w5M7Lykfeuc84z0AWAqIlaxkN6Vleus3QhhVSsd4qpPSmClaApHJlRc2JRj4OVMokompqbqgLJGAmQZssWSTFtbu5Jvh73HON+v8cMn7kcZxRa80fbZKG31lwiCsxVsQXKcZY6Guh93X6CIZzKZSETs9ZFtCAloCKRMc7KkMpV1L6qlFbOXNT7dnm6PM0Z53EmqrlGCgF1a9dmDXWm2TphijgEMGckIiCCWt/YgtlS/iFgjcpkAjVRXPtEVXIMziBHbbuKUJpIISJJiENXEFDEDMVMUX/qIjbeZiWVwiGSwiQcaI+xuSJzZKlIg2lrXY+Mfm0uwlEZNVDSzFwzIrMyw7v73r0gZovOSihU+oW2GUSKULN2aVmzKoRq1t1MKRVEJlNAcVURilGYFcgoDSJLqc3beZ7H24s8f3F/+/nDhx9fXn98O39OfVMZWTNjJBFknnEeMm7FRE5sy0cHJfj6eqPANxcVaGZFFLXrLshJMz8DUgJxlan6aI2yR6az5oFEIceUNgdocoxpXUsLOYSMzJpzqbei7fLUtqaZTOVILG/5MQtmpnrejnE/RAEKo7SpNkqKmlYWozRNzlVm9ljbrMM2FYqkIR6ScnM9Rh5vg6Kml67d+j4s0uT6/JW4HIE6XpdSGZmZDJUH8gflLsd4/dPLn/6f//7/8Ydf/v73P/7x9z/+UaQa85tvf0CXf/rjH7754aveLyPybd6attt9HLfjr3/3t0/7tXtXcQARVQU166272rKtF+szZhiAAox1NQTks8Qjsr7TEEBs8SNqxYAVgke4VvjP1v7HQPhzDf3n31ne/qwKMJiRMeaMsUxWrFoTaQhqfdvXsowCdZnuFnEEKwwo4qRoU4GaUdsaoFnb9u7evLlpSZ11hCwtOAzUKqzmopHz1DhrNSmsWbEU3MwgceQsQK3vXaq8eHCWEkDONEVrtv53BVjYb/WuUBN52k0LjPTuUswplTUH18rR9uauWjXfzqrFtF8bpdB1oVvERZisSSUWxaaqSiAGhakZBOBjapCsKgXUJHJm1MhkuaAcrBE1o0gsXdsLyDmRWaK2APGZKVDp7msgq2lK3I9KKQjcV6tanVWSUlQKaWJQE7fqLlKgSYYlo5KFAqmttd6hzsI4g3U2CUhBSrJWp4qqN+uiFpEK05SH1TMYDHXpT/qgo02ByqopfvRgiIi4w5EcZ04yK+ecgnLDzNp2tWZCVcoZSdPu7enp+alv0UrVjnGEhHUz7+t2KVnbboMlBJm6xl9mqq6LmKtEUdtKqj88EqvlBACTS4SXVcySAMo6wFyfLy2X+qkGmMDN3Silal0MYqxU1XUWcKgoFsUDj4ZvzcnMRNHMshSzPrdkI0dR5fJ83Z53gc6I2/2Ogpk37yslnVUTBaU1mipUYlZRdlMN8yqZVKiuf6+0ZmaubzrViIpfCxGRIskdAmCe8xwx53z79PHt9fj08vHT208/v/6TXd6aHGfGeQzdpYocqClCUVVrn6GVpuc5S2iuiVJRmFBgW7dmPGbM8q41SDLnzJEiaC5Lsri9REwu1YtAKdreKglVCt08Mz6NCcVu2ru7Kks5MZNFZjKDED1nVqnvOmbUrNZUVMzQLl2NldVSmLyNktWkklooe0SU4N20CJMGqVJRV+nHyVl6JlWstfa7v/lrhf/+D5+0ty+/+CYy7/cZb/e3T4fUkY0jsHcVdwAxC2r+5f5WH//f//N///9Fz3O+vX368qt33//wm/3r62u+/Icf/32DXvv10tovb1u3vunlu6+/F4pAjMp8VMCYajc364v+8CBCK/CwZRaqivQ1GlTa+s2HFVQ/44EWlmFJ+isG/OtHEZ9rx5be82u3zNLuf60NW/UUOTPGmHPOmLleV2sMrHhwiqqK6xkg9EFOpNTjby24u2BzU10LccyqImUUMLOWl4bKp6ddUSi4iAnFvLRgzDSkYkngVWLVFC6MY2ZxhNjF0fXyxVWCm9RAzhllc9ts21SIKs752Tt1nJ9mqoiIzMi+bepSGXnGjCSopiLKQp65vH0wY6EqMihm26WrOZWiUBck5xkLDc+UhNBE1zyzVc7k5BqjQUAzbdr63kg957yHlCBPxKwBN+T6L3NmVqXkoGlRJSspsE3VTQRMWqrkYlAqlSplDccrc5AQjIJCXX03qVQDhdpMgWRqapAzA8TmLt0lUFUjw0z2zbvGZNVUJIKwDqaANLHIjJCIQJBRQVGTfVcX//yCqgJEpLRMtG3e3ZWaWeMcK6W7msig1rZOrW1vy8ftCrV2vT5dt61ZV8nRQtIEJS4iLCYz4HKcsQpmRDUE5ubeLVRYcQa5ehrWOfKBmS5wbUdcAH8hXJQUpUGaq7mgFlsQoqpXVVXRdemXUrhbZcXMWENmMu6pqioF1RLBauPOVdAn3q3tpl3VlnuPMWomt21/fn53ve4x6iUOZHhDb6tdODIzC8vN4QJR1mPtg5bUkXmbeRSaNu9b6yxWSSZJJEM1BbHYOAqtBFeHZYFj8DzePv2I4z4HbsfrjPsx3lyOkedc7o6Dy77mUHelVJRSqz+1y9O+qgwjc1E5xeD71rdWkUdKM1PovA1kniPmjLapbCYqdc4xWQFvaE3VXJubak3gYGQUgwI0PD0ttJoic95jjnlGTVaG5PHAG8dAzFkggtXqcmm92XbxIuDS3I85xqy4p/UWI1gpBnWYuClqogahouJKjcEz5kykuoler+9F+v3tULbe9327vhxvbbucaDVvjCRh0qy7u1JFEtbbzBmR+Rq4xxjRNhzVZuMt3366/fjy4ePHnz5+ub/7zfe/eb48Xy7Xv/7mr791+cM//lFKnq/X6/VZTVVWoA8iUFGgIuphU1nKewWXPNYaFw161aLwASpe8tCqg4CqwJZJ6Ncz/2cB6C+HwI+uYXz+88uZAKxW1srKjJHzZEWx8ADhEgVCHj4/UB5dY1JZhYXFVla5mkCqXXvzFdEexwijTzln0R6Ad+ubNRFCVHDdeoq8vXH+/8n6tyZbsuRIE1M1s+W+d0Re6gY02EAD6EZPk8OLcERI4dv8evKZFJIywgdyuoV9me4GKivznBMRe7svM1M+LI8ESJaUVEpmSWaes2P78mVqqp+m0c2dLW13UyVFs8aaMdZAwHIC5mZ02CYLs4oRF1pGfWE7S53HkQdQ2RGrQiKqsirPnK3adxdQlWGG7i6jWWfTgBIB32PsAVvXPTezrqa5qiq9GiRWU02pS0KjPhokh5WaG5SriBHRJqlmWWMdHmbwzWdVfShLlbBgU2YmsAA3mi2ugNswk7WR1Ma+DApAbGbLDt8esZWozgbMY3jMLKhrwn3rAiPcd7Qd51mzjIrg8AGOPB+qXr1YefbA3LZhvZrHUOc0aYQdmYE9uPkKvgKyNgmGwQgbYWFC1cyjM091uy+GppkbzMS6mlaYagzbQlsXz66qs+ZJTEOju1RLdsEsdKOX3tkLEkXVMiWQrUrap3JNWz118RKXxagrq8xNFId59RL6sR6+AmA2SLoxqhagju5x2/cq5PnoZymzpiDkWT5Mhh7c74PGFqprbLbf3FxjuLk1UtWnhDKIlVVno1DVHBhuRJ/zzKy2tXkTuXyeQMHLAC9Vzzwfk9N8RNCGILOyRcME2jk0YncProqyzvMUCjFud7+76jjfax5hL9+9xuthP/L18fHWUzEoY2V3CqI7IMlljtvdt9uIW2ybH3PWMyGLzSy4RQzaWc2mmc+PI8/ZrXNWN2yzSozvhkdUvUPtm8dutn5kJlGucrNxeyWNEdtmvmJP5+zK83GeBxhO0MLjFt11nmf3ij4vn4zmY2bPGPby3ffehFx0fbq4akINGWzHrJV9aaPDTVJaxhZjOBk33/f7+PL3f//lbYJj3OM4zjYIM0IpMcyHxb6P133Qug1Z1cnsSmT26/3F783RGnyf7//1l7//05c/vf3p2879N9//2GbPzlDut/3Ll59L9cPr6+1+I4RGV8dmfnENZTCz8J6iBGVmKh3Yt13XIW1Xh7sZ2Mb4vPJfb4ALmv+Pgs+aBj6XipdjdKn+/uvKQGvyELo658zzmOfHnM/uEvrTFSRchshVQnMJJKXSMkMIpSYR3aUTtZ0963zWkW3DY4zOosGcJroRKFWKRfgzKyfPlDgCrG5C1XW7O2VgXeGEwt04z7ObT0LuYxsWizuGzNShJb+oSyW6lpYCgmynerXBQqg2YiMh1bCaLbGSXTCKNAS2jfEybq87yJqdZ1XrFM8uK8PRVfKNKMEAsbPxaE0TwR0I43BVHo+GAVNKRqwVPmnJIAbNI89CwzfaDpKXsNRWAjtUZY4GCqkCKGdbo2ezGe5upDlFmGb2nGn7oHm16szjkSaiymMziz7rZB7nBDHCYRB5qqdpuqmtABUs2S6i0IkuVcfusW2cvt9u+x7znJ1ooGl0Dfq2bYuF3OqZ88iZZ3qYxQhjjLDYGt0q2vKOeZg7vSsf801iZ1Y/pWRQqpxd1t1NEwm4KHXLeiWhxIp1N7GgcjGr1Gw64xbxuhl9HtNAO1aTCxAYuxkNkqrVDYOBRjewplq0CLrd9t0HhUS1zlbKIIYjGoYuxOYevgpRzcd2d6PGYLhVlubK0GDsg+Dj7ahqG15I9zTOnHmcUzJsCxppy2ekk3oWy1XzrEaysmO4U3Ue7YwR5gsNYBwce2xBg2ntTMk2msdtv99yf/H7y3avM9VJ+f227c/x0bOrR3iWOqEJlTpKJhi23eM+iKL12kSdlbfb7f7djcK2poSzmzweJysFiNp3tz3odnu5/fjjC6B5ZCHHq4+QqlunygTGzW7f3V9uu/uWXbPOOo6cz55HJbrdw2zf3BEjfCxUkH+8HQ5z4wqsVUOlcQv1ZES9nXVOCJUpoQqzsYXpkCpBC3eDnedU13bfYttGRNhwt8pvH4+3Lz9/bPvLtgt4zO6o2XnaNuJ+K5XvC+fQ+TxWhYAhJL389gfm3GlJxGbPx7fj/SNPfnf7/u/+5//N/+Jv//Y//ce/f/t4+/2Pf3j7ePIe33//cr/dh21GCgUEzKoXbmGx79dxi6zsTlLdXOXZZtZdsGVwM19LAF1RsLV2WYAIXHf7f5oOw+fGd6W4LjO/7PPNklj6zzzO43ic50f10b3EYateHZuLAwf7LIRZCgAJsbvldEghw/YyUGiZGV7vY7tvBpzHUreE7p7zmOozOeDbJgSMW8C3PajM8zymji5ijF9R7Ov57+iup7AHwmdWfZw0OKxnn0ej6NGkaO3u5jLjWo5UpcO2bRs7covj44ig1mJS9ny0qtXF4doUt+2+uzs3idvWhrfjvQrPo6CorD7TliTtrgQKDVqRZtTM2eED2XVUgcp2+tg4HGaegkMlylRQO6uaBkq1GH97oMAwhuVZLaK7uoSOACkIalrVBSQQrY1hVT1PjM2re6bOI6tkZOxkmNhaLSlu5oiNZszWbKUGdje3TdfEuMB0nYfQ2z5evttBG6dInY+eVdmEOcOGj7H5PoamzsxjzsfjkZkDoMiyiM3NF6ihWzU1nIZhRJdl5sw553r6y6guLHC0B20YQ1i/e0ETNaEhM0HZJXST8gGAbFggXuL2uo373rXAz0lTHS3CB30hI7LzbAnmWlt4MVrqdkNs47aPDehquJswY3OOdVUzobewdfovQxdMXdNHGL1K+VwZsaZs24ak58dxPB/YzLeVf5p1XkYVMzNfpH20qLP6KLRUPWe5SAeHSpiztvvGmdZaKXA322XRXMMlsqzSW05u4uu4fbe9vI7tgL8fT8hnHpWzk3SvK0fhrZqFFabbNo57RKB6BTI052GAUW42yO4+znnOOmdrTjW2Hb75ft9jH4TRvc4ezj3iI0VRi9+ZvcLtvsUWHuRq+3j/9nh8PK2nCW4xIszpm+33fWyjZh1Ttxt6dh9FmHr5vxNt4WDWs888m5DvtMFKeIBkncIDsdP2GFAdNc80NwQJSXnO7EdFsDjv37Hy+NN/+c8RMcIjtG/x4+9+KObH85CgWeeZz/fT96GamGU06Vl95qXa75A+vnwBbud2/4j5f/kf/u/vXz/++i//1Sx9/frxZ7/5s33bCkjMDWF0C1MpaXVM5wpyNc26sruychvBX2WcVb+49rsGmK16MLrTLuVHtk72y/zza0Tgsy++f1WG+CkGEcvC35WYs87jPI7H8/zIOlp1HUgUqIZwVRMDNC2Y0bX1bDNmn04LFbPaVoWEaRtlfcyTKphbt7J62U7H7ebDTY5ccpi22LZhxO3pH49v7yXpaBvkwMI56uqgUZ65VtA121wi2ehJTi1ehd/MaMaiue9Gc52GkjVGuhsmOE86HVjVJQGHNTDcho/95oNuRgwJ2cd5HMchh5VmfRx2krPcTUz6AGBU7CGVGgF3EifqVKa4hbujUZkzxN3MRh3VQpkUNKOZ2RANi5OsuSqXBUKnUGrBXNwWzRJmEL1mzrMLtEF0Z5VvIzjyVM0qwZwjOF7c3bltoGeqNNWH+5on0U5uYR7mAtmZmrOyoWyUkeO2ubskc8w5uwBzmnuMEWGxmJ8UdR7zPDtnWyMGkAkLVZWjKk9ldftwRwRYqsrzzMrjzJkG2WYK1royGmnw9bWbQqEFThgMCyuCFtm12oFWdTLgxoi4bT4s58w655k9C4tJ5JSqJ/JZWbC164zhY3RikQ+XVqWWoet51FkeZiP23c0BdCHdB1cxjjDPRGbQqZ7PZz37POv4mOHmijxPCfM4qcaD51DcV/ifa6xVZYNE0KOPxDQl1FXnRUqKoAUBcFjNlFopymHGNtXoLDRzLiAInBwcAd9te73te8Q5j6Pe3789P46vZx37690GG2eW0Nnmtmt7ibHHuPH2spE851mzzrNUZ8CQadVNzprnkfNZKKlghrE7Cd8YQYCV/f7tvbvmTLHz6GTX4lPBJNnsOmaaqH5+PPOY85nDzWEtKmsRAu+vt9eXex/d+TZzxrqqZnZXA3DctkDrfKZo2R03227DIGHL7Dy1dL/NRGWdmnPlSZhnvh9tplIbZcC+xw+/vX37eT7fznzMp+P1h/32stt2e//4cjzb9tj3TZtVrRtMPR8nzWue1brfLbBp4jym+8jM97c//Z//T//H0P43f/l39/t3+3b/w29/78PneXz5+s3pv//t7358vdO8qjLPzR0R1wLYIGGeE0BTYVxciFW4bcalXKobbr96OnGx3vj5fvj17s9/cvSv0NYKen9Gwhpqoq2zauY8j/P5MY9H1SH21VdWLRNXgPfqIFiTyhKEcGXEyFZGpfptLjzDRmQiJ+cpGd1CsthH7HFZARtInGfPeWxj1LPOxO3lVvOtr/IhsoHZdRnfQDct6eZsqXNWbNfxMyJILjydCaxleY9937v07Hw+8mw93aXsRqfKje4MbLuTVmdZ0BEoP7t9s0JLeTyO82wJ6rOfJ6a8zcN69QAg3Y3OUtnG8L1mlyQ5w5TJWpWXrW54RHgqiyAVt/DqngV0bNIqGOoFsLz2kzB0imwGfR8xBkqpWgyTln0SQpvD3VyqAov0za3bnBYWYTAtQh+qVWVhI7zNu6BHrQR4qlZqgloDIDzC3LOg1nHOeWaEbyPMNxvDYFU9U7NPonxzq2llQbDkPkib58oIeJPd8iypJlVnn6U60VmgzCio7eohzdL6KMylYveCt9JgnaxqK/Roc7vaemXrFUWiziz083EezxMNVV+I3Oo8FuUAMJibRcS2WXh1yhHhYaZzHj2rz/NxqHO/2Rg0Z4Qd2TF2LZeRah4lIJym1jyVPJ/1fJ8XkoLIeZZMxHbbslNslVZwRyt/M2ESQqammwokEZ45t81eXl5iN7mJVObKWw3QRCOq0ccMsasr28N8bAJ1COSI7Xbbss+P+fanr3/6sOeMh4eG39x6NsxFd5e2PV5/uEW4YDaiupA68qPOacK+A0LOWYWPecyP7FMRUMDCzLyv492qMDMfH49ZMzYT+zPMBJJojBHudryfdc7iaiKbwzXCOLsm8lBJASA7H6msfGY+E1WGVonAuIVtw/hJTmPbLhu6vVgdLZlWU4YDrZziqVLDLIZ39TyknKuqxByDHmQZZqY54KiG6Af7+eWn5+OwET+87rfbbkJVHh+Pek6orRnuR6VhOH0+1YQ74Xo83+/b7be/+/0f/uwPLy8v23c3DP7D+88//f1//vjT++9+87sffvyxierKPLPmFq9oLQstTIuFDsBELL7CFlB7uIEWXKIvrgaZfxT9P0//608FEGpoSZ3qXiBcfr4qLrSPWFX5nMfzeD4fz+f7eX4oT6KFXvYHSpBWQdmiAkkiFypsLV97XcJi3PD8IhkczEPYLZMgw0KAhzsDkqjUrFSd6BN5ZL4/654Me358ayWN2z3CPWfNbiUXP/1SrrurOzOX7a9n0y0297CWjFa12Im8b7eATaNo6UD7KVQZ1Ott2rJx2+737TwKunA2x+MJNE/aINEzWyUj8phEm8F3rnhTVUfQRrcwbtt+39Vdo49nJ7CoHlaNzTxGx7a9bu4wFaznTC0/x6KWnzCugggDzdx8o4rV3ZE0sxE+bnRmLmMOQPPRAsxhN2NsaPREqWMbWLlYVM4mzz6UTbj5ANwtRty2JpB9dVIXVGg0TJ1yOGnwUd2JWbPmrMwK9xEWZssM03I1cs4xwMuK3LQBTTWrEbFYOkC1CZa97viVKll1w809fJgZLsNjLnhPM6mmZmMRcWla+IWyyuZFyZUWW9TaaUKez+5Dx7PykWxQomOAaHEhqAPhThu0cHPB4b0jwje2VHkc53lMdflgBMFyH20m87PRjZxnz+XmQ7Fpctr51PlRNWVij2qpu/227WPQITnNGOKyToPZomHcYr9vmmg1by4PpF62GOFjbNttx7Azz1m1QMCARrCASw/p643iMHcPYT4E6vayceB5nm/H+9fnL097HnpgqyEse6ZbuMHcX767//Dji6THx3l0PufxnJ1idu8GuJvbcZyPx8zEMqIY0AN0wM3aunQ8M8/MrKppwLbF2Aas+8wzZxVo7Kp1C11cuWxF+DbW65NVPadqIraeRz7yfR7nx9tjVqNkQQbG8O02YDjPapU5Ikxhvptc2+tm9MdjVk8bzI+EYM64jQgv6fHI45Ths+WzoOBxzI9vR0m+MW6LIIGP92flnNL3981Mz/NRna06n+fMUpiUgI27W6x6i6Zb5hQ07vsPP/7+z/75X/zw579/x/Hv/uHf/78L72+/nN/efty++/2f/QHGBmdOVe5xb/VZuY1BY8++UhhcGo152FJaVszFeBkyCdjl/AnAfzX//Hrx/3wxCJdov/7rV0qgcZ3w2T07U+fjfL69P58fOY+lR9JsUQlaqzzmsgTZBdLX2iWsLcRyL4UFLOBmq7KjUzB3I1GxDd98Zud5OjCPk/SWy4LO8zzzkR4Rm8ewscftJdwbz+XiCt9usbtRukDCoDmDlRk392H75uauUyodJ46p2GxkfTwLPmi47/s2BtA5kblazWTmm22mYUDngi+MzqMxkc1mhJEYUZXadlYTC5BizvbRJ422rYXuMKHp3QWDm1mpPAi6b7PMdqf24WO3fK+jfHGeBDdN6UDRibraZ2JZ2nP1uZvZGBuKedR51pzdal+el+Hb7n7bBKujKkWws0xoah7TbZ5o0iUDmQP+4kWdKLC7G5UGU65cUFs33damtqays0Vki+vGQZToSzvELGiWdbNWFGDNdhR8drPb3S+9o5vdy8z+CQ+C7YwYIH0Q0MIp1YosXs6F7oZOwQwGugBqBSY/oY9tgIn0mhDaXOrS2SZRUMI3+Kc3Ogx0cA2yqSRgGraP8C5V5vE8Z+X8mNY9vjNl28tmziSSbOqquzdnZ/esxS6VQI6NTiglwneN2GIfdXYm1qoXVeu1BdBcsfu23xfZTuYUbTOYLGLb/Lbfdt+ffVoE7YTLl+ZKSczK2eVXwM+KNtE0N2t1V/XzyK/P96/H1/Tz/fj6rIe3pbu7IbiZImIbt/t4MW2zj2Oe7+9vU7kaYjduzvIwsc9jnlPuVwQJTndWQ1SL+XFe7eDsfYt9C7/HcDb5PKTOLl02EHBsDqMZbmhq1WILRAEMjICZnx9z4jznrC4bgIkDIRvb5gPHM2emDSi8AFutIB6xm9NPL1S5mxwmjtu2v+y3F/v25VxQXDpudw+Lliqruqe0Dbe4bSNsBxKZ53nUuG8e/fH+oRQNz49nHj12h396yEUZegomDwlyM2y+v9xK/Q+//L2sv3378vz6TWd/t3/33/7L/82PP/5+dp3n4+YxYoPhPI+qbnQourtVqx1+nbPglfai2kdcio/ZiuHDbCHV/8nV3vSPb4LPZS1+XSPgs5gYainRiTz7fB7P4/F4vj+f37Ke3QU1jaBUov/jVIFu2a+1k621d4YkGTpyYtxJ8zAHupXqnIdi2MvLzSxaM9VdMolUbD5etuPn95mg0W4WW+x3DzdZJhrRRrAbUanZc1EqLk9SquHitrk76GT4zpzy7o0Ns8djhsy3oDPMRwzzdkOMoW4tD7h7nXk+53mePrzyZDUcDXg4d3fD7FyADS+TwdtFAmmBMXy/72hWKbOyamYZzUwQMMwYvZpjEueUWKojK6ULMOoCYWdCgtG5GZ30pRRIDm4W7ujsI6so0WCoBhi3sb1s7gaisueRfWpxrwRUznxWNmzlGEhZ45CXkEqfWANeV8CVqoJQkszNfAA9j8qz1OZwi6W2t4k6sxtIstGs1c+OJfWjsxdEDi6ccw5Ft5SFItaXyNhYfssBdxvuzlatN9Ci45GIgFKf9oWGfCwk1rX1qZ5NA3ZbA7MK85x2LVLoILnCY6Bfm7G1QlMrZ1UjkzSz3Z05j/M8z/PonG3PorELJpiZqETJ0BADzELVMlVTWKBCuvluhU6nB7fXbUQsKEgfVc8JCCE4bEPDxvD9vo19ILtspUCpljvbwH1sr3udpc7q2UqaiFUL0bMqmT7czc3gq/43WGdxM0s7ev7y9uXb8+NjPt7evz7qTWMZBGqeTxbJO10YI/OJI5/58fF4e56PhmjmwyTKpBBofh97SE03cJibLcI7jC3luXgvioHtNmzQhDkrq47nfB4qAcTtZlygJRJsctl32YkUfbP9xTrbyO7OmdUy9yVd0s3dbKy+AuyvLvaKefTR4+YBdmZ31nPWUZ3Low0bJtTj0YiW9+1+2WY43KUVH4sNseQWZn1U1pxVMLD7eD8lOA2zOuk+xi6tdiIibkFjxSogKhAy7W7C80+//Mf8h/p4+4oqa84T+7/4O8X4r7/88Tff/5aLGmZ4Ps9vb1/WoOfu2ZUz3UxqwEm2ptFpZmGf3AfBruu/YeCy/1/L6H9y+n+myZbm3yB8IT9/XQL3orTOPD+O58fH4+Pr8fxWeYK6IgfScvYLgBaO9BMh5FwC0xot1M1hMQIlcmhZmnp6ztlAbLFt2zYGqT5QLd/Nhm97SGmbAj7GRroN0gmvQmk1eJEC5/FcukHXasYwGqCiU10NnWePESud0zA3QMiuqqaVXzU7XVndWgAGA6uVU1U6nrO79ZhGxRYxgBFys2CfB62cWH2DoGtS3TCF233fttiqUOc5j5qztNqMVNWy2Nq3rGqc/Z7VmE8FRDKGgUNG1axuwGSQaAz4aoQrlAJtQVoJlChw20ZlNRAjYlXpEl01n1lH1oShCVW15lWiQkJdsHVPRT6lzPWdEeHBrkauUbMtYA6y55nzqC5z0BysFcBgV7U4z66z28CQb3H9s2W98LJm5qRRhVaj1RM9pa7KbsmG06FtWRoWgak6U7NQaQX3NXia+coBkxttc9C1iDqfk+5SNAGghVK33O1zL7y4DlAZjUu87IISddLgXQvz9Z7ePWse83xv82hhODzg5hEb6WABRbU0UZPrJLphvfs7F4CsFWLDNpebwgdD3froPtFZDPhNMDeT78PCkCXAh4HQJAlRMbjvAAuefZw5p1jmyyUCQcWGMdzcHYCHV+rMMncfwyyOmeDj4LM47Q6fsGFSQRJSpUrr4DEfhZPpRx/P8yM7YxhN0ASahA/ftqHu6T2PXu0OrTZbTDw1ICdaDl+z2Ty7DNU6s2bJNhLwsG3f3NGlzPLocDdGd18YmQjnWo9JaG7msJ7VUwijO5xZbY4YVt3dXETKkKFiftREqZcY1UaYgy55y11Vc+a+u9kCv7cq1dr27fYyANbs4zErOzuX1/3+ch8RlTrzfD5ONIACoCd8Nxq2ffObd+kKoFevb9us/NNPf6psdD/enjc3qF++/8GCj29f7Pe87bctwh15nB8fb8/z8f33r2Y4zyPPcz16PtyCRoAMY0TQTfjVNgbiwtDzcnz+ugD4FezWV1nU9W6wz0gwBbABILPzrPOYH2/vH+/fHo8vOR+rPK8/CaC4VJ41Aail69+69KC14CGMpBQ03n8c56OQNc+qZAHbjv2+uwXEbfg3pTlsMxjOOkF16LaPfYx5NHDOE5NFFvoKyiySVDW6AMKldflSggMY7AQmHufZzTF2AB4DFOukbOEQhm9ZOs/n4u/F8EpJOlKzwSpv5XFygwdjs4qSzZytKlIWziZqDT5XJV4sGwyROc/zzNk4m442Lud/Q4WjrKnsrn5OjgGZDzdGMVMBmkXFOgvaVVBtXY+uBjqCschrFpUgsWJAdOMgrbtV69OewllMkDJeLKkrJWvAVSqkLi6k5iIo0NFJo7g6oF0K0awluMdtsBkiKcIisG/u4Dw7j6wEArcNQQArTNYLCcPAhaqaVWgIldBUnuiCBBaaJ7F1zOxZaKk7s9cba8Wi1DKKUq7qGXTQfey0Ks3HqTpJmOnKuFAephIm1ChQpBnYXqdAuFMmFesQi3SzRtV8Ps9t2GLS7Jt1tawLGBFNT8FdBg/TYiWWmoMYBhdyGfVWEq0FyIGgSHgkMOecH2c95TagNFqQvAVuQVg1OGiAW5S6Z8FpIXg3ZzOL2Uo6bLjBrK1NXr17RDBgjDCzx8fZqT5becL17fjySJ+az/qQzXELUe6WlaQBba7S7Oec0+SLBFxjmIWAqqp5Yr8vgqnUWWctBNjsJuHmFFbQu7k4vDncurJTAotqsCRzD3N3o1lV1qyctYmK6GWrmhTZ0Hycj5zcY7+N7RaszgfOY3E1NLPGFcfVKmi6vrTGecyemIsKZYCw7TZuLnDsXlnPmS2MGPfv4nzvM+c8k/Qz67uxA6zGrJqVTtBs28fYzYTy1olWz9Rwi+H3+8CQW8RgJpKwgMFQZsGCVKie82PSEMFxD1SHs3N64ocfvnMztA6dWVl9/vj9D9t+M/L5eHbNNXxvEbBlA2HE8v2Ly8NtYQizIPxX/edXAgQWI+26Df36AiAvoM8lFnUps/vonDofx/P58fHx5fH8Wv1ckB8au3spjpfi+ink6rKYNo1XMcDK3gnhzvwowuXcdv/oGeH7/bZtt3lmGc/zYYScsAa6ykiLYWO4kDR11nlBz+GADKrORCV4AXXRjVoFgQPbbWz7ZjKYZ6q6R3CMndTMonlrljqgcz6OudDSrROn0bh6DwzuMYDqyr5C9UuWrSypCxHOAkrI1TGJtY0xi5Jy5uM8z0pRHKIVR0cMhklavwKjO9qtwopyllfhBLI1hu8xhlw8T3QReTxaXSV3QxvLPahk1Gq8Kp1NN3OvFJQzK5Nu3rtoi3ddpqRQCRjWCl9SLypQmAw2MDY2VNXw6+rQbVbeJGnuMBoSfoWvzcwhzMZ5qmFTbVAjIO9EJWpmXBkVqrsSmq0kqEohCbWv7TGbg62u88yq1UhBwVdSlxq7C1bVNaFVmJfyTRG22ybTMXme2auOCFRrgb4Bq1obigVYIMlcYpFZpVC4SLRIUd2noUXt90G3Ovs8qgtylMGcR6b7AMrrsJ7Q1M3JEO1ElXoBTri2a4QF/TaIsY7CfFYeDRlNRqiaCDe/rBNsaw73QgnqThoNrrV7vyqSL3nXnG4WtCVfxPDhXCawsd9mf+Q8wO2j3qqMCYaVT5DjtnUnIBow4RYIL/TMYspv5sMtvLpASYBz/87221bd8zwf70cV3XzV90SsenNQqnM2ryQTjg6wSuouI9GxwcAwwlQzlZW1PAeuZqVyonq5hLwhj9j2bR/hYLXQdB/mquxwp6BUnqsiCTSzQEM1ux7KRk/EC8YO32ne3TgesyTQl2bS02jXRpQki8833V5dKKCNGLeIMPeAlOd5Vs8zRXz/4/37376M4Trr/e3dhs3Zx2yMcJO5VbdqtRRXZ8VoC7c9RgyUzo/uifvryxZbdR99rnLd1/vrd9//COB8fKS6UQ5aqK3czI023MPdfL1D3dw8DAHY2uh+wh7+f9if3biOOmBJRPbrnKyGSijUrOPjeHx8vL398vb+y3O+Vz/BwtrmkVxn+9JtBUpmtuqbVnYFRK8/klBHPrtXdYkJjv3mHjFuVn1q9seclbNT8eKrXMCGAU6pNOuoPAUp11nPC4qkRn32/XqwWyhUQg6GAeo5See1mEBTFmYCQjNBYEB9HnO2rxRytUoMb65lJCBVNk1+p63ftRoSrlZz2AQoJXuqE2py0N3dlccxsyt7WQ4QbVRsANO8K0Fp1R/HkAvBZeDQnDgnzGmNyrUO5hQEM4OarNX/0pqYmwHiBKs11af8hi6BKLLNsLmN2BCP9wPhi3NMwxB79hreUOiVxx7DnI7r7rzuz7RGa+lqBRl7IQdFtDD2PRhoVqPb22ehE2mlLELZVZm5zMIgOqvEPrtPLb6DGuswgLeFuRscec5qKWUwvy1c7FpMeDXo5mPQVc+zZ+FsbahTPTrow6088kx0mV9Z2iVcguRVR0AYKkWjLRqkg0IdqWofzGoboJsHeSPYcMhNZ8tNgVL2rLLuzsgZaADbGIY4W3V0troFyg0WxqFx22IYDtVz1rP6kTjoI4gCLHYHjXKkFcpBd1DFrHmeWuRqZRbDotRVXQkzVaZKmw0g3LzXlX4sG4qM5bCkVfVpE+5ujmgjWS5rH3spLbVEbUGzqigzrA9vCQlthEQuloaq8fb1mBPbMMKV3RLCCOvunJWptQE0sU6t7OEy2tigm8ylyqU6XEXQl6uWZylLa6UpEGFBi4hwQqrnnM8yN3Qv+wfrmr4dpmyuwqlu9dIfoFfEffWI9CytU9CGjRHehkkdVWfWbPcIH+FbZj+PoxoCzCy2MdzUnUfWrJktcOzb62+///71NnN+Oz5SrFn92RebmSsoU7l6tFQt23y83mQ4clLWhbYer8MHPp5f86zfvH7vGhH7umg/z9McEb5YxVyvrGERdF+XL/h6C9iy/cT/X+L313fAeqKXv5DoFZb1haBb6995Vp2Vp4638/3b14+3Xz7ef57ne6tw+apX98wSfCj15wJi/XybsOpexopePn9jPB4I76oTO1ZacgS78nFUHj1nroUEZ26vK/vAbgKdZ9VZ3TDADRYIv4YZOZxQYnkM6SRll08Vx6x5FmFu7rQ2Gjc4SFpahL2/pVlrdrjZ8qK16LCNPtzcraHuLJkD7sseunJYDUdVz+6qliMv9OnaeRpVmdCy7LYTNijRhvuw1d0oAq5Bs1XjPFVFtTpRKaWM0VnwiOHN4QYPuklzTmPXWYfKhJIAlni0imrrA+YGa27c3BsWw2XaOWhSe2hbAkifrWyWZWG4j20zM4o1aw11uM71MnaDKjVX9h5rUe0AetUxSu1x28e+n8ch9syZpcpWN5s+aBthROF8FIM6RQoNOnxfsolZuMXadKFa8mq2e+N6AxgMPsbYHNA8OnzUIGb1UbMmz+a+UVjsWTW0hmSw0eJKly+vqrpKJrkFfduHG6V2ILsh0uRhcMIXtNnZXbGAdGxRpBt7Tl8K0kSLNaBSt3pS3GKHWwPtQdoCpbHRVZXHiWKEDSflVabyNs5TNmR0QHXWxffvXrc8jzC4RHNfqN2q7ixWVWi/+2WKVas7GI1W5TpKIHYVHXR6+G3srnHUYWFBL7NaKlb12rsYY5GIYVhNsxeKL1OEGuY2DA4HsIJrBKoqj8z5mS9tdCsTENzljm6h1nACI7saNDXcbbOhs89WVXmYXMoWYLutlcCZpjmfz6xZVhxLNRNWOG67jz6Rx8qySQ0LcrgZ48btu/HJ+UtV2wRTgKq6n8pn5ikYb/ft5XZv4Tjr+T7hFlt4uIcDqqp55pxlFqS9vr7sMXLy+ZyP44h9M3ffPLMrU+f5PIsy0eIWNHfSIxZpUBXdvP/4w/d/+MMc/mU+/vT3Xzfbf3j5YWw7sjurtbwCxjXALtUXCkcMNyjWFwu0NoPZskb9o6xvn6f/lQP4VRLlWhkv6+eqGG5VqmbNZx8fx9uXb29fv3z99qfn85fWU31Zfdez6XYBpLkAKiuYZVeZPK9lA1cZkYDYbnh8YN88iNg8H13zbLLOrtloo7dvxtURt9p3e3EfUSfM6GGgfNB8vV4w7q5knevVJK3mvF7uC12JZfbYtRR5UrOn3Okisd2D6EShuymZYoPvYW4LUkBZJ3DhgVdBDpU0gwrIi0DAVeTSMLMYHOFjc5j7sNE9/cjMpdBG0NjuWmUBRlqUViEssBqDlzN6GxYbaDb2GO7TXGbb7rt7++n2OB51KFnkWXm2wWMxg4etgi1BMQYBuMmde7iFrwXNzOwWYbdm9YB3W8PXSFkp1cxlgqsrnQoaDGMfWMvblISVtrWzcqJLNO3Yb3uEG5z97evinVGKzccNPqyJLjDbYX2K7dibN4sbOQxGjwDYBYfYVDsNbnapmuLYNhuxngptNUvRBLyemTyuVmhzIyxIDt9ii8jM9Zu2zdzpS3moqWOt6mhhHmYIhQtnHlh9wjTGGObeIMIo+vIXVo9YS2oEGd1VPIRzKrhYouHDbq+OXsF+CaqSmA3rZTly8w1iAmoxs3uCw8buthzTpzqzZvrmscd+v3EMVdfZU9nShQVc1x+YqkoAWC3HZqaceeacc5JUnpWgoi33sa/n1T1WXJMkLasr6xQx3EmgEkTVIk4uDvGaid2G+z5GQ7WMHYsTkjkJ0MKFREOkWqGrj3zxLUGoUFqgxXVvu7CRqqosgeZrnrYxhscql648Mo9asQGC1V0fwAADY2wYprPkqMIYti5gs2qEXyA5d634ESorVcoj52n9QJXOZ203YG8ToamaZ+a4bR5uvgomKs85z7UjCoI5Mx+nRh9n2Yjvf/Pyen95fpzvb8fH43w+m/KX376MGFDN5zOLXWIxS1Lcv//+D3/2z33c/+Hnr//Xj//xL3744e/+2V9HbMbIfgb84/HYtvAwoSIQYQTG5ts+zCzcPZwLH2FhFp/Uz1/V/1/3wBfvc0HflowiGNsuTJDQpXzm+cjzmc+3j7evv3z98tP728/H+SZMGtUAe0V717vgkvm1Ni74nLKBbpotFHSYd1X4iNt3uv2wD8f2gm/nnI/KBkoQfJ3sQZJ5VCVo6z6NLvl6laTBV/aAICIMbVUCXS0Vckor07wE1CsEKnV3UYOdmVIMD0CscKqbzpYYdLMVH2WYB8hWyVzWpHqF2lYzc2PB6GBNER5XdGIFaWLQSRE3Dw4d6qcyuRw7XdmraZgFEV3AylOvR8VEp0eQxmW5GTPRR6bMB7b8bMMgbRGqr05mAxzmpjXJZNHdaglg0W06GyvvsSDIiaLc6R5obtsQB2irmNiw4ziPfC8huKRr2YhxN9S6mqillQHo54RShW1X+1EtMzNZmINaNCqMNhddS240Jxp2c4Rg5sNtp627IUuiqqQ2ho1lMFd3yxRjpXOhhgebMUY3SqhlrloKpCjz3ldZRsAHFNGnws3CV+Jl2dQ8OM9WoKrMMLYB89iR1lKtiQSGUqu6tNC79knoanMj2M9KIost8ai1/fDeBn3IGjIOQeecEBgmkO7bFs/zLG+wV86uD3XDyyRdQ1YWgK4OGx7bgt501nFkdy8rhEkAYvhw78qenUvoKqTNyjzOxmLCZWV25rHzJndvkCaRfuX+0a5eoD2uDpGeXQWh4WYGLotyCgbKYnMSbcp1iXM0tN/Hvm1ZeT6Vs224+ei8YGPmlpkNaHaDbqTRbK0WUZ2ZKWJsZPi+D48RZmpmIxf4CdjvjhaIusiDCLIBnmUuD/jg9rqFM5/dR8tNWHAyVMGEntd8O8+uiTwhyBw1oa4jn6pZma2qKZrrRBkk5IpZwsMjn2cJD4mHgfzx5btXv++I2ZOpnuqkb75vMQLn0+r0bvm2GX11ENNuxP0457/7L/+vf3b/8i/+u//DD7//XWz7+y9fjC70FmPfwg1TvU4Fc3Mzg+IKX6y6pYDFr/veT/wD/gn+4WqeJiD4r4iIS8IHJcwj66z5OJ9vx7cvX79+/dPb20/H8UvVozWvggCtbfI1P1T1VR/cuqx0uvTh7jInRKFJRJTGd5uZuel4S1VnqR4QEDs5nLFQ6uhC5bVaXGlYwQCf2ThqHYsW7CmhS6qzG1KLAnih6ZyUE1IELx5DZ53nFDxWhks018qADvdhK5ZEI4jMlcoRs1jVal4rdFcjs3wzUXCukixRWPEq2Dw7Tdt9z5rWrZ5XL0OrurrQDUz0BAhzBygYRBkZ9HAOX1c6GypUnZVTKh6dCapb3TlrzXkkItYWU60s9HIkgUZ1yPqcSvQVFUXBkFXLB0CgET7gg20Fdfew9YExYjgV7u5usa54i5RAg5HoKp0re0E1zXUmoAnjohJ3o623m3kADhsU26lYZhuCdxhFFwJLtcgGYF1dolnTsYJ5TZkPj7F2H9mr9plrK5dr8wkPQwShBhNUDKerkTS3PUij0wk2ZwkiaUaocJ4VEeszEdkBymwDDVJ3VmcLvBIEgGCr5F7rUV/u2VDIlBrhDJb6OBBh7mPOmU96WNN7CjAMcXMLxm2EmYSZlWflTF7NMaB5d43b4DL0Ziq7zu5T3d2VEVQryHVMduHM7KbDnufzKs0Ge4qCnLNAs27karJkCbTyLkBdvXDLe2ICtS4+aEHUWTKTVLXmIlN2LchBd7Nt0Dfsw1++28xoySZtX44Uu3Bsvo4wlnRmodqcIyLuburjyGwhEAPbHts+xjAzkDjPrJnzObPhg/t+UQNUPJ5lbm5reW+F9nuYc7u7uwG1cbRgxqrW7JwJKcI6NU+NbetE3BTUPKuh2TXfPmgotRkj0OoxQlTPS+swk+bh3cfbUXPYiHEPj82Ix/NxfDzO92M+S4Xh8fh66mXz2MYLvbTfNlGrL7fn45ef/sP8wF3b93/+V/t3WxvePt7Q84eXHdk05DkryoYifNs8bO1Ctogg7AoC2LAr93uZAn6dA67zGQJqkT8/IXHLEmRoAd3ZnTofebw/P769f/vl56+//PHt/Y9zvgNTaKx2eMiJVQGGlputc2HxHlY0GWuiVJNcUWESQWGshsnSnMhFcDG45Ou3saLJWnIDrtwiqNQ6c5nSbJ6wAOP6G0rs2VUyLtrfJZlJsgVEhQzV6qsWbdUWn+JuXOYKrNeGrSDbYph2qqas6RC6e4VUgkvdipuDMIa04IWATL0CcJ7vZcPKZgqomWfXcp4bZdZqruXesmG2ddv6OsBkw+XGcEE2Ynk/ulYCEPl8zoaFQ9Vqj2WztaqmUU70eoWKAlHCWYcgthpohnUHff2wgFotBiH6OVV5zuwYA9FdLebYtJA/BtGZtcYtmjGaaqR6Sh4AWnRZNjxxRT3o7URstt1tpdk90CWv9ABm2x5aZykvpGCpUdD62gC9ii16BRTpiiyn8uyCQBo3o+QG2yJxUU2E9SKbcQ8LI6wSizQI6+UQN7KyetaSPxfCpCvPCZK57ENOrsDNVZ0kYvVVsrtB0h30zmyzOmpWOozDhvtQpDwPUdg9NO18dld4oLLX8j+Z407Gtt1icydNrXPW8zhRPdwXnDqzzOHDs1s9u3pVPKqbhITwMLOGamp2ryuaqrPmOZsOv/pFbM3j5uFtq7BD6C6aD3enttvmDH0cb6xkWymxNPq56kbajOYucf17OrMatDYXDWPz7Ra0quo5T6EsTJV5oAoktwj3GLEfM8sBLhCbGawzaRq72W7bHvu+mV1y0zzPyjqO6lIEt9twazP6GFWCpUA6lQ0YVSTc6cC17QSdVrNLqKrMHGPp5qpMOmKn2gjF5q2SK2eTGMNiGAPbGJA1utnWqGo16tR6Nc7qfQfAec431ePjePyc/YQ1XvYdicrefnj5/vuX9+35eByXiN5ZR57n2zQ7nvabf/Fvfvj9H77+/OXn8cfbb/98326EeUT1OfM0aB9mMcwQW4xtxDZoXKK/4VeF1IEgAvBf/T8C1gml9ecrQfBP+RBifmpr83E+Px6//OmnX37549u3fziPL90PYRpZaDTZDdLWgbnWByvp11qT0VKA1rZfn1ZQN0Y1mU1Kq2uDfblRSbtQ0oT1CqFBxsbGeOZEobNN6ENdizlAM6KgVjUoC4i1sj0yo2qpXaBp2czT2d0U3eSXQ16fnQYmk52Vah/UihMnUOzZC42tgA2DuOh2goVTghHV3aluagoilRY+H5WqnAKkK7ABdc1CF6+GVHIBGEBKRle8hm+rzyYEXbmqrouVX7POJrw7ab1sMwLazTbwmgCaJZOGkQVVGlBdWNtpLdKRE2ByHkUbvWvOs4/F/Wjb+zAWV4+xua0D8kpZpxgxzG0j3ex09iyLFfeRh4UVOgGZy0Igbbdxo1k7u7vyzJzquSTcXtbO5Tda00zzslnBrnzC+kYZ2X2e53KWgMSyLrqDDqmvJmoVWFVpvoAkVVnHVJWtpIOsbYWBI6iuc6IK1Bh7ZylobnBFOSiPpSVmk04q+zMAY+YRY0fbeL0/vz0KFhDO6TBvK3GqbYUXe+Ys7xa83sssYzht7i9O3+nmlEFXR5lTFi64wapShrFM310zK1Or/FlrDvYRNFKl88gF16JLa4/b6zJ0UVlEGW2AksiSGpV0Z9EY27g7trGN1lney49OqLvYlivDSGKJjYv7k80q7rYHETRj7Cbm45HzrCyYg1IeyuNKYGYgwsfOOdGV1WUGj6psuobHcqCNCN9QqfM8jo/ZhUrmbAv6oHmW2rbhdzu/TQ0uuHSr+5iLnFbW2UB7dddZbrZ4ROvoW00LMWg7yWL456HF+xZA8+O0YSPQ1ZfPuS2nOKGzrVmzzKPRcMTu+z2Gcx7z8XYeb3M+bMDv22irlG/7/t33Nzdq1jxSjj7POp4okEJsrz9+f/vefv75P++z6/f/s2F+s9vYos5n5TFr7rdVW2Wxxe12i4jwgdUAgDAEsQGuf4x9/er/0WXSvwogyYv5Y1fN5CqcPDuPnu/n4+3x9vXL25c/fvv2Xx6Pn2p+SHMNQ+ujW43Qn/f65QteG1isgkWpzK5M8DUrSxCigPk2/YVE1aGawgR6YfZWvB+9COYNlmjIqGVWXXcEuEAxCKBPAVhAIoulwqObJBssdC0xxw1Q19qsimwaerEDelkdSdNKbK4WhCvOtYL+lKoZy2wO1TVfoIlmU8jOqToFoWe5GYjhDiLnuRg18EGYstHKavRyl67wBbpVyobFGOHDP3NZXShjZs1zUbJ7OY9iYHUQm6/CL8ngxq6uElQShtPNW6glpa8XoRmdEDALIE9hNmjdUHU3O9tBPE8EOMxjnb8z8zpOehkD26Lp7uZtzHEzqdalwEOaU5AFY2DlEhENqLJKyNl5Kud1abAgLNbNWlUSLGi+MmlLnkH3Ag0S1fVsmJs5jatPSwWsbr9CSmJL6yYIYCTUp/Kss0TYuv7BKGNYVJflioqXLqiIwgLGoKkaoDd7EUMLuJS1VTvMiAgZFZh0eKkpqoHSM9GUDD4k5flsrPhXV1FuHcaxxbhvIqqnMo/nCVivbSrJGIusllm2OdTV2X0K4EAMEE6ne0SYkTWRjVztEd3dWhURDnisctNrA6tSVveFOlDPcm7B2Ow+7EZYtg00HdloaUJ95OqI7c/5mGM50ZtQ3NwH17KnVecj5xOZF764T1VefeFQ9wT2ejzOZvtOp28jLECCEWGQrDvnnMdTVT2PM08tGQsGOrvrODV2l+E4s7uPMyNGd1VWpzyMDTYGrC7LIpBJYxXGbuMe+8s+7nbY8+XmwLWDVoGgu2YKFHnd9N2sClZdZ/fsmnILCxdW2yZWJ0q35qw6lE9dzY2pBrfXcX/dQD0+jvevHz0b1aocFnJtr7fxcv/hd795vn386fHHv/1nf/fy8p27G3Gcz+fj63M+7t/fzBWbm2GL4WYOx6X8rPt+6LL/Y4n7n+r/dfnHOp/RXKfaej6wbkzd1fmc82MeH4+3X758/frT17d/eP/4h9lfG09poi/QwdJK+Gknglpoo62M5/qrdvGsADNpVbAY1GGkmvXIml0TdCxfaU5lKdYz5oAvyQiadSQkIdzMUDTrgC3elmbW/Cz1DrdoGlEEseBjIrldtDGDlKLa1tBWWAQSC/gYjDWTGVs1l51BJtLh9CboDLJbnV0JjAYs21owsVOdqFkWaMAGZqU5PMayVZO96grUug4Rp8ReCRMRMdaHcwH/hFapbAql2ecCm5RB7vSAGUzGATjUK5rdLc0sCrG6IhY7qpuXucVEkUSxzkKLE2yqsueEmcgIC9IJsq2befIyUq0bOdFSVq3xLLKSUMJBkQ5roQSWgRawcZEVu7ufALoLPVlpPVukhcEgFAprzyNBzgUvwirrAfLqJry4IlCvJD8hUrx8i8hW9hIAnQuJpc4DyM5aKZDqrkUsIaWeyyEAJ83dwQDDWu0yZDMhrMod6SyldPXzIvY9bK2V2Jkq1JxEEVlVTdca7TzWTOObC6qcDlisEAf2W9yiHkeejwNn6dlVi5FEBNFql4gYgYE+q9C1SDkGGm2Y02luEWEOaxa9i4XuqWXwHiQ5tiB9FXCrVt1ZUWQXYSwDfIzXja+uG2eYnSXOSowNs5BamQm6xj5sWTuCV4GU9QiZiewq5VPzXHtUxABOoDECFugCCmaW3e5Yb0cz81hWcim7jTnn8qIokbPXSoYbVnZ3WQHUYECzSEwAKwNsYUtfAECYjV4r/12aR01V1nYf222M3SOsG2OPPSKz5qylOlKs0vk4Z/W2pqaGmfrIeaob3bQxbDgLq7+kkWyfTzsXm/iRgJlzzuJmrz/efN9j+OPb+/FR8/1kqs+0YX53vw0bfn+9nx9zr/2v/vZf/6u//m/+/Dd/bvDn+Xbmx7fnL9vNt/t+e9m2sd1ut23f3Yfb8NgAA67QL+G6jFr/OAE0CNRl3PlEPlyUtm6YaQXMzs6zzo/z4+v7t19++frLH9++/vFx/Jz9AIrGy42jfzSSrghlX8AHLSvEZQWTtH4dn4P9uuHFfKS/7udHe4OCJeieaEEeS5pdQ4QiYM46W0W4mS/vKka41ChmimNR7EGSDrgjSGsRRgu6gDBzl0mLwm4EV3qS6zZh5mFQz14fWltD7BSdEElvAu6iNVREqkjqJD2sqMxO1amlqTCMthgMBuPqKF/6eHf22Uu8wqr6KCxn4QpJkA4HpsjVz4U+WyV1W7WFzMll85EgreyLBBjVVGc3oGXCJYSs7oa5bbctPLqQM1tSs9v7kJc1KmvSrq14jM9G8+rq5qqGCi5uTz0bLR0oyal4HetfvVQa9IrIrQq6Zf6Gi9WQWLXmJ/W6DF4RO2oui9b6ALUCrQtLxeVtIUljTcqaTQMC4FzuMHfrqjxb5PV+DaPV8jYUoNIlgerTxe4uNoi+KqzFQRg5vCPc3cxVQKkTZ56GVW6/XF/GMei8b3enrwqzWWefZ82JSrIR1u62Dx9DBgZi9y2YlVKtri9QHqrOj0c9Hkcd7Q0k5lNykJK8ZtFjDEcoK3NtZ4Oda69FGLtXzUusdZlZwxsBpDEzBiwc4Nrccrn2Wi3R0LNUMFCFl9uLeoR/95vv/hCxz3rY80/PL08cRzcoemPfY+ybjTDDLGWfsu46W/C1NEfPA7m0qbiuneaIzUcYG3M2uj1IR6LdGfS1wWQpq+o5nxPV6pIm0OhcPW62vXhsQbeceZ6p6hXRR9NdgJubudoxH8VmJ3sSxLiNMGHXiZ7nHLttuy+Hd/bcbmHBeVSd2YSFEZwN0X14d7NllLrOxxQcFu6uJb1G1dnnMcNxu2/mJuP5SJm52/rOxG27f3dv2ONR+Xwej+ozvaVqbFaUudj603/4yXn/3/6v/pf/63/z3/3tP/+XG8b725evP/894gzr1+9/9/J6u+37/eU+YjffLDaDE4H1tMCJ8et9/3P3i8++38tzSyz1xmAXWwyJmsojz4/z+f58//r165efv33547evf/x4fKnzA0isCynZq3ndjaZlh7mwkrospquv9hJnP7sBloJrNJDhRj0z2iBtRralANI2o9tFKKs20oehl4eFrFU81h7OjU6rEwzlLCQuCOVn5o3Dx+Y0X92B6PWot7gEUY0wb3bLANB8iZpLYuGKGGG9cGwY5AL2LeYxYV5nK2gFnVVZ3WHl18LaMQY56CMWyKilFXaDRJg14OjLPALWWij7cmGuH46uaku5wWjljVmYveJvsPZe3BNiQOiU2XAtQ+dcANDF1ketlkTSY2kqLHQu83Rija4ltGju5hhhFmJ0F7q7mQWRCrti8kpVUgnIrGUOZKNJiStU6SJhpLDeIFg44IXFrERNkaAHXIYVaEQXYDIzc4KC27gPM4StqwrmImuSmW00WROSutams5ZFRVhoFoMPeQTWg9UXAE4hpNYhbpv1Cos4DCaXh9seFttVACDkmVbMKiXnOYGyEUaYie6+zMKw2fPZxzwnMpkFg4dbMPZ93F+Erm64zP1yMPa5pjIzUv04zirUWQ7lRJ1ggCMUkMGcvjkDV3uH+6J3WRiqJXSKMHMv0ORFwc136y4nzWzVRC01tqqRXQ3WWgDAzWdLZpRDdN9++O4PP/7mL4bH83w/ka/H27fnu7U6c7vF/rK7e6vzzFlz1tGUBbY79nuAqKNBmMHGtXs1W8OHEcJsC6wNVfY6EeDb6MrKqmflWXn02m2sA0UEN9jAuEcMM2N1d3eVlDRjHVrs7hG2fIIexGb5RJ/VgHWHQeurxd62GDHCiernWb45yPMxj+d5PtPdsb5ejdgCgI5c5rY5u1qSRojLXMIGsb2M/fU2RmwjTP3x9R1ZciLoHKYYWxSs5nl8PJ/vk2VmYDAiMssS+TxPcU778Xffvf7429v93uifvv79z1//WMfX395ffvPj99+93u/7/WW/79t9ixHbblhE+HX0rzuRCUSvQ97x//WfdTwuVf6T99lrid05Kx/1fDvevrx9/fmXrz//8esvf/x4/yXPj+paHqJF+v9V4enrXYLqZfdcIrSB12YgLK7NKpfvbPUyIYx2zPQw0VumlYoeZsO47JfXvloLB1uoS2gq2Hqo1bWGGdoWPgdmtlFwdorS9jqWMzapKmW2ZtvyyZfYWgUE1WrCtEqBgV4WYrtW5KYraWAC7KxlikYMW0ChIrpt68Urh4V4w3gJ0mLbZGwti4i1roQ1l91w+Sh9QasIN7dYBcwS6yxzqFNRvlJ/VsPQRVX5en80lBChMmwBHwK6qqdhIWLX0kIijWGA16RMlZpPteALW5xrPoDvHptHkK5aLkytlyXcwOCyWtdEFQFEcAH5NRemZXm0pDAEZawSFxHq5BU5v7rlVi8jba1sad3yAILE4r2AvqAAK1hHNVTMo9XE1SwpVTdFLZvDkiMAa9S1mrWxGHxCdzdQ1aVaUlUQBMNRbWGstdKk2XA6GdaclZ3zfJYKnaVqeK/ONVtaC9XVVX3MecxpSPPm1iBjd3FEBCmgpAyP6sycc2Vb0DCI1iDAmS1aV0nQBgZtMy3MyuU+W2LeskWslhu2kEdbuLtnWnfT5mpGIOUmGrRuRm4UMus8u2dTNBjtyoG6Wze7INjrb3//cvvx+/tv933jG/f5xVzLT2Nm23YDmNkf7x85J73jZi8/7rfXcXvxsSmn3gAgG6JDyW54jNWUibkugnWVjrvHetKq89F5qJ+V2Wu57mPBhNWAD8RwOhZ5q7KPo1oyuZL5ljbgu9HVLSvGuhATs5DP43rLBQHVZXlex0K1EcXzVC9w4YShNWeSGNxiNCm3LK1CJjjc1vdT6tr22/2+u9nn4Fu1rFQ3wUdsi4ZEZc9zHh+Pc+Z6BjzMCA/jZM+VxOHY99jj4zz/4eefVfPjy5fd+5//+W9/8/Lym998/3p7fbm93PeXzbaIjW1uA58TgC7cLQm/RP5/9H3+CoC75gGtvUAv77zymXnM5/vx8cu3ty+/fP3lj1+//PTx8fN5vFUeWIHcFb5f/5iGjIsidFWPLdt99cIrSNcdoCl1Yb0fSNLUFccj28Bize4AelEsMYaTqKrlOl7m0HV9kwRb6XOU1l7XVv/ZhYJuVVXQMYxwVtOhbpU0O8+CKIkmk/FaBGJWr7fOtbVr6JSlZIAJw1ZcCpRUadZGdzkwjFN+sEbAIYM1u91so4LD4gqJwpYtrwuV0+gBW7tjOM0MLdKdDISKnQXJobV3q1ky1LLQOM1lt522klfqsk6J3Hx32VoPV5rTyIY1G2gyTMUGzhZt8RiAJlaXVl3h7THCB80hk7JW+BoGN/hY6KPFMCHgdplVAGolYBe44kqirkIukmFBKtFVCAOIWj7PApu+4oTGoG/xmSnEym/lca7gFdtqYq5ee3E5KCqLjsv8aiYDyYjgpYMBYCdUk8Y1BOnaxkqu/XW70CUpri9uKswpKtus1KrjmMfUlLQ4xmbDtvtwDwtHk4U+j2zNM5HNLqotkOwOET11MLP6lFM5s7uVpVxOwwUq7+xsqgggP1dfHlS0yFUXVCJPdV9nFyEj0N0nUBCZmYK4FB4S1u5N9PqdF9pWVKDVRRWwvu+lKoWTQZx0G8O3feyv9+++u//obu/+rQsknZozDezsOusxn8cjzXC/xf2+3W+x7xEhVSnlZvRAtzlI64SEyrVEamtx+ZrC1ybXgDmrz64n5mPxxrG9wjdbwSEalnyaRyORiWrUFB0+liVD9CuC75QjxvB+ZGajEM1txHxUseGg0Yer63gmw2Gmqnz0qpFfgCiQ7vQtKLHR1QZw3Rn3K75wzvSIVXdshkb3mc/nI7NgHrfdxvVD7kJXv3/7aJS5j9dhRIyInT2rv03SVR0b1We9f/3TH//D/9PHtz/8/vcv9z/7zW9/+7vvv7/dX1+/f7l/93p/GQh3X9wNXtK/CQaY2sw+wZ+mZcxsLM3+snuuIA3QaFOjq/NUnufz/fz4+vb25evXP/309Zef3r798fn8VvnUBXxo0qQ295V7gWSCrjvbr559mnllmjkuII4k/GpTXtGw+PE3f+C+sfLt43GeZy1osozXm4sqdpuHrWotYBWV2MUda66Qly8SkRGQAyWJcEDqfDY6GKiJPptTa/9wjTBEaw0RWL0JS01YO7NeFe4Uqt2vXfdSlommLrHbyBHGKUhlFMSNPizGAFnr71tqyK/4x8VvQ1O+duk0avH1aRJSaTLCVd1VsMZuxpBbu7pqRf6kli0Tujxi4Zyv8gEjAFuWyYaaVcaIFU2g2zrLuj5zzJ8W4hUbFFp10dfdjC6SztV7cSE1zYgEgw05TaIkJSQ4kac0aRtsc3Ons7PRQErZKrRKkHW3VggCy/26Puf1hdHsNlI81/KvqVb06moU0TAtHCjdABkttljtsjWVrVLnOZFLajdb5WU0DI5wd9Cg4iWmr4GwrU9UZrFUdWbmkVz0WqxUuLmZr/ILqFtZtcyY60yHKSWEiVA3jFlVffZsWz44tq0INAGuthhV6/NQJl3LobPMsGqpdFEHszVBwEdfuJGEmfVzEqye1b5M174B0WbLD6m1J8Alj65hrJbLbs2Kyl6YS+N9H9/d7t+9vryUBFhVH4+PWWezWzyfj3mcs0HCA2OLES7x4+2h92x1zpqJEnyEcVgYumt2g5proNeKkEqmqc6r++E8SifseiIWxJJaBwOJrFXyUbl2liQRsSoiaqHxKIazCy5iClP57GVCfH6ccMgRsRL+OM8GwUzGdetHtBu2jasy7CIlLAxElqB9cxo9zK/GFGtBlcf5/KiGmN3nxzFG3F62/TayezWkLrS1OsfLftt397FcCr2Am2aVud9u9D6e89Hf3r78l+dvfhzjt3/xF7/7i9/95vsfXn7Y7y+v9822sDEQ7ptdK1/D5+IXAI3/BPizrvvXNnid/n2l1+1yfM7MZ+bRx8f58fXb29evX37+49cv//D27afH88ust+pzoaRga7dpvMoyeKlz10lo/SkzVc+l9a0tMKArEEBef6MU/7v//X9/jr84fv4f/+2/+3/88U9/dG9z4JyStXWvg5MBMDMbcudtH5/Xnq5EnqJYUNCXsY7AWiB0dqFtM2X1hFJKOBG+iKGSig6l2Ax3mvUUGw2qaM0r6pJt27K6GkR2s+YVe3ZDsFPDyUY9e0VBNx+2OC9hxTXg2RJMXW1mRLOllVNGs9YCgtkWHghYt862lQemccBGeHg7uyWyG0sCgIzGMAqrQqQMCLZMlFqJVvV62oSuRf8HrralBqtJA01mYrR0WSe6VEXg6he9FoYUSTo5aUI3Kxubr8t1das+VzXLErIsddd22mAXelYtuNNWk9x6IwOXcXKJNVd0Gc3MBii2XZbjQkPdIGixOvC0lsN0w6L4K2ueWZ3qmSwh4YYeZICB5QaDrbR613NBQK2roNRBnbncweuXsjCTSwPt5dSqE2AXqjLPxfJpcwhdiQ5zYy8r7pQIoAjVCQp0eIDr9SO00IUmrqln8fGWNwnNUoN16nJaJ1kyQcdlgzIzNHQol4JKgXJQz1Z0LS6EIbjiOaipKlxXNtJBXV97mQ2dFr5v4/Z6/+72cp/zhPXMo3rO+WihXTWreJUybrsRlYnzo85j9ifhlQu4yDA6BZNVg33ZD51oa1Bd7FmdXc+WiQUPLHY8aAHoLC5sQy7nM3uqGh4wl4XZ0CJMVy0Xd9cBklWZbTm7ZxM8UzTEHjTFZmZGoME8r7Ztd7Owsfm+hTk8IAjVnzk7NXW77/eXfSw2qFn3dMvn43y+f8u8NM2GZrZ5o2d2VSJLdXZ2V+c2/OW7/eV2o1nOPGZ29uwSRDcEGGbtL6/3P/vD7/7qL//wN//iD3/91//sexu/eXl5ub9uPrax7WNjD7Ntmf0Bop3m+lXeWed+X0Gvpfis039xdtcjhivf2nn08fb8eHt/++Xr15//9O3LT29ff358fDnPt67zuvhiffllsFITuGDgl6HiytdVFkm01pJgkXSktWhcvq0rJxx//d/+9z/92+d/mv+262BNDzdTo+dM30aM4fRqVWd1m1mM4b563imu/nk05B5j26vEmtrixISjCzbW3X8NQTSHO917zs+n2kBabObB2fJ9OBhVLXWqEupeqw1uZkCnNLWMQ7aRO9EcMgBNaUPXSbjE5opx2qoJDwsHimaObrJMqs5Fx17CmZkizA2O7oYvLo206n586VyguqFG1/V/W68eE8iWOVJaEPnFKFrWFlOZLR6zsQqCA6SvtfBYB+IqOCmapIX7pBoUURSuG+jYbeEAJPaEoEx4qJxoVhNOd1vqHZ1NAW1XhLexIAK4UgUXeN1ASK3M7ileIVzBoKAtF9S53lslrdsiOWjrzURA7GqL6GYVNbtb56H5LLJrXvTi7rYCPK5htbqmC51TdawzvdYyzCfUhVQ11bKdsIZRAZg6mJVstEnLWWsNwAZ7LS6MJAqqU8haMXzyH7M4FLqutvGlw5VdCfzOdTODG3Jlt1Nq6uzVNsjCKoVfGq4PrNV6T2TDsEynxFI+E1z4o3UgrG91wYzrczf3ZVyuKcFRFrZF3O6319v9xZyc1ajUkT0BMNy8i+umBd9J1zHn0bOEnkBg7GszBAlK5aMIKtGTkJz2SSeQDFfaewrqegINDiwvu9TZWEWbvxIEQCDgQOyEZC4bFKUCE12oqZKuKH4KbQx0t7/APXwb676xMIuLwNgpAmoV+mZhDQ9bDNSsrup1Zd5vcbv56+t22+6zFMH3twkys2bWWt3R4SBvNu7WXfW+fghmhqBvL3G7bbd9N2N35cz5WHxc4boYsluvP37/N3/71//mb/7Vv/7Xf/Mv/+ovf/vd6wvHd6/fb75vvsd4cey0nRifDe92QfU+xf11uQOAtk830FX/q09DkLo1O4/MI/NjPr59vH/58u2Xn799+enb20+P55fj/Jr5aBQcaFwgN3EhwK95tK9coT7zKlzAnOuKZp9+HEGgmVTml1kz/uu//7/9/U/xH/6nf/fl2y9Yd1WJwe22xxg+hpFH9Uy6r4gFL4hBq1ISGU7BfW+4+areZGzeKA+P3dQy2uoQsvVLt/wUE0XQ3UcYBBflMDOaQKtTPUtZi0XmJ7saYk0IrpKkNFwys1NSO2kDWN73Gh1NU8pBRStwUbWyrjukjJcLD0YuQ9kGa6mzpD4fZ1MWC+ooCujqVURTkIoL+xwOrWGrcratX0wvSuPyhfkInB+1lvIWwKrUKoCwwTEuXxiXtIdWr90M3VhdDEqS2SLg1Yme3QkzytAtd8JJd2q9/rTCWzLF7iukJVgtOphWy+bVPGEkHTULgKpQ1i4IZnKSBhYdrEw6BRm9g2bygZYIzbk2N03afCaEKvQpa1PXIGCIQF8hOdWZDY7dcLaoOkqnMVGqmgVXw5yukDXl5TvXqnrxmYjsokrdbGnZIhZD3jerqdbK02jVKsQ1PUIGd7hfysn6i9cnb9CamT9jz5+eJbCAhmqBW4iJlUTrgu1Y36Wu5fH+xzS/Fo6RsEGC6xW1MoB9leIgItwX+V2XLNRwcHMf2zbCzDhZ2afvtLCepLNRjSJBX8Yu1BrgBBE+4Lv7AGEqYLZOlTRTnXDQnRlwDxthALrc5MMbBAtmpga1fBnu5s2qdXOlu/tAmM/O8RI+DACsAfFQCWHREzasKMAZ5ss7gdrGjW1Zlc+zq9DdU5R3lcTYXaZwq7M5uFYllcoFKgxsY7lXtN+2Lh3P43Sex5yzepbBxn13olGd0qI3SnmqUmItBL5HwNFVZ+o8zudbzikzW05cG85hP/72x7/5F3/5b/7mb//ur/727/7qr//8+9/8uL287q+b78M3YgtsxA4E2mhx5bya+jWaZdeVEv8k3oVP6s36YQvo7JqVz8qP+fj68fHL17dffv72y0/fvv7x4+Pn4/iadXSXFp79erksSccAGhbDBuDlz7b1Vr9CYgsCZOsx6O519/8MjwFS/Kf/6X/4j//+py9ff5pv5/5iOYsG28Jvm9swOmG7FaybCzre51MtrMJbC7oRMAsoM3vKWoviyXD3RsNo4QZDNtppy+1TvvuKrS2Y6gWBgBodq6PdwWiJask9T1pQEopFQu1TyTaHh8XmCGDj2o8BNIsWugowdaG9ZtOqW12FatRa14OwBYUPcBh8YSVUVUtTL+TatLYWGfosFNBAprJl146iu1sZ9vmmwCf6wuE78rESIFort7UJF0QDyWrRr9276tqTd8tKKNbaPbsI1fOCJan7ssITsQ8fBmdn97lELyxIxoiVYWSvw4vrfwHVao+4uqilqrV9L4/PT+Yq3wHqc2psXuZNpwKi21jUcqCFJxjsNaoUOeHrQQh8mv+JCHH1F0mFykSrU0j0tfBWDMQ23F3VMwsWcHFZhi5VoNALWQIJNkgLhBtINw/UUS72UUSaXXYLAVz8pkFf4UiCg01E92IMClo/nP5VTLV1rFPWKhFoJ8MBsIthDANpuTIfAMW49lvCSl3SzEBbTQErQKta4Aiq5RfLAstzbKQ7w+kBYFYeVU/kaao9oqAWkVx7UTa4cR/e1ihUiwuIDbCJMhVqqgECFkI25FVttxW1bRoXnu9k7za6rlUjHbd9M1gd2bOq4W4xgqYYdhs7luCxBsKkqBgmgeZdogXcIpyJQn7/+rqN/Xhmvh3n0TXLV9Rn2QlgNmKVbixDQ56grLK7FDv2iMXY/P/Q9a9bchxHsjBq5h6RVX1Bo3EhQUoczZ79nfP+j7UvGokE0F1dmRHudn54FKiZbx0uLokiKaC7KzPC3a7n0ykzjyNfvl1iIo4pgGins/fTidCISm9nDMzIOYHMFSsJKmL/Ng4bBswR46I51E7gye3Eft4+fvzwH//z3//jL7/+f//tf/77z7/98uHnp+3xzO3U7t1aszPQDGdgAxrX1G8EtOoZ6jn/AfvYzX/xY/yvuIec1zmPOa9zXPb92+vrt5dvf/z+7es/vn/7x+Xyx7F/H/NaP1r9y+OBGldYB0jUSqsUaet9wPLrrC9m2cBK+4P1a2GxTu3r7//7GH/PlG0wIxvMm7fNvEu2z8gxUjsomjJnjhyz+sfK9yy/SRMSPMYoTMB6AXxmcC25PQ3NKsfLEqZyHaWElDEp+WSGQW6wAkPSGs901hhN1YhGtqyuJjdBIXaX2HpvNAVUKr6QsjyzRrMcBwypmaXULdCS7t7SzKxtFNdJDyiQGTGQ05BxRJ19iTLFJGPxIEtfKWQO3OpA65OmL6yZDYqAIbiYnwXFQSyXFwMwb1Z9ulJWOjlEms1EmtzAKiMrbTeJxhTgZic7358gojNbBJQjAESm9WaNN51+2ZO15AgpmbIQAECTCZb0gp1u9RELsXxhCVnp3aV0makKzb1ZMjIiQwwpCjGpNBACCYdKuCzUCV9bSrUjFIe7AliITLU7evN25wSrf6r+MetYyxpwXKYqTzYHzNvW3GwGSlfpYbHP0mhZR/0CNPzIeJWBZhVSXkkWGRn7XBDt7YFDATWThMnMz2y9FbRemopEmhvIPGam3Hgc07dWm2VGYSyoaNry75BGV3m1cgbNophnZYX3+mnzrdMRc1wUX19+//7yx9v1dR7DWhcGEpilK6vBz+isyJPC45TKIyOoERhMlZwG3hotISUYmRhwN+ucxzEibHM6fWjsKfLx/YO1tr/uFfK39ebdrJk5797dgTHjGNd5jCNnRsA2tuYml1iZINagqTmTp5Pfn3PmHON6uVze5mYIwJ1wFZfPnu2mUsxMBTOQ1S2YhoBtrW8biNSImTFj7phDouDYtpM1LwVyGhQRERkkqIA3MzNrpreMXakkLSbzgBHuxs7H58e//PrlP3772//4t7/97fOX//HLb5/uPz61d2e723zb2r1hc2xEL5kzYUJZ5IkllkaJ85XJgv5t4a9YRicoK+Mz55HzGvvLvr+8vf7x7eXr15c/fv/29ffL96/X8Xocb0JmeZqUN6C1gLt6MKkbzFQOjTrwVy5bqUUCgoxWGnpwOVhqP4XYbFO4+inSOI+gQc1EE3ymXt6uHDILb+r01k88p++cIwtJybRKpk9WdDsR1ftB83rvkaggIThpjTRrZolcHBN1K0BDUlMFW8B6p9i2jKC3innPqkZRgjNpVMupbJ3o1ltv3iibyD1rElFVoqAiGg0sLU798FoFmQnIhaHAgAyLMY+IiHkEUi1DAa4svJrwit2ViZQVqUNZs4qB+/FTJrmKNIuDX/pdmN8yFLzSrxMOMxqEqPRlSLY0S3AhjXCzZeYjjWgGwqeBm2/n7s1LhxOkn7STs5w5bjJGKiLqaaB4s4ezLKMlUlpRmiRXd7eh7sGZtagRVvddRAq0O7fm7dwMto5vSzjjqMBOCYGlOihr9HLvFkmrlDIIlkOmoE3ztI12Mqe1jVT1KSGPdHOl9CaVxE2gm62Y2XVQI6CZ1hxhrSZ9lbsEbLWTl0AcajWZU4TRDWaizNVtHJOQysNtNIK5vont1Gi+bdupN1+eADvikEIplviSOt1vGYFgBiCvsKpyvdVSVs88EpmBSpyNFX8BJ3uTI6mAruMYl+Prt39eL99iDE4EQ+W+WvVftFJ/TS10AARwvM6s5SDWAmOteKxKMJ1U9o3b6eS1NZbLRQIwI9n48HB+uNtozROxj1lmnzSHb61phrlyaI7MwzKIYr/hVYSFScUcM+noW9vaKUfG69v1dZ9HOOCdfVsPM5x0a90yVnGbgJiBKDm98qBaAz2n9hn7njHK3ZiJWWaGOadbq7cE1Tsys7kBokwTAmIPBJXOEi0z5Jkh37af//b5158//3/+4//591/+8ttPf/n53cefHz/f+/19e9y2u4at4QxsFfKDZfVaq+EqVbiN+cTyLy2ox5aAsl6AiMwRc495HdeX/fpyuXz/9v3bP1+/f/32/fe3y9f9eJ3jmjlTK+u5VIr16bDmYCGxoKByttYdBNU2QNw4PJbvtwKiyVvBMuCGVPv+z5d5qD9scRlUjFAj7x4evJ/e3t5MRRm5M40yQ4NbZ4YEs0w3P5/PII6QhMyoBFFlDY6MmQpf3qNW/RCFfUdkUFmlTzRmMFZQepJtBtjd6blnIGVwd2tepJPMMMLakve3djK37h3ywOjGOUaJgpUBqsKWawepmUJZWHTOOQND7uGt0SVlYSE+63CymvBXYJ8BYIqqejPA63zzulPBGxRbCg+vXxEZBYWAwIJi3Mta1HoNtmJmGeJkBpXZl2juwUy17kam0ioLhRTQ772E3+atWu/nzH1SbYn4pBwhJiptw340KJcOlcqYTOVNmH+zilMI5hrfQFI1xUABmrOpbpfF5Qp+6jEOGa2LzeUwpVIIV+EaK3xMLiFZyGXxmAZmHV1OJ6wZauU0KJCREOYIpTBEkY1mzmRFQhWFE5YogTigyLFPTUBe/pS6OrNWyF5VdJWUbpA5ezfOmU5Oyg1UaEwv6aIk0YzNe935UewtLaAIJLnYBrFyuZEMRQmdJfKQlhCIgJaC2+tSZ0q29C5ciTqUUsf18vry+8vL9z/+8X8u374eb98zMjSyhRJTMjMP5pEEWV7fCq8LVdSzAS6jgy251ezVwARta7feqkUIlWko4i1D9O69bw2dwsxs4DFjDtw/mJnPORUzqqo7i8i2tpk5TEtvFqk4Zszs2+Zbw9j3S1zfrjFkgjX6Zr6RSw2Q5e8xs4jI1Hrv2g3ALl/QdWrtVRFZ8nN0r6HSjZjHzsqNTy7bxGZO5sEMYopEDDLrfU1v1s5293D++S9f/v3ff/3bL7/8z7/9x5enzz89fXzaHh/b+X577HbndraV59yBThjSaKZciP9CXm6HPnBDbJDrranZH6aIHIqheT2uL9e3l8vL71+/f/3H5eXr6/evry9/7Pvr2Pdj7LRCkNOWekFgiUHqV88aZwhbMqAF92ghJbcvI9dXmUUc3768tTG0eVxPjxsqp0A0YOtVtWfW7rONEYfbGpPzmDMxQtV4wG3btpPZ1t3GeIWCDDHoJTLOBJMSoxTP5UZIWgiKqFOHNwWuinqleXOTLxq9BSYRoDu9LjakgKbW3NtW/QZKS7MpQROESiWTQU9zS+VK+V+j74pSKziiQm2lDVa0aJLhTDapJhBfiTpwipZTEEuyvqg3lvlQq6fRIDEBcy5TG0DBARkK/6VAS4qtsbuASkWl1kdX3farmpWm5vDNunuMGEhMZR3A1oxbCS5q2I5UYoLlxsgycidYAvqCIUM32WcAMyLqeS51PEp+mqUiVfl/Sr8iJkBaA5006823refMuQ/MKh4CutnKgoYSwdTIZCnha/KtyAgtvELVylDGH6BOTMM8Bt1yhmYJdoVQ1V5yugxhpBmqDRKycL9nWU6VsJRmbm6KRMpIkM0q44IOUz2VhMCq2FQgD9hkIuzGmAHI1YLuBeLEnEqFS/JCGW6vH+FohS+VVm/F7lU3D9jNSEczM0WpNUBjI1OWmfNI0dvWLBB5XF6+auzfvn59efn98vp9HBfrbL6FTwZaa5rpjUhqzzlAZwyggKkDdZcYaVip1tZsPSVcDiYF8sa2ZmZWc7q7eyu5WyrHmGKwyUI0Zow5x4jjh/zV3FozOmKGkoSlMA/FyJzJpnk5dkXGDIhufhIC1uohUQFWMdPdKihelG+tt1ZFevsRbCZHRH2lURg4yK13JCLpsDgm6jPLJYq0ZvTundtDz+uMIyTM62A2OiOj35+ePj3++teff/3py//4t9/+9uXXv3z+8tPjxwc/P90/3/fHrT0aToYONEMX+up2t1KNr+P+v+T4r5MLymIFUBhg6QHnEXnE8XZcv1+uL68v375/+/0fl9ffX7/98fb2fYzLvl/GvNZ8WnAhjcrijUpmXcd3AUFSJq2EFQvjqW2gYCJAlf0ALIutsmSg699uY0ZeSy7BLHgCVGRwHvtE5G1l4BhHltSBBue2tX6+U+A45q59HvvM8Jb0m4W1Uvpd5ZRXqhkNJoRJlRRPOlCTejUmW/fWW2/hMSKZQrazzz250ZcykXCXpVFta5YzEhmaY2TQXJkCRZtGwTLqe2VpzJRlQA9w3n4G1hoYxY/kDE0w4RIDvVCCUluYzHIKphziTCU0l+kbZSBKwIR2E3QU1hEVIwM6vEAcwWSI4s8Thgav0zgqha3QKjAF0bz0oeDCCeaqQ5aZwBmD9f80IKSYGeGGFX1Rf1NLhuh11ReQVGvQpAJVda4iNdwQQGbMdUbToajGcMFFNyF7M+/ems9qAVyDm+YKcnBDExURbDGO2Zw0IQAXHEith7mIcgmo6jZqzLDKmTCNWfGkjhwTOUGRUU+wihuwJtscgo5bDN5MhCzW3JEzlM06zStjpJTqWcoWAcORiQhlQlNLGYAWEYHlAwGySpAIJinvAS8uhapYVTTTUppazfOA4nbmwghf7++SbluDNWvdMn2MwKTBt77Zlcic+9v31+9f//HPI9+Ot1fbJFNrG9DObbOOyKOcplF9k6bCHCG40IjmaC1BHglExJjqBiS8iZoxclJzzJlxlKfd6G7dpDz2K/Zd1BwRY5bGDMw5536MyJKxVvRy3f009+28IXLfJ4hUWrPqkxNDrn4yA2KaHeFdgjIq79MK5WAlUIHnu63VBZCCHbZ5anKxWLDG5o0iyt2RMUORshriSFJTOUZ42/zuoZ8YYonckpNIU7s7nb78219+/cvnv/322y+ff/33v/z689P7T48fn+6fNm33p+fNzg2PQLOF+biV4SvtT/kw/sXvVQk2P0SfC/HH4nxTOZR7jH1cv79dX16//fH15es/X77/8/Xl97eXb+N4O67XOXcxVOadhAqAJG4CHqxwSbLGKdV3vYpMUM6+2q5vsn8tGxCqi6JgT6446GS8fU/6QqdBhrTvY8x9jGrsIl2Blf4Ng7vBHNI8doVHTMWhnL6R9RUlQ/WqJAFL1GVkSouZkNWJiTK2tpoVDaSZt372ZjCpik4zcvcGILg6iB1VgINI7agOrAAyJsMs6yWHlsuRlVBcuGlIQSURyiEB3uCqUyjJmIdUgcYSfCk1i1MULYNx5DgSKQtmStfiYsQ0BgQkUr2CnMFYUR+Lwg6WTisz7ebcI4E0rT4ERYgVBVPZ1GZwkyolIo1e5VzIEnIpNTERa1BHJagbZaYZWhqeWddmMdVFR4iEJpQ3HWoypmDJZlhKHJSUwlASwLUgVlyDb429DtIi07No56pG9d69daZnzsyIuduGHHNpZUuPs0JYWUooVbO0EINIgZkpalJwU05lnaWEuRlcxd9Q5maW1kAGRg31YlQoOsroXWOIaJlEmicRAMpLgUbYVAyUE6sBaXSWiIqKlHvdIxrl4KZoYPiCTVJTkrqLVt50lJeXUYR/eYCXabYy/yt4ld76qdFXgaU1GHuG7k/nx7tz97y8fZvx9Xp5Oa4Xk5pt6Gz9pDEdjQ3SYK8EF7HBN6cIBM8i4JWevIr5TFRGwFSFSQIjVHccZYQcXrncSM39mJF0A9NPhggI8Iw5ZOhbkQ+p5H4MkTR/eDifT1uju+9XG2TOfcrSvCJG6J2AcJDukir3GKR3I8xqRBCazMENZuYBhWcGlMiZVfjs1mp7Ehj7nDNaGfvb1uoCMNHZT48PD08ENSd826+vZhbE6c6fPz//+uXXf/vbb3/58tO//fLXL59++uXDz4/b3fPD+83vGk4ne9dxt6Q+64xf59mfs779l/9af21I3FITSvGZqlKEOOK4XI/y+n79/dvXf758/+fl5Y/L9z+O8XZcLzknTJqxHKor3wEL0SmD/mpMjCIdWmsL4s8s2KxegduUx7UNZMmAYNXoaEzJzVqUM2iKhDK82TzmqIiAEl1nia2rTRyNm7tHMnNqBCaVypjeufKOBRG5YurqoCpWhMqs7ywTlX0diDLRN7F5N1rGvCbOrbNbHsftN88MpNFbFRKL6+kZysJziocpN1Cy0q8KeamzJAGBIdKUyakcLsi4iDsRI1RGjd4s96QhTZGqPhiVa2nPPLIkfjoy9pXNty7/+vmWsq2unx/i3Al6pQoogzPTmsmXhPsGdiu1OIoZEKEmKCAZnIlZbU2Sw+FWgj7ZsvNLaUxVMH+qktrL2AUoKy4ksQKrSbpEQ0MtR8UYsPL7F3m1jINcg7LQIBdOlpySZWgOjH1XxJwjZQ5vrZuayatZ08yC0wE/IRW+NtFif1DASRkyCmrXSIm5MnaAwLT1Iyq+x1pzWhJGWrljqhjd64AAbguIDDHhpnS17izvvNLE+smxdHwyhKAG0De6g8xSXEjlwUulclb+j3lrrCCKVlpRTYojULluZqSRFlsqKMmMZlraLzOzKsU1W1YwAzKZvjFCkdl7v+t357ZRmfMK7vPyHX4Dk73LbJJTYd0tAx7twU+b+dbYiEzK55gZEMJOholmRZgm19sp8xVyjREGR1aaS0LQ+CEHgzWq9nggU4FER2/V1A0Ej8hyqXXTbPOKo20mqp87gQy1jWQNzYWOouJCtZRioptv3txpyIzc14w7p5wxp3IolDMzBtybm6uS+DIJq7wp29pDvz/fPUqICG/pbqeHd+20Xb8fb9cjFJWr+PD+4dffPv32y29//fLbb7/+5Zfnj3/9/NPn58/vzu9Odn/u54Zzw/0PoSdWzIMVbVxnOpelK5HlgfiT8q2/qdu9EJERyiPHMcfluL5cri+X77///v3rP75///315T+v++uxvxz7W8SMDLKCgllmq3Wr1I2dKv9emUrKkpep1SwkWxJQYn2yQnI5hG8UcsF2xawTVBsHMjEDrcMa2wk0j0l6d2+pUnz4dtqqOtiEanGJI+aRqyaWohtTpDIQWQkqKxCPbY3jMCAQSXNEIaO8MaLdrTflnEf28zlJN4SNOcqWeiOBHWIBa6YIjRI1lrw+DTLKvDwRKL9uGMpIAazcz4LEQJiYR6JbAnBjI62wq9TMzMoJg6JKSpQTOoRE5QvoSEblBxl70glnOnyjGbISNCv94VAIGSxShrLiFuAJR+5iWyxSfZ2hpSvBLMNHWU6p1IxrR1NjRAhl9W7uQGYk6v+20PsSGk1oNZMlYKy40QomdqJqqBKqRMxYfREoj6p7xTmV8hipdMCVCCcq0vu4xnHNfZ8xZAQ9K5gls7JfZD1bTimg6JsTqCqgKowXlLMYjORRRmgIKN+agivCwVfUmndX1b65tXMrtquCrJUps2qYgLvRlYpWNRMrcRbKkcdUVNidCVHDeVJKNdLkrYGWwsyUFaUopJIJ0rqhoHwHMlIpDVHWlZlEA5a7rjoBI0ALQBUBvSSn3W+5wSyYztmk7KcWh1kBYpoxIyLGfrBGhSl47+e72ZgZ6k6ldcrQTr6d23beaEjljMlukZV86smKDgwxhXSDdZmnqqqt5rYpkBmoIg3rMjdvtAaJUVG0jb45YWVFioEIobuNlEER+2scl8OM2/2pnUzIbWtwSelONgMUEzGJGkSQMliHdbLiBZLKpLfrNRhJqLi6JFSCMEJHRWUpZ9CzN9/u+939/buHd71vb9f5/fvvftradtLk29vxdnl7/f5yPvvTh6f7h/NPv3767dcv/+Ov//7Lx1/++vnLp4d3Pz19ut8ez3Z/andEbzgbzkDHine+oetL95k/DL1L5Xn74zb4GyxzAplzSqG5z3E9jrfj+vpy/f5y+frt2+//fH35/fX779f95Rivc1xzjlQYGRlmVhqfEpTXfEbJzRIrybg4rXrHudSBqhpFqxjlm/2Xtw1g/cVNoFg0XFvux74GEoGR0U7nvp3gnCMr5La1TkVkjBFzRA7FHgtbqOo4J52VLlv2J0OdJNAEbKkjMohAVBC8u2KVAog5MaSw05ZzFPJYT8qiVEsOkNO8Cbf8uEgeASMM5unN1rTO6qTSDOn2kyItMyyBRHlKS1CVrWQtVjyxqc7KtdqR1Cg60yTRaSFECfdhFKPIi8RGlMNos3JZk6Ijj0BHHslMVBILYUb4raoLK93HHERGWJGkNssWKlTVcInWotOgzLRJOm4RQUtiAwg8RprxFsyzvov8s9wHcFq3WpRbz5TiUEWA0Cwz28lgRGM/98wsAj9meENEulfsWyqQkzmlAw4rWL8A35xmzVPpFq3ZMqiXKjqWczWzaiVAA0U2R4HsVVr5A2DVmoLMsLT/7nDmTSibkaQnDMzWmyWtdcDozigJMA2MDBFyq4afYkuhTCZp5sRmdPlmoGsmFBEjuZpeHG4A3FFwSpHlZa5g1uyfAGZU0yfMzTrMRhxtY/dGICPgDlrSITZ64UAIoVV2QlEXObhzco5btk9r1vN0dzqdz0Zt7WA/8bjQSeO29X7qFejM5oau5Bixvx1zinDlqCo6s+U9LvwGmRHAVOwlvWFK1om2Pq/KOxmhJFrjMoSs+I2CKKBFAmBcYV0GpR82RZm5RSYAd0M5DYt7T+XICKVja956a80zJo6EW4wY1zJ0EKQ3d6OYGeIukZhFurG5bQ/nh7v7bTs365G6vF73PSKnt5j7uL6+xsy7h/v375/+9v/89tOnTz//9PnnDx//8vmXn59++unx42O/e+yPm91v9mBoCTdsN9y//QnwJ2qPW/QvVp7POvlheRv8S6SSiQzMPWLO43Ls1/367fX68v316x+X719fv/7+cvm6X76PvFzfLioNLyyVBXHkjzbH+kToJLX2KK/kKisFxE3/maH0Ne/zX16a4pu8MDeA0CrzSrlbo9jPnrl4nDnCz92bEYFkRrlNbBxX7ceYgbAUMJRTTKDBz+4N3hyGMRYEVOFIjhsz3UhnVP6Jc0SYeUnB6+sraNK8m6VbFQBPKg2pPGBrsrXcUpFiDFVyo7cKJq7QWLPmoBSlQJXkNypEcUynUVUBVsHEEJo3JpIOMxbWvhhShzUTWCZga+bULCNoRkU61p4FyTbjiaBbM988y55f4fclNXGqdFsONsqRTaBZq0VgBauVhDEL51KpULKm4QLuzZNGecBW8U59AXmD6aP6FYzmNThg0SBmK3+gW4XompuZgrSEmcUY3lsirZlvJqafe+sWKVbchotAW/VDmAHjSraxAAZqqQwNkGii0Kyygb3wMaEyRDRuOjZUCj4JQ8zSr0SxXzVlrbwbqihtEGxm9GIOcmmwUGU03grbIM1slXH7gh4CkZozIDe6ofzWszYxdnpzEW4mlGF0RTsoqiButaplwnsHqakxy7HBG/Jrloz1/Li1Dc01s7dz36y2lQlMSaEKgZJ5M0fB8qwLmJ6bpTk2GHrb7u/eXWhpYY3d22bNwJ0ucoJE9t6rjjCpiIls1UkM9Bgc4xDU2Et53FqJnc3AqZgjECtOLAW4HEAHO9gRYM6cGXOKhlGDXYN5pepVxacwVZCdOtBBh1xJOhU5MwCnJSMQSdLcqeU0kbV2OrX1Eh6GNCJzTpQdSNaa9c0EzVAegQlzmiiaJU4Pp/N259Yi9cf379fLDjGvGsd8m1eY2tbff3r313/79bcvn/7jf/7t5w+fPn349Hz37uenn96fnx+2x7t233FqC+43Q1tCT/i/4vv2Z6nvwrR1G1LyBxYElCV+3jKOxuU49v14fbtcLm9fv75+/3b59s/r5fvl5ft1fxnjbeY1c9yAmdIlE0r326vtnhnFj2PB30Wp8ofIuHBI716f40pqrjjASDNfMaS21vZqA6pu8FY6Ce+OZdFspDKOlKc0xrH6uY+JgRmgsmps3BGCb2wnd4KmSJG0jUr5tmoEy/ACWQr9dPJKvjxmjMGS2fV6yWHd3M27ezMSmUrkiKFy37u7tzpgYoSCmilZtuibmxtE621l7XEmjAYLGYAxM2Bq5f+qT5CetZjTYVm2p7TKmSHdbe0oMLgxsIrKXEmZGigyq1gYpHW3bkqyOa18bloXXOHJRFkoQVl3NtKYSt+cwqpBX2J4ODgMgTRTMyMNbks8KrKRrS06VNDMWz726nO+kVWsEINShUEJyhqW8ppZJgZk9f85QjHDT+bONHlr3l3Ovm005BTC4ojSGo+ZNAtIg3kkJlt5q0OpQDMStNV4VCN61UbPagwzOg2qcaledcCkSKhkqgumA7EqZSvyAawML6E07yVMrM5jTcETYWxCxhwVZFNa0ZJjxmxYGRBSGpnOdnJvjcYqiWPMmXMcgTAGMSVIWxlYDFY53QplhFQ54lZ1PMjFUzS3TbAM0jtJUdaQYxYdCOuQ1j1XugkiZmrK0e/7/YM93Z8eIUKcYwR06Oh3zbl1uzNkR9+jbMiU0Zw5p5g3QXjhXpVuR4VgbNbWWlIxvTHHzHmIAzLKbaUtEezWT26dc0QSYy4ZXRzLx55aCRcVZqXbSr1t1YyrIhJYBJZpEY8rxaCcbzBv7vBWQSIzZx5XEZ6jvHutdfbet2ZJzAjGev4t5J1KWO9kn4MZse8vb9+uMbCdNpOlcLp/+PD53aefP335+fNfv/z066cPv/3088fH5/f3T/ft/sPT53t7bPbYUIFu9acBzRbiX1I+q59o3vy+uv35L87e+uQBWPUNZOTc53Gdx+Xt+vp6eXl5+/7t9dsfl5c/3l6/7fvLcX0bsY/jGiVbj8qhQrnYstTiJf8nCGbIvGZ7q7itJfIESNFvGmrCFoyw1mPU6Y+yOZqUrS17fsFXrcRXSBmBzZSa44hZoehVmMIcgVlHGFatToEqQ+60pSgs+AHeBMGczZ2lZKcyzenuzWUlJcmRSXEzazR3QbZtBoe73JQzidCMSNB4C+IUMiPrSjRHiu5mXmmWJbq3VChdc0IwJTKZ9CpEtPq2VYomb56SNaqOLTCU3uqTdsAzy4QLVrKyt+603vbrNJd5ChMpM3dvpKWlrA79yk8IhaDlFnYX4SLYmm2sbAHUucm5shZQUkFaUI2F7ZOVdVYEgZmbLdM/FIqInCtgRhPcbA3OcoJAWpQDqqj2Vftu7hBUtFJpZiqxhihFlrnVuQ3TImrzNifO+s0TCcXSfcssb35EUGYr5QJAMpWWy7lIc1haA8IEWboCIKPoEQIVpIX1/K8dAUXZkzESgIzmVUVJACqxRBKeOXRkAjZC7tXVETNHZDSHK5glJw4Aza1Vcqi5ZJmRc4xjrrjrPQyKGZFGJzzNLOeMEGno4M35TwMCgJm3tt231kWriqdEMjUHVMJTMmeYmTVZCgiacqRmamRr7WT96eHdfb8r5dN+vQsDxmvftrvT411/HDG7vzn7vlJ/GTNXMZbkDk8jpAjmZEQeI5ussfaNjDj2kZFxJNNksIpwP1l5B9vWrRG1lgbcjQW/ENawnDiC6hPFut68m5VEEMw6QcgShRddrDU8hBJs5oSXtGGvFm9oEBEKoXxdbr13q7DuzJlJEJEwl+iby0/wdrlcxdive8xpW/fH3szbdvrl37789aePf/nl558/fvz144efH58+Pj59uP/wuD2c2v3WHhvukWezMxd6u/60wvHx48S/BXyuoz9xw39uTACQyJmpzNA8YhxjvO1jH2/fvl9fXi4vv79+//b28sf17ft+fT2OS+aY80hVpj3ZDJI5K7HAnaUJBCCkGZ0WGSxPh1XGY2k2ubhWUFLNlAR7a6UItbYggIKSSndhpLyC/djiEFq0k4lEMYLJIAG2xtaMlnFEFWdYw5Jua8UQ2YqHL5p+NYmwwZrbTSQXE3OqN5IO+da7tZAhswnDWxl/aKeOuUZ8KAJDRPWKLBS/rmTCHW5YRoKKEcmUPDMROUOAXCZmqL7jMhnwhoYAJj/9UPJOuGlU5giTcrOV6ncgMwhzb2at353cbcZg6zmDNrV82EZYJjIZmZWYWr9RxWwW+MxloWrs3TZvxhwp6Rgj0kJh3gz02tSWSFQMy6jeyRJBssZ6qNKblVMaqRC9DspEGipiANRIDQiAgzWxOFt3I2IoI+aIDOhQzjrFaAbbEIg8hMnKZ8lZRBQkMjBHLrFdpccFVAYSiQ7zJTwuri4DTOYAAjCxiY4siF7QGIXiwMS2zNL1fNvNgCbeXkcgc5ZvgKsbphZhyym6BcCkosILXbGrzbRwgzdHTMWsllvVj2vp5EJAxjxi5jFjBEWN0JEK0oH6Gm/mFrizmcf8EeoTdSf0vvWttbM1T0JzKj1DgkaGQa15HIMBZ1jKoPl2uDGqX2VEO213p8fN+8P9Y0zFnNvpdJ1HRfoTxvSckYEUnScqMDAQcFmDtWZpJkJhiJxHjohjurs3Y2dFQs2ogCz3RnNadxq4cbmPmgmK1BxEwsk5VeAPAKUwy+6BCp1kc+tsG1tvgMYec585ZFtrDVV+UaA/Q5FKqBF0z5i5x7FPJUDnSAzJQeWUndrJz+40zYkBpS1IOQgZ6cAMacQuip13j/fvPr2/v7t7un/38+effvny05fn9799+fnTw/svTx+eTneP7fHd/ceGs2MzPAAd1g19db/d5n2sgwY/erv+2+mPW3YrCvhPy0RMZWaMOK7z+nodb5f9cr18+1Yqz7fL1/36cuyXY3/LPI5jr6RIo0PpbLKoaaKGLBoW4w1kBI1Oo1tEtrbC5EpWgIQkayVcg9Nxq3JQAm5OVhSoLVipLGKkwwxNAiYmRS+ChyqvphFBIOQr19AcXJ0ngLhM/4Aqda3k81W+eEOrtJIjzZtDGTO27eTuSMs+slZWS4jmplyHhRCQmMFMsxUyjJuGCVgYPmyxDBk5BhSrHoyimawtFaxoREv325BLZ1YhSbWelFOUBinhhezUsrkKdMyc3rxv1oxOd9e0IgEqbaZQp4RlBmAsXpkolYcykyYHmnuV/lVXHGmOOSKmahCkm4qgL+rbQDJhVCJyJf2RUK1AGaEIKZJRhb9Ak7gI3oy8dSdU7bV3L1TOy59sGWMor5mCDhit0CRsljNzx4wg5GaQCgGvbTePIJwhNJoxpgzIWeD5LYxG0ExBOTMGmYmZCrEV5W1sLZXKZCYkJ1MgTb0ymQpBoiAWnuVUQMzMqF2oaz0IUCkCoZlILElcNRMxlarkp5TECMxkFrVgBHsBGpEz5qEZyZyQ4sicFSGHdSPxxrv4imGx6vxyVOgDmvXWna225IpfY6SnpQwZK3hjBEiM1GQiBYuICUPqvN3feTu7vbt/eDzdDz9mbH03XcfIaxyHb3ftOI5jjzdpt4YN2strIVdrxjOT5ViLeYSGMsr7bMZWe1vKttMGMGe0Cq4gtKQ9IixHxlwKFgWiYoy8TC4oo2T9nK2Dzra5bTzddSOVDB7zKgwQjAC9yH5f5URRRgxlTITmmApUcwIINBiBhu2ubecGYsacM3QEQ2aG1kFjb4QNTc3E1k53p/PD+endw4cPTx8en/768y8/P3/8y8ePn5/e//T0/O788HR+d7b7u/bY8QScUbnBaMvS9cPYtd5mrBy3P2neZeu9XQPGWn8TEHLOCMwjxnHMY14v1+NyuXz7tl8ul2/f3l6/vl2+7sfLuL6NsUfsM4464upEpq2ucUFWzmGJP6JDU95WBIKkVlk45dsqk5eXnA1+o66MlpFmZKulvBTI6y2+3W0LS20l/tOURpHEFeQnqopYwAQbzZftXsmkQkrhVntazXk1RUEqrWdU9IGSGbEKRzf31mkGTIFpRLIUeio7QcyIhGRKIpYYpMx/SKOrlv0FvJooTGUij6xihEx2a2gIiQ0ygyB3qtktZUrMuj8AlZ5BSFBYCQckgImVllz037l3K0WdkJKlZHVHoD6bWkKq44dKyjqJm2hYQLPKzjS6AowCsyJnEoYxGyz26WePUmOUbD3KxLZUxTQZEssxUP0wXNRuAzu9u5uxNc7KNgiKnGm9+TKR3C7UsVzpGhn1rlbBT2WcvuWIzJH1XrRWoh9Ai/z3JVkpM7qUWO77Us2KjFCVS6Q4iAmtAArRks1oCZMr05JpAHxjzvSNcKjiFZzWjMBSDRaHCFPISVQn0YgVnFbx6DNJc1s+SgFsRleBcXDRzCpSz1AhzaqkvElFVvzR0l8bsKE1g5tc3Ny28hKLxRQYuzeoSsvgremH3B0VwDqZinEkDnJmxRQv0yYjlAprG7TyFzaSkW6kKXCE5sBb8Jq+5zwyPeZ+nd/245r7bjPomiMqjib29G5C5sjRI3OMGTGxTv5yhg8mCgkglGxIpVIV7le2hKquzKmM1ESMQCvlMJWCUGFCJaGyjU4uPt/Q3Mc+csqCWmEdAuhuaWYmRZqctJwr5s1aM0P1MMPL3SeejF2pqSNj5txjDpXt2xtpZKeAvm127g8/PT29f3j/7unj+6dfPj7/8vzpLx8///Tw4cPD08Pp9PTweNfuTvZouHOciAdhw5L6FENp+nHK3877hfUbqnwLmapI8Rvun9XlAlNiDs1jHm/7cczr95fr5XJ9+XZ9+fZ2eXl7+bZfX47r6z4uFaERRWCWhAQAZG5EoeSGrKyctXfUciWlVRIi0swqywHAyvj8Ie2zFQ0EoPdeIhIDgVuqJH+gICj1Cs0agEwhYY4sYKtOfGoBTKp1ghRpXidmllqgREUVwKcI3Tal/MGOJwJullo7Q2hSPmJOReT05eQsO2lYrRO59GV1nmYI+nMQZLPKUCFXMWkc0pEr5Rhg5+qtgm4AvpUTrLq8lejrlkNljlVRsnnV36F8pAU8VGZKZbgkNCtWIHUT7VBcS/FKGtMiLSNQt4+KG1wEXFtGbBiOARFHcqTNhNIhDrD5MgWUnF5astDCKrRSjCpzzx2qFgunNfqpI2kkHDF25nKw2pQ8EG4ZrMv2mLnnPKJ4V5FmLomsNpVUAmSG3JHXkChaRjLL8xYlP0ZFARA5Q+uvq6clquCQYM4kBAOd3Izd2qmDKVUquCo3FISdrJ7ZgOD0tuyzOUMgW6MtWwaFAvNRcUeCQG5muRTHq/XMquNeq51NQRBRaiIXUHnqScDr04qErBVaJtLhbs0Cad2s2lNW4SQyNWOalIEUnGndvbXCxuoZmnlkTGHE2JHl1vayq+WUOWMMms0Mg0+HrE3hsl8AHON62V8u8+WIS8aA6+31+6GRc479qtg1r2TOS4YGt57IgWNy+snBRGbp6GgcCY7IRDJZAySRcZNslyDAyi+dlepRFswfy0+93tWgWfICsSqw6hwwTmZEvCkuwnSHY4iZdr+ZydwmlHu6LZF866fWKeXcb+E1sDxCKJXEJhC3LRYhTKCZumdC7tb646end89PP/3y6dPzu7/+/NOX5+dfPjx9PN1/OD8/9ofH8+O53523u4Y74s5wR5xZmW5rzLcfZ/2fYT63/17Tmy1L14L+zXJmSZNjImLEiHHMcT3GcVy+fb9+f93fXi7ff98v369vL/v19bi+VduvEBFH6fEB0K0YOBWfUG/TirdFISJ1BwBe6ybo9cKtwbsOMLMqCcANrnIut1gtfCsATMuQuhjC0tRJLQesWaboKF+qxBzJDoNoxqi7LlW+oXLOuEotY160s5bMr36HCQVGCfQbCjZo3qQRUwkdM7JCdN1gZW5NKHPW5FB7RVWll6EfZlUpYPjRlVJDwYACBtcQZ7KJLcTm1opO9+ZlaxKA8DX1V8Md4VaTL26imsomq82Ozcy8WclqqKExRozKPFAWeVr7vwRLK41hgSyq6yYr69GFEqxPwgzMnHMeGRPHRJZ6IN0bIldgkISxApNXWg6ApoK4K52tjPVWtXybmzfRvJtmlr8/ZnAV80UmPKGJqE1oaE5oUkk2WgcdZg2kRDP3OgyPWWKEOZUq5EMllLBWSR6Viiw4UrBuilzOhPphlgbJYV3utJOxMr8ARdaHR8EaVv6JWSjNzXtzWon+0LyorgxlTrYypYiJXDln4Opdq4VaN8LZrFcMYtEVirkiKxalAbrBzXMmo4KPVqemqxI8jG6dJgpQRs65ophJBiISrF3QIv3IlOCCKWNer1F9EjOQmSFvK45LqapcAJjV1ck55nHo+jq/zTm/7y1yfN9fLvu3I67JmfucebjecsTMyNytp+qmVKvC7RyCK2fSYE0g6EyGJqWMoJTeS7629kiWJ82YIwRmTpRBlzCXZVmuAAdtzf51Mi4fjNGau5tmHKnxEnlNXSBOa8wGO6d81ajWNEegnTd3MyJuUUm3ChDzzfp2oiFTc2QMcaKhTQ8ZkrRTf3j+8Onzx1++/PzT54+//vz5y8cPf/n08dPD4/P5fGft3h7O/enUH9sieE9AIzbCDZ4LXS/28jYkA/8K+NzG1/wT9ll5Pje+N5Ezx3Ue+z6OeX257G+v+8v3t5eX/eX75fLH9fIy4+14e5txzKqiz6h4LpAVguLF/a57qDQ+lRlbt+EN8sYtSaqUNlre/CU/w3IFY+m/rP5OyTZK17VMX1pjUt0BdNecrTdjuSsVxmqZqLa7snclC9WRkhW2x2Ik2UkSCZUEpXDbXKkGyAUE1ylQytMYoUr0K+NSZo7QAS77qZCiqSwqqnKA0rSiLgWrfFTF6paMct1VJPDtNo+Z1hZFTTM314qolhrKETUgB1r96H9kJQWWLDGQByRLoPmCzOacQ+tzpAJO5LqxaplCnf6syMyoryDXN28IVB6iew5kzpkzNGclS9NhrdKdmBmFsmHWDa+bIUYZYIN5sXlQhLnL0s2sN9VRWPktIbCxgwprgMu8iquKYs+YFexZrIhZM9uW36m630RoIOBMS6o+dglm9M29wbqbVbM1VCdOTYgVpSSaF2UkNtpm3mlO+oqczkxFXfarE6CEPCrMujW2istR6y5DBgWMjExmlRMgDbcrBjA3uLETpY80uJkZE2FORFFOgtFaQ3m1p1j62kxAKnIozBqtmzlhVBoAGqNWv0jNQCKPwJLYgmbefIyRbhlB1cmOMSY4M4PVRV2TWVlTzJZMImnKCKRGuF+ObzEP07fWPOecmMd42+MqamrEEKebujLMExa1Ei3KXcjJSqwD1O5gneYO92ZQTEhJi73kxkhUl06FZVWLb4SWDubPCIF5Izy5vP0imPAGNu+91ds9xkoHOK6BUfZD97Lmu5sFE7mRjW4shHMeiEMp3B4jM/N22txciEjV4gczhM3M7e708On989PzL19+/e3L599++eXXTx9+/fTx0+PT8/nhrp3O5ud23+2+2RNxhxXgvJSduAW0/jjrf0j7eYP4bydu/vint+kaGcgsZiiPI2LMse/79bpfrm8vL9eXb/vb63F9vV6+Xl6/zfE2juucRypSschvlsxu/Z4SKv7U6twvXMbKslo5AvU2rUsgF0HBtbOQtbL/EEzb8jbVRncD5CVWsiJYr15lwWDCyObdYlYtFObUgu61/P9l26u1l7UUO7mqm5BQ2bXr6C3Mm2U8V4VQVDeNbiU5M0Ixg5sxQkdl1IsrzI4pebN00EQ4XHQiCKu1HBqh+qMpRRW7FfXdpTUDmWAOohMs5X9RwZKjxhlFmkHOzLpQiRSmJIxMJGIyg5C4EQzfHIqMjIiYwUxvRYNUIsnt+q6+KmltZKXNN7I0dCrFPTIQGTGn5iyDhnk5QIWm1Rv1w+JdEUa+4j7cup1IwyJgmmWmbQ0GMBmIeVt23EDBa0ljIXqo2LYKck8SXBLajXJUyD5MXC0U8nPjoAra88Xnm6GcpG1rBCJSoy6xolLMOskl9jWgap/andPdHYv8mzMyFeltGWxYVT0ESPNmrI44sJkSdmuxggzWEFNEbRJOJut7MZbbymFbSXygSJmFZtF4rJCIlV9VV11SijFzKqEk/WTeGozVp5FW9qySvQrKRsw9JGQCDawyI0sk4jjkznSigvYic7IDBiu3eYm1WeghlJAnwxQB6hjXI8aON/cNynroMqMgsWCqEXGjKBfHAe9W6GfRb+6UGRvZ4FsvXAdQbQ1w0qhYxgsS64VOEQzBb9hOpXQrFv1b7k43AxUpc/bNTlsvyGtWbHAFYTu1sX7pJDCVltaRYju10mFnaj+mdswpd8pkJehzIyRHFVTYdjKXnVvKP9zfff7p80+fPv30/Om3L7/820+ff/v48fPj04fHd3d+PvnpZOet3ZvdG87EuZTcWlFuMFj+EG4u1PBPhU/e/vpflZ0/Bn9hRdFlaF4jM4/rOK5v++WyX96ul9fr5WW/fN8vr9e378f+Mo593y+pUaf/jJmZK2cXJThOYmW/c9X5/rh8SvPMxSIp3VooVPFsXHsBV8hzneg32RxRv/JNpSSu2Za8fXsFXlcyG4hGIzcnpKjDB+YEgl73RvkDSw+6OieXmzulKVTViam+NRIxocmCT9GQUi8VPZFAZlKReyDFVEyUu4qM5YJAVswFvaJxWAHxBswjax4BEaFVq5gECgWzavgrCwWKny10OItuUWoyglY5ZIvUQiAPUFabnVYnl7lbtaBSimNIyjkQAqXJsjdBpScxMEyCbK2KodqHMJCWKbF5XeFSRoQKCEh4QxAkK2Y7CcwEWd0sdTrSzSg2o8HN6Vg65EzzxsoSvQmlShIDBQg0A2hbNckIYEzMFNetXSNo6caSdG/M9aOrtVzFkRqVTTKS7uXJqNe1yhXqManOa1uXvUrxOQGDb2aUU1BlY2UVghe9bfU2ziVNWMjCOttMMsnjVsKGkDNhEtRc7pbzVi1TDttGa1Za2kiJGeMoW94KN5AUoVkRpnDZiHK4GshWjt8yCs/MFQfeMqOCVlbT/IJxwQ6YBeVeHaODcyrQzFV1E5ZLFWSrMK71MqcrwbyGqkDRPCsUxXX7RVKhSsfsjSFw6TWZI0q4WjqG2kHrSVcnuktAMzluiUVF6FZ8CcWVrEcrLxcK9akjV5V9vTIABYpZ0A8dhBsizOCtmZkGBckEpZnotLO2kyGsPmGQEckjI+BWHwEBIDKHdIAhTNrZuewpFjMxFDRA7XRyceun+/t3X3758uXTh18/f/71w0//9vnnX56ePr97/9gf7ttds5Nlb+0edjbcFcGLJZVceo+b82kN+HVI/Dh1/xX8Wf+OocDnVB39yJnH9cgZ4zqO/bi+fN/3t+vL1/3ytl8v+/Vlf3s59pc59zmPiEMMRRT2ghSX/NIKPFxD89oilx8AN1E/V2xYvYYlqwAX8Zv1L9bKvNhKQlhyABbeAVRVQP3/gDo0JOkmq7YZ0QDR1LZGOEpiR8BMGdUzkKB79zXDx4jIQMxcqMm8/SBzrfsVwVb+w1Ss3IA1Gq/m99vNCutwu9WnLSiFLJjKVrR3aa0UsMYY1XNVayE0hZxKVohN3ELTls6JdeBQlXA8dbsxiNRKdRBiD4ma0lTMZFkUm7F5GdDWgQ1xnYkO3tI71hxqKCo4QlMp0hvL0sI6QIE5AjLziq7MlDXAIGdxxRBDMhM3Q8iNgRuAVrvRZn3bvDndGSOJGKECVWvjMznNIzWVJjmqJsx6YySVdaRQXTOUooNu3o2bwbxiz0KpCVVwMKGFgMC7Yc32hVZmprJU9SkUMVjQ49og08KgguciS2aYijkjhEinBBgr6ZAqE1hFwEXYVr3EztCMMAMjcqbhxkgR5uad1XqnzEpuyGSGquwrAjkTGYaKd1gcRqRi1gEuRFqZZ2wZ7GGhtAIqRZi7g+49jlF4dWIlnbTerMyUdedJcU2SrbmOqHj+vjVA1s16F0QymW1zCgipMwcIylELM1aetFRRG4A5ZiGybkrItSAKFYyQS+1aoJsTFhRkSe8yySrDE5m0zUsXbaelG8k5TcoSC4FpMGfOVNR4KmURhjflRW2DrZmbkkdMDpGSRYm3uaHaSRt8HpmHMpMTLrKTBmYoEEfmgKUh5WZFOmDBh5yHrLf7+/t3798/3J2fH99//vTxy8cPPz89//Wnn39++vzTw4fn88Pj9tDs3OzMdNrZ7ST80PXfTvu8Dftmt1k+Wejs6uup9X1dDCvRwW7BPjNTmCNzZI45r+N6eZvHsV8ub9+/Xd9ejuvLcX07jv369n2/XsbYM4/IkUxVaa40xhRkdMorq7FEllUdVxzADwFhzWpaejx5yX5KGBoZGBDSaBWJr5tYRqotVVKyRCppbqpo9UisuwR1oGSUfIKtnZxbzzqI6BFQRpbOlerV/I3WzBU6jjRr0xJc4ez0ip649bYtHW25eipMExHJrMTGAlyqDh4kVuCCscIMACwoIa3iak2s6lGJWjTIchnlNChzqQCVJq1nNaskVa5QMhkJ3Cz8IFRQ2cSMiJnrXRqQaEk089Za73KrxCCyIaeqm80qU0lolV/uBfIrlZkst+oUV6BubZZWEXnkaqtf5oRaUlo9oKWC581EBsLMlWsDzbaZn5q50R1VSFK0zwJJSxwrg+gJwZd8xgAiALHKHSMWSUSnulvz24uhmCkrE8Va78qUkIDLzW+LRoFZFcZT9SmAs7KwC4mWUposzrNG+sK0at+yXLRKmfDk1FDMGrtzvbmRbA0RSCIlKGZIkiVL27V8CVn5ZIq1+CEkWZkBM7KCx8myltfTtXJRS++qm+BTOQmHkLGkx3QiK3NeqykiRGtA9DubAWvmvZbO+v1SSLAdRyLU19VuzbrRIMo0lOUeMJRzp7i6rBA9hFi8tmDkGGrdVrTw8vbUC1TKX8GQcZMim7XmZlxPFGGOyCmptkPvRT9m+WwLQOCpI2T1b0uzGkg85nVAyBEL8C23aujmvbT6p/MIVqBvzZld3swId5NkJ5smHGFp9ZZpZoRySiJi4diAEczVPmRop9Pp4dOnTx/fPX36/Pz5/bufPn76+fn587t3Hx/ef376+Li9e+pPm991OxFn4mTWiA3gkvaj/Lyw21Z8A3WW1ixvJECJarCexloSUpmYiJlaTVORI+Y+5nFc3/b97XW/XI63y+Xl2/Xy/TheI479+nYc1xljxl5JZBkhRWTs48hIbx2roTVhpoiycd18klxFHasGMkgwS75YxA0z5noCWRH95UgkUwB/TJARkzQpCASSK3eDdVAoa0HPTNEsMxrdZkxvbZU7WL2eyWbNffmzFoZNGDXCzYCw9ieIlokYa5enUQ1mcG/LM26r7mdpCAAZevvzGETFelEpMGrpYCBhLJa3NtUKIqwzhjeArMSX6EWIL+IKDcDIkK03xLDsRwLrvI6cyJmawECdPKDYmttGdljz7qV1yzmZVpoVgy0evpv1RtCLYSlZPsSEKZCZ5RsqL1jUdGxafHxN9qiWJZIyMECjVZlJDYAslIq+dWtu3syqSiciFxlatwQSqBhrJGRsBrAZxVkW/FgFL2xsmDNFNfa7XhtjLvkxYkYhLRq5nGc07wZ52fRiTiVWLImosY4+uMyFJCuFe0KzGlgqQiIBVwopjXpoSMir8k1iAGNtgIUStL5m0iLDM6aoyLBGUSwQtwq/osorVPO7dTKXsr2uY5sqDXkKq0kV1YZU8UigISYywJblDK4nkJKbO73RJhQRoAtqJyPZV344gcwAS51lLFONhQvIDHU0NkE5YvpKPZLSYTTPmQpSyCEInCicicWdbx4RBFsvYwEQZV4sKwEIta3zZJUw0lunldVrEqkciaxMF4BuLiYc1Qzn3lBFxd1c5mwAcmpkjIQYY8ZqOCXWliowC4+URuSUhcZc5Ja5UTkz+8llqru2rp+6/+aeutXMASwKX2QQCLOt3b1/d/fw+PD+408fP/76/OnL07ufPj59fv/u07vn9/fPj3f3D/3+3fl5aw8NJ4OXshOrrdduGf3/5cS/jaOGG9N7I3t/qEas8k2QkC2pjzJjZAZixHgb8ziO6/W4Xt9eX/e313G9Xl6+Hvvrfn05jrfIMcYuacaemkrEHGbMxBxzjEMg0jWnsYli1B0uzqQh1rBfawFw44q5ls1Ku4o6qEs1lMKPvQFTFeBBS5rNGvGK8q0ip0xYNZwiU7ehloJ0zBaa1rtMI27h/yBdrNSoklTMBIQo2DTBtMZFJxfUEuQJS5xaA0snrdoNjWWlWR0bFNRalZP/gLJK2WkRWcEqjGLk1GurJRGq1tl1ihaGai6HpclAX7AQYXCQJVJMBJhZK17JsFHG5Vwljro9kHS33mnO3mzbsDabKWnGrH8pbknLZmZkqxGbCETdSVDCorhn2GJks6hwN81cQL2W7xc3G4dMpGh110GRsfIrFjFKMYPBVHHRqDCbkKqTPM0h8zTSFnGq22aZqx8TisEgmltrFfizmGaDZtgEFJlZlVe1Y0KSZ06okr+YpOkIJFFZaRIcaqAzgxIR9ISbcWSmsFmVW0YIQzRYA7KtxXRGOWIJeck+t1ZCvaWljcyYpdnLgLVWy0RO5fEnHmIUnbgmrFoEqIylSprQBCrAr7aywvndzSswJFbQWLkbUwBsqwovRKRIOzlhEry7EVmDGIE5aI2KZq6mGbKgl59WweZhaYF0iHJPKTEh6zlFJWT1cWcKmRRMnmZw0q2fO1NYqTuoQJ6M2dn81FtvtrV0mSXNHB6mGEcUU6WMiMoSXMJBrUgfsEFZ5AGUSyVQNetpcaTJvIBVR3GmheFqQg3aI1AQETBEkd5AJa1WZaQLKk8ZtJoxEjYiZgjm5oZm0816d2t3z4/vn58/fHx+fvf+y+fPX57f//X5/ZfHx89P757uHu/P7+/709bODfe93REO0NCBjehEx9riQEBm/CHcsSLL1j/CnxzvgoduTw9UcH8U1h85NY85jzmv89ivx9vl2N/2y+Xt9XW/vuxvl+PtZcZ+HJc5j8wRRRBHCIo5SKZizGM/rhET3jIm0ZOp9UarErcARMSi+W47HisKM1ORNFcGDUyOmFJWHw7dSVMEBURpDpYFXznYbEZKMgChBRtELrIIApkGE9ucYXC5bjzZwt7NRM7q4J4zbpQEAEpl0vGFYoCytKwFGUBl80JApSRWhhcaFA6mudevoTVigqsfjzAhmMmYMWfC6C01F0IWNbJ1FltcNLq7l7dRtduxtORLgnTbF2QAWu1Ai10EyAIVHEaT0a15b+Rm7QT2XPJLCeHucxww0wS3dTkTq2MzogTLqaz2AlbwVtHiVdBSvjMBGVkH/p+BIpaIsnUsjJV1Fq1toaLPhZggMzMzGdHcynVLZM60zQvF1pI4cJZzF8uuq12LaNpkTtsQCiGNHjNTypmYUR8ihaSQzABCsNR6mso9CaUpg1rE3nqyKsKjMjtrHpSSNFrloikLZ4tqVyngqyp/DWKTNZiTkMHq/MrKXIZQnsmCWkqFEMhZYf3yeiRmaihQvWeoMT9S5daomS+LbD+ERu8m1MeDAhAL3uNSTEsZuoVRuTekwVvfmiLMfMbMmWPWk8sVacCEJcOY7N1r40nPKKUpki63BiUCcYTSGIw9SaTYejO6t83c6b7k3DFzxlSiKfbZ2l07937etm1D734yeMQYEceMuc89M41S1UuZiVKElMHkj2i3CUUi0cwnlZYlxMofNumoGhZJZZVZ8nM5rdt2ckTMaziZAXoRJ2Iz7VZ2oeoBAo1tq7PC2L1rezhx83Z/6nfb/f3D+7unL58+fX56//np+dPT05en9z89Pvz0+O7j3cPj6WHzu+bvNnsH9EQztFwmhDtgW7r+BM1+jPALmbj9ewviW3//h9p/NfXWv1wC/5iIfebMOea4Hsd136/78Xa5vlaMz+X68nq9vh7XS8QeOebcI46MGXGsC7IuPWnM422/Xi9vNDiJRICovOH6PatuLyptjPN2P8ng4Eo/oDQnijkeCS+dQGBCM0nGnFV/N2dp9AUl3VT9nWBmIeSZhRRVXAFEFjzG5oZ5TG+k05u5NwNkWcJp1bxfDTMVBaWsd2Tx+RVkWmnmFfOtSg0y77Z6v1cNDUHQ21olC9KSem/mzdwzNFdSI9aRCauAyOVKrB9EAHI6S8S3Pk0KDjrdDVXTuk4/Zt2KNUijksRr8wSyXNa0ynvoG20jepIrssrM6E4D3a1lDCYUSKRPLLcECkAQpsywXlsrE0JZAsByOtSHGlqB3XVlZSpQ3BdXwQPo9BoREAUGxkg4CvIqKoHraIEyrEqUJaz9vHxoS2igGt1JBa1BrbQws3pER5n7EghoyCpbeoUhQUGYMmoi5gL4B1LLXJGgZKAwkVQ5D40sxicsaW25fyLrg836TGv0FJFBgi52Yy9rlJQZkYmyKoFlWreCq4SaP0usFGHLV8WYQiC0YPGca/Bbq14tKwbfAKKdnKhGIggWIQYSZXOllWHLrEQRZAPcvFvryjT1DCEthxwNM8wJc9IQbiamu3GOtNOK5aqQOzOYNwiYgYMYmaNY/NK1ttZ7Z/e29dbpzcEYMfMYua8Hnloxbv10Pt+3rafl4Jge+9u47td5THOhkWYlY11JHQanwUwI7IwQZlAMMMVKEGcF/uSNNTUhBKxVtVLF3Fs/N9+sMpkywENslFszWveFu9acZWa9wza4U621dufb3eO5n/rj+6enh9PPnz799P79L88ffnl6/Onx/ceHu493j4/b6a4/3PUHxwk4A2fgBHRDQ6nEVltLW7P+LTlnVdH+a07nf5f63ACfugMCCpQ0OUMxIsY83o6xH8fbZYz9+nq5vr0e1+/XyyJ759iP61Ua0oycM47UzBg1nJWkZ4553fe3t7cRg6Pc5MIM8yal3Q5MgrqRY7TVA0PjyhZH5YpJoDJXejpRc1xEVgADlrWLysxIGqqiorwFFFjeK9UGKJWp2GzMSdZUHGsJ9pOBmpmzAp98ITp0OgtkVwQ0q8m5fknkrEtUVZwBwmrM4HJPVuR70XSFtk+pbmU2ozVaqzPSXIqwZiY1dZuwDIMiBTP32pwkhPCntl0uke5m3dgcgORKwSFNZChZ+GcppKrPS+P2w2d5PbujlYczE6DEVHoNhUQ6JurHOBPlXZhCTbKhyCVigK29B+S8CVC54JhC61F0KtMiUs3Y1lMqSclbVRltgvQokT0iR9TvXltTXRrlIQeqaKNi5lQCcY2Aec0k1eeCzmyyBlZTWMbMWYK5lhZHWAaTMRf7VKMRi0ury54mIWeaOTvpyB+5FKG1bjWRUWBL5X8URU6EUQS8lTcCUwE0VJyns911VtxpikjkrIW1RFZYpmPUHUxCASbLL4b6Aub6SVZ3Q4UUFb5U0V00WkPbvG0lEFSuHy/dSkC8SEk4vTX3bjTBlJUu53OGLUEYFWgg0pLOEnUInAqImyuybc5OWXp3MCOC2AqbwRQnTQ5xRmQkujea0c3cem+tOzuT4Bh5KEUpR73KTLMlEc2ZmtdxeRtvse9zTI2oYheV57YRMLis+eqq2stQTYubeC+Rkrdb51UD3PNQE5Jp9HXxV2mSVcexBGQj79BPbGbWzLrLSomxCS52a922E63ztLV25/30dHf37nT6/PHDh6d3nx7vv3x8/+n+4cvT04f7h+fzw8Pp7uxnw2ZWvYwnof+Z3QYrrB/LF/tfxZs3b9ftf/7J9K4Uzx+ccHXqFeZzAIk5Yu5jHGNej/36drxdr5fv1+vr9e1y7K/H9aUUPnOMnDMjUlOYGRGaQqSkyl1yH8exz+P19fX17QKYW4sjSlGNOSWZk6wMsh8eL64kPrNyh0EVNXTDMPImG+cN2MhlM1ImVu2XFBJ1w/GxdGLl4ym7VKpAjYBgUqjNI8qY7q3NOfd9CpoKa+xs5kaLki4akSmiso3L1U3cshPMmtdDdhOyZd526cyMXGaHrBulhIOlzBYQdaaZA81C9PrfmDo0DLBVI1CgcaS8tVLpFHzuZbGpWAyvlkurHysMjazELqQQ0qwfPgTSaatQZmvcUh4FJyzDdjF5qUrAKDrZarZKHZnB6qQG/0RaVXExuVo4l4MqU5IV4p/JsrM6rejb+r18LTk1FFR1gyYyKv6i/M/1qbOUAxVwAGNjM7qbZWrmhCxL0VKiJZId7DSntyYlytAeSQlRfKFilPG4ygUQAzJkBZE7ffOtbxAzFEeQgkus3y6ZsIX1m4BkSRtNLg0JXAn4S/KU4rLR1a3Z3IxyX8KNWkuWg6WKulIKlgo/Y6WRF5aY5WawIv/pdX/nD/8daEsZSNJPZs2teb2EwAQlBVS28epZgzfvW2/eIY/ETCqZMxJaJhGDkd68SE7KFESAoZrw7NRohgaazDFzmrtU2uVENCaRiVkEVjOxUjYT6VFGPSBizjHniDkUtQenUQrpkFyH9iOOfb7F3GM/MGdBg0hDg/VqsFFB+WLmCAlkPeSEEDczo0qw5VY/GHoy0c1rDk2Cjb27e1G+JG4RMh3N2LaN3oKGMDudiFOwnx/e9fMm8Hx/d95Od63/5ePzr8/vf/nw/vPjw6fHh8/v3j2d7h5PD+d2v9lDpfMDTdgM54SzupUL2/3z0P+zn+u/Eb//MvXX6f8DCfpRK5tZLpCZeWheI2Mex5zX6zGO4/Xl2K/X19fr28t+fdn3y3Fc5rwe17fyghYVEzEWNp0lhF/S9Tnnfoyv3759f/keM9C6lU3IiaXfXynxuEU1sMQXt+OGtMiyC6W5Lcl+CV6MWUUtrLVeC+RY9qwlH4wM+o8liEsrIYGeMwuUzhRcSDUJK+KqnE10yZp7683N0FShGbQ6xpfCNCJzwtpKhiLozVQDnjUKWWUoKWi52EhbwsiCYsxoMjPzZSQwA+VqYOSk0JTVdVVlhze4GE7vVhlEpT7rm9d1UZB8UuUKBYG5sDAcKp1ZZc0pwZVE6N6d8rX5Vxs5JUvMSaZ0IIfmyAgzsdObma+iekVWOKtbru4LQ/GJKVBWumc3lDa+PNhGlqAiIMvKckVrnreHmlzt2zHDvbD1qGfFtqJZiuFngvRtkd+oW1dZHWpgQc0uqvxZtWh1albONZFEVsknBIQjBLS6wGEbUuqn1k8nc3/38OjNkbq+Xg/sGbOMpXf354xZok9oJa3QCRMyWTFpCfOGXjohmRjXcBJT3GgMb80INrASnyXQy95mRlFsBisiy8G1EdOthoMSLMA5j4qyKVSeFVtvRmtWyk7vhRgFpZizsvWXCoDZt0rNpvWttc3pKR5zjqrWzXrUyxcJI5RzzhDhMIUNycoCuZl3k2CbZyYaPTzmgSMjA5kMs7QqzAJgASsCfk66IvYf2MQcI3PmPCKEDDPkPNgcsY89MPeBqePA3DXGykEtFqU5fSkdtRBi0EssYaRyB0p3QRWm4puB5eYHqUpyAErJR9uqLwNaehAxsbk1Z2vNNj/CzM7eTv3+yX2zfn//+Hg6tXNvH5/ff7g///L04S8f3/3l8en5/uH59PB4d3d/uut+bnZGbrR74qwbvFM+RqDV+vcvJ/tC+f/L0f8vST7/kuuwROkqmhdzmUUCMTJnxhHjssc49uv+dnk9rm/H2+uxX95eX/b9+zH2Y78cxzVjRM6SIUaNgxE1RdU2mJluJmp/O769fPv67Y/XlwtJ2qS11nrxexVKtXIDuGgJoTQpYbae8HqPs5yXdAIxZrA6ilbwj7IoRbIK3yspS2t3vuFRAIjUkjCylG9KReV+IqMBGEf45k54K8QG1t0NtpzAAlMjMzJmxRBbKYVWm0Rp0ixR6iOZxJiaU4ValralQGVViJyDWRYocoVpVd4bZVb9IVOSW/aUZG5otOawWtfL7bksCO7LCFVk3q0qFwhBsHLi5kpYM4HreDKjk47q8cItjYpi9RIzlAcZbgkTKXOwWdtMKvBHabAOUPXzoiKJ1my5T5RUNmMJe42SrT1EZbW/aUJLcWWoqEuYewMRQuSI4Kqdudlxyp0jZVrxoqx0gYwgiq8n6adKuLRiQ1FBpWBGOlHBNpny2hG5mLJCQ8yIpHKS6Od2d3f3dP/+rm8pjf2YPMKMZtvJW2shjOPI7hnDWq7fqhUV0HJKs2CvMmcYDDnST6zjT000sOpDRSgjkmYas7pzrZjMUpfW7J9ZUtea1mu8qTfK3WKutkWUyaFq45o5Ufby0jNlebVNzW2BkKR3b24Or+FZwRmIkTGnIXOGu5l1pbuZqZwf5UZQUnCDl/KoWStNKQmktAo7okaQxEgqGLc0byAlNyBnKBJhVVEdZd/IRIBit+KwRR3HdSbNY2qO44BHuaAzVy0xoAgsWyUhyAnQeRPe0TLgSLEBhDVfxNnKhy2TODVlNC9/FZR7wMioeYbYHO0Uvo10+v3p8fm03d+d7u9Op8fHhw9Pj8+P50/PD18+vP/88PDz4/vn+/vn7WGzu3O7b+1E9Br5YQ3r9O+AM39Etv33P/TjP3M1Cf0rFCT8V1tvLkGKlrhTMaTjR2vjMfbr9fVyHJfr5fV6eT2uL+O4vl1ejnGZOcd+jRwR8yYWyoyo4b0QNtLmHDRE5kB8u7z8/fe/f/36NUNkM5NZch/WPOb0bqVuKCCtdJ4CfuiYM2VeNdRrVM6a5ZQzS2FSsAIzpSrsq08TlUwWpVwv4q/yhyAYMWbJb0kImbO28JhNgX7yG+YiRPVVR8BipkZp7wtOTa2tXe7W3NzZNisttDJvd2SFqgiRmIJZpT6sq7t8KwNW7/ssnAhw1BxKReVCpSiDmnnBst1Joq+wgEU2dK7FoNTbYkulCZiFirkyQpVLIk2Mcp/WyOgyJzeyKU1gAFQAQQkY5KDNeYxU0LK4Geu2xK6W9EqOFByyrCDkRgJRRQdYuqzFo8AMRHPmj6y9MvhImIALQDU7FdKuZel1OVrRucxC/4obWpbuKsRUmGnWPwS8mTVCojW610egXNF1FcRW6pok2JAJ89JGGa0APxapfDrf3Z/vupE5NUccxxhTie3u/uHdu23zt2Nc8mVWHmzuMvSzsURiOQlORfknrP2AFygYG9BkNOvwZtQS9TOlTMaazBkFXUOpiskrWVPxjcs8C5hZaR3MueDRFeVZlriSVpVzPouOqSvVmlur3j323po3JDOY81ZRcRwNzHFkiGlhhThVSTy0viXK4e5EI8hmvXcpc2aM0JzKoZhOqGLMNR0bWLHAmUz3tlS3ZaE3ghkjFFEXN1vZdGHeIlKTxtOYCATY4AaMJcD3ZXmprlgsFJggq7R+Bbse6zWkAw4Smcs0CspUdrUsbQ9bSQdQ0Lmp0Tb0U3+4O797sH5up/uHu/ePD4+P/e75/uHju/PHdw+fn+8/PN5/eHx4fnh6Oj0+bg+bnTe7d5yIphLk4gyYYIKtFAeY3Yy7db7/K+BTf/0n/vPjuK/74Mbx6nZV5Mxithfms8+5R4zjeNv3t7f97bJfL2+Xb/t+uV5e5vF2HG/HuM4xQhlx1NG6LJmKJVimoOVQNKOZ7XN8f3v959d//ud//t/L9xfa1toGNrKVNRpInwYtQ6xZhT9aZKqi4/OmXSneT6sPcY5IRaUWVl0KgGXvy+W5A0oPsTyEkbN4gnV/QJG5gqMzjZj1DsVsBOKIdtel1KzKjSzDYJ2pmhnlorwtXd6rbTaNvQK/uKZVxAjk0mUzSEFjIhSVUs01E6WgAaVMyJIYBMvGCCYlNnkqoy6/pFtJ05s7ViQBF39bnfIgFVbXdMzlRkhHwqvgYkQdupmyVtn53diBVprvUtGSMhdiznlQQ4yy1xEIqHKv3Eu7Up1iKGChbaQWFbPOZqiWozJrZKpQg0Veu6p4psgamlXUjJVQceYEkASaG7h8k4uByNKO181aJLNAotTfBnlFkoq0kkup3nekLCQhI+asXIN6kkufSNQ2R/XmAL13pzdrGbHPy2WmOOd+zDlP57vHh/ePD++8ddhljmPub94IO9HhnlYaIiQIb0ikdzdfm9lYTzrZzKwudUKlDcuVF5KElV0e1baqWRyScFNJYMHQxMo3KhlqkehYwqriZW5ppuu+cEgos7lvdHOA3bu5kx7CiNTMjMh5IKdQPnCDMUnGiDcWellyKQNliz8wsm3dSciHMoMDEfNV6DNvlgerhqzqKTMZk6wanJuHgwXVAkSJQwx0a9bBrsO8bUhHGLsRM+c0o5RciyzLUEaTAqqCIIAVeisgYMHKdk0GwJglhBckOtncjXkoUh2GyueMOGDCdn582u6e7p6en56e7s7nUz+/e37/fP/w4f7hw932y7vHT+9OHx4fn+/u70935353ao/d7hs24uw4YRnS6oNvNxW/rdjOfwV8bk7dP//Oj9Pf/vybuh39N7RnnY+ZGUfmUdnDOfY592O8XY/9euzX6+WyXy8F9x/j7dgv89jHcU3ljHkT9asyvG7tR3ljWFXWJwIRuV/33//+j7//n//z+z9/n8fwFsYJazDvrdcXWKNsMXz1f1+fNxxrrosK9omI9eRHFKZV2D0IVa1DCYQSKhHeio9ARbPrZnNb0x6q8S0i5CSBCcxIRzRUwNCYNa4JIKuUvgTmylg5imaICTqYqFQyoPDmsoNLdfRnrSsrzYghVYe4l/oeSFSpk5KjdlSD3AQzJa36pFVi/6wZ2VYlGZ3WytKoCaGaNRbhrYoNYMIIRdSijUxEFQ5RklnLtRYXCBega5GQUP3oK04jIzHrJiwJaqi2q0J/6gSTGdlNXpeyctYVzUqSp5FZH3slnLEaOGAmQ7l73J2wXHR65pK91/eSXtmJiLpAl94zQLFRGVlhggnKJMCq3nIEzg4oYioYE0BayMwYtRsmDe6odLAapURK0bwT7Kc7N7OEMmceMxSZGYdiEvRutnXv5z3zmrGnwkCqb81KpIksQ5mA5lTJeygHJtU2gyxmzWxhrVlzxYQsjyiqDG4gWFJarKNcJBuUQW+UlglVUET1g9UWSa9Dv14ymVvFJiuQVteNW6uUwQoDT2MXEcHpiJhzBmZ1LgeQRW9AdRCXbZ3l+hNZzsPm7sWytM6S2GSEZuDIOVOtZlMDZWzuZTTXcicLleabYDNwKdxqWmAlhTRXc6IxWt9O2JupyZMMxfc682wTIeNN75CCmyZ1hQZWF9DyzBMkz2ZuvjEU62UfLmUvAOoKhQzOWRqqRjvfPT+fH56fPn5+evf0/uHp/f3d88O7x4eHj0+PH+/vPz08frq7+3B3fthOd9vdZudmJ8MGnBxnLqa+rTSlm6on/0T5/2R3/9sf6+/nojT0L8iP8APqyQQq8BzAnMhjxpFxzDxiHse4jnF9299eS9VzfXnZ99fr9TLiOsZ1jn3GjDmXnj+jsJ2SG9TVmMpGF9bsX7rC/di/fv3jH//5f//5n39//fodQmsSB1sz88nl/KnRD7opCyNWwdPC9VnXL4TQ+leWWaoAu3W2r7Qf1Heat0YDQss2nFzCt/qpMbNuKkmYFfu6TtfZ5oB3xChZt4pYG29plY6g5ZFjNQQ7KFQ/lkSNmnajQJEM5QQyYgiwJdmetNKJV1teCZzKzVTOGKscMOWRlQmSFZlFo+Qy6zd3UB3ZUIYYCWO91RFRg3ThLeWc1lQGIShDc93UNNcNB4uMmkBobG7mNCKUImJM6uYfKIy9kkNHrc91yIiQe3FrUmJlg7LYDrAZsALGWPInalntnIGFALqXdB8phdKgTkiGTFVOGVSeaRiQNoEEKoSbpc9bDwkKHcxafxI5KGRMRFJTZqLhyHABJi+7RoFZkJyhWkU2Wnc/NTQmFcUXVfUwl62jt+I3rvN4G+Pl5XXGNOSKLldJnqrZztotWxOKHwJeoyOAaoXr7iwXlRFJtyVRAEXLRFHuhYHVg8VWKw6ACpheNNoS22H9OFKFjjiqvKdSsQxoXooxgpGJCXaHqshsppg5kUHJXPX18IdQT1rlWLKcoDUZ4ean1qxZq0YCU2LMMWPMYx+zks/qGaw4aq6w5fpihfXloaqgyjOkRezXswaSzhWhQw4iCDm8p65KD6V5rsQqaf2Z0PUx/iMAAFoZSURBVFQO4c09DOnFcq0huyMhX+mKINJpNaZqRlRGvHWzzbz1fr5/enf/8PTu88+PT8/PH94/P95/fHz46fH+56enD/f3zw8Pj9vpsT9stp3b2dENTpwNHUvKuSZ9ZOmN2q2V5V+cu/9/Tn/8YH1/gDyFC/2A+LHk6TmBCvCZmTPmMed1zOOIY47jerxd9+vrsb/t15f9ejmO6xhv+36NGLEqP3IFzCzWfH0G5R+sXtFVmBJp3QWNMb7+8fv/+d//6+9///u3b9/363DjnCJpvWeqNJQkFFfoZFYifd2ulGHsP7yxK61hebQnF+bF28h/C/evhpSEgRVzqVJ1cR1NmenulcVQyXECFOleSD/dmWM2c8TxY0cAHRoJIXL9xBfpD+h2Z0cpUlXAvxbQX1dx1Tc6V7OfqtadZq7qFM4QoqznWPU3ilQcs/eOwiwLx7YSPFXAGNbVFyv+UDHr4awP6CaZhTUrgk+RQKsZvdSoyxNopBA5YZ6GUKlJVyChKTnjNm4UMnDLzQAgalRA7OKPWS9pIqqmqz6SFIWVQFcy16zQUopuQjM3txlRToaUuNDvKn4zKjRzfW91n5S9IJlElAWqPvNuVDWlgpbykqCKQA7VQp9BlEC4Dt7yKjiXqlUUmYWXuNGboXz802qGHaPwinR4a25m3lKQzcv+x9vlbVwP6dhOosQYGcNEq/b5YkBClOWIKcjZnIVXZkJimRZrsyw2PhDWPSN/SN1WUnj9bMysCYlyy63AE62UR2mpoqvUwM2LMFgWDJFhFfpXg1iYNe9MS1R3i+IWr7aCIFI0q6FFqYisM7K0sWJ4697Nm7u591MtwZoz98uMOfYRTMW0RnQjje6WlZyDKFu4aJEK2loKFxtW347VHGJejHEWK5Dyo8Pue7+jXzJ3ToBkwl0wpKCEwiTjTIQhzWsEpdSpJm7GTvZaosnwZo4GpdvZvbXtfDo/vjud7+/P79+fn56e3r1/9+7x/dPj+/vnp8ePj/ef7h8+3T88b48P/XzXt2Zbw8nQDU0L4dluCCXqDlAN++ui/vM//9u5/9/2gJru8V8Y4AQQCWiNuzlRVcY5ch4jjjmPY+zXuY/juh/Xy9yv1+vl2C/7cTmO6zjexjzGvEaUUH3mGqYW91ETyg381roPysAJlO81cn57/f5//+//+s//+7++/vOfc+wGy6HC9pHTaLF6foua3wUFbucYRCK0ox7dOr61JLZWErgVkFlnECA0x8yi9AnJ7RYP5QCg4vArDs94U5qArAx3wegVRde8iX+23a9BrU5/4Yafrot5WdGs/JZRb9uisXMVGdPXr5BM0NJYX4w8b3dUtk53A241OSIQSgip8hZWQmMVX5gsRQScVGpARbkpkcqDLPl/7cubLagOWL1GhoqqrGuxEnwUsIybct/pko4MV9ZMlk5Zo8Fq6AOBJAusUlNmVaEW83hjZFnxYRVNGjMzrHSba2sIkbI6+61N5M1rpRL0QFm1Iiprtamy8DOFBnjNx+swACn6MgeQbA6iinVkQIZmRTit9IOcVYRZjAFFF/8FJU0madbKv+eQIWYkUpbLEgz3oqxrJ+nsb297jv04rpzDrbRnhMiMUsnXFXPTNBPpUpQ+pxTVAJaM32vUIpRENVLA3SPp9VrU4Q1o8Xz0yssmKq2qfiqqdRlACcS6Y4UZaKUw1j5W2gm3iv5e+5eUkcUOS2rerAErhVA1QNV5HVqulGotJkxpooOtpHtjjjnGzH3OAYYp0StTjbAS96HcjlYRgQDNAkkzTVWGIQIUPLk2yFqpq40yzATjfduewWZIUy2cWG2kIJhTUTMIrKFZOcjNoYrT25rM0G2EvNnKic3N2ul8/3D/+O7d++enx/fvn97d3T29f/f8/v7++fH+3cP58fHu6d3p/d35+e7+sd3ft/uOc7PN4Lw9qTd8/we4X9vNJpglym/232D9//e8/+OPvP3HSj1E5qzxH5mIWZU5yiPmMZUZx5zX47hex3ibx9t+Pcb1erxdxrgc423f38Y85rhGjjGP0jVi9TWquNM1193OaGnl3NQTDkRKdIuYL5fX3//5z3/8/e9fv/6RY7q4pIUhraa5KHPFOuAzV/IbC5NWZpj7bWutGm1U9a2BmWqrysxmZHNXKjK6+wq7TJmxuWeoBBduLJV76f8L+mmtVTJuPbcBy4hGawi0rQkSwxIxgQlgtT//i+qWS0EprRYJVtQngNVp5a3mJtQIJYkNmJox68nfOgm0U2+dgFmKYyYZ0yyLPUDxD1h3bm34iSyiQZLo9V0Io5z9TMu+NTMzMzhWb03ZiMr4VvJbgzc3KlkI4SrnK+YQAFEocpLpQDMNIIYkU4ghFJ5fbnytxve1bedNmVtFtUYl3ayimJFpaaAsCLcQQBeTmFBQUTO7l0zRUlYo3RJ+lUIRYvHYJODWrI0Jo3tvrF7mKYGZ0yJSOSNB8zUVAyWgFwNGY0RqWdfrfePaxFVZOm5wsoG1vVHmhJZzy7sAjT2Og3N6S2+grWHVfKWFRmTWAhk1wMDkqprvrGQhLbVRRjGeAlHLo5XFi0CVUa7qZQKq07/YkMzMCr+rPHSVM8vd4ZbLuCdN0QugM+8lH6ZZK+FxllWmAlyR9PLrCXmLoM2ckRW5XbzqGokaUfI59zkDrZqeIucccx/HiLmzFX3FEuVUzOPtbVqKNECJvOVd1w637q2VGb1C4k2SBoneeWZ7oL8De8xhQbKi7c1IOZRGbyBdDRvUEgYyvVSItesThI2R1rZufPfp+eHxw/t3nz48f3h6fPf89PRwvnt/9+7x7uHp4fHp/vx4Pj2c7+62fn8+nbftjM1wbtiIjei3Q7utt3dVadqPkb80nfoXSc9//+P/JeisgH7dwB/BNLNM3boJeyqsXzPiiHkcMY+4Hse+H2+vx3Gd87of1/3yNva3fb+M3Oc8Yh5zjFQkIlmBOaqfeM2NNTfV+80FIGCJt01zyTTjOK4vX//x9R//++XrP3W9dpDmR6T7evFQWbkG+/8V9y5LjmRLkpia2XEHIjKzqm5Pc4bCDRccEW4oQnJFoQhF+P+fwQXJ7r73VlU+IgD3Yw8u1ByBeFR1T8+C6Oq8CISHA3A/x56qaozrGrNeDYYrlJB5J6YSXoIq68h6abffnNiqXHjzVJBpakx1zdAkNEgKhqhqMVBi05EbTCgWAAAlAkWlYJgMsx4pqaRZaUHLBqWJObmAcHWWN6jK38Qh4Y4w9JQQkTE6SfKZtJA2Dr+ZJaI2oIJB8hhzm6phNSnuWGAsmh0MFpJhLsvqqAAo1k7+VCITMrSqUoDEUK0sFSkT5jLViqUiLe/fKakQLZsO2KSovYr0NDEIhHiNoNpJShGpqbTB1GWsCnSZIiszFAC3KdgUcYFqRV93Ve39UBWhkiCTWIqC0GA6W+RX0y8VISPEpjIVgQ1bRpaMkxYGiRZJdrMXCuGoEClTAnCTgm5akCQwKbtpkcwHAdGSyO6L6BAW/pgzygDWkMgqRY7lLOTSuFeF8XYKYT/U5NIMIdQkK6EU1Ul0uakAyChTaQm8jJydYAMCEyHlkX+eOCqijZzQYSAFmSVLKu2P7mtGlo7RnOrqNjt4cm42oKqU244G/gAe0dQCkEpIpLecZUYCQp0lnpkQdajYEDXabJs+C0Buft0yA+Ji1VQeIc67UIQQsPbBlkr33oACoi9QVBVihnZiBIUgYQRg58nqweRxyKPLMHkIPIieIkPSIiAqZdCEqlmOGpIoWUoHsipSy+X0cB7Lenp8eHj88vPPP/+Hv/zyDz/9/Msv//Dl4ctfvnz5tK4/Pzx+Op1+evj8MM6fTw/nZR22nJYHVTWs2mPWW4q5IALavBcTLsChzt9PenK43pn7e6N/39S9hf7oYnMlAEVoepLGVZ4e6dtMj9yn73O/XicntV8u+36d1+dtf/bY922f87Jdryke6ekePkUlCMAEF0oXQRKNYSBDjjA9fo4ohgiSGdu2f//6269//adf//bPMXclgZLFNJEG05oAKmZSLdo8lAEp7USKafRkmNLReMaMMLXMUnZslGgD8ttIA2JHFHKEti2h3w0D9DQcIV4HZaoE+9CLSa0iLqqIIVRBHFLk71CsDMroGGi5CnKP2T2SEjhI+bK18ZgcPhCqpkCUiXhCpCSTkAjCpQUi3Tooj4YLiXfd6ajIQIriAmglvdt6UoGACXB66RDOjI3KmjkU5XlLrBi2EqWbxU3FX1XHIRXd82EFysSUrlFKFxXN7qyIVIkpjHBN6RsdSVgAy+8SrYUjzUKWqpRKHSZSamMsC4DMCnepqHJUIRODGncMdDne/jAInQLxgkkWZFEDPKpExJS2IsKTEheeitIQwRAkfAOVKlgn19JhxWm3nD5klDXSKmS4iJTCUK3FZyk6RDQRrIIsughSwtN996gqo/9RxhzaisIoKk1mFIZ01EC1Z6X88pFGEixG7TARUTURKhEJQksyoucPaNMdpXv3YrZklnJ63ejyEccJqyrFBgXV8M9gwJVVIjqIsGNNn1VPMTRylW5h9zogRGxbiwprPikwMyllmIeKmVm+SwnHgFCABKI2rJSjAwQkolclRagSsqiUFodPHcJUTdsTsDVCIfPspnNqqaVY2iKr1qPpl4RqPSUWpK2LZWapTM8QJYwWpWILlhqPY6xjrOvp4aefH3/++fOXn758+fLl888//fyXL49/+ennT+fzTw+Pj6fz57E+6vJpPT2M83l5XO1h6CJQxUmwAKqwA8mjhylXgPNQD+OjR3E/bzDNfv3e3B8Qlpvpvym1AegwvwC4ZiSKRTCkh19nePq+z33zfY9936/X7XrZ9+e57/v+PPdt3553v3rs7jN9zjmzZwBlw4aQ0h4WqtIbra0oo7gDwUOGLSBm0Cqf+9PTt99+/frXv86n7xryMMxZO6k0syioWkRQ2N9GV1+IhMeh3yCiqlCViFJlv1NMxQQyRkWB4m5oJbPVNCOY9QokMlhNIpiNKmZNFxaIKU0Uog62ENmniGyuyKDu81gNiLimQWWIsMlbFJ4S0TrkQsB3yQwQpLRDxsgsGcY2MnXiSmA9z5YBbtFkZVHpvrwqg7ZYWqgrBBwdJZTNaIAeVCAt6A8jAUAJloYOCA0KiqAR7nGUSKkku7vFij27EBRxjNRO+ehd6hakiVmJVHKoLj0QlA7SRNXkVvhpmAAznGz9vS6pKJs/EIECNmSsUKJmvcqFXYWGanMRamNVs7r8xXJ31B3AUZHqXkBiGA5/FrMqUzwRlMokZpHAlZSEioQHoBUFauUfolQiDMPvgESH6KoI+bsVGZ6pqrCqJEiSzbKO6pEKQXIKI8m6UVKcaQwbCsXQQY1NOt5upUEgbJFBoE2hJl86oQVRK6Sirz8bwiIio4OmCmXwpj2zdhQZEOzpCNInnXY6pw+KQMU7+o/MCAdKB4jIB8XoUFyKUpl8a/oeG4pSHUBP3iL3hTSanF6MVFeDjuJUG54HUpFSMK8GVBSNZ8+at5br5JkwjFpLDeGjc9AwqaXkUdZ/WD/9Rx3/CYHwZ+gpqgwSlZBVF9PPJ8i66OPI5XQ6nz8vj7+cPn95+PnnX/7y+R/+w5d/+Mvnn3769OXL46fPD4+fz+vn0+m0LGe1s42R+rg+DozR09VXgQiMHN1iWwLH9OcDLaJQiu0IQDSn3hl6wcu/949bkfn+38rj30A4UMg90zMja/rcPeee4ft1833zfdv3y9yv87pdr5c5r/t28dh9XqfvHtv0vZDhfhCmEqztCBrfkYccK8vMKsJvdbghORpRBGFnxJzbj99+/fb3v8/LM3aO9xEzVJUOE4GJElUBIf2nsx1kUii+20giyxiRtRinNaKOiyuCNGSkiUjXjSGooVZqNxHmhDKAE8CKwjTKFZdZQwBUEkJdhaIqQTOUhshIB0r9moKUyV6HjEUDKNExdKyDpDXfpm8znWQdARCFYZZSug6yuRCHDrOWpEiXOjuQZpE3Z5Qc2Qpn9UhVcEh2JoSpD4QmDmwMEKemhJaB8/REVTLIrc2hKsHSKYBULUrMt5RFaWUduGNavKzIMrGW8G89GQKGHEd1X8XYzkSZdQUhA1EJI7QPGZGSpX2bi+rkVaSfyqDwUURluEemSYoky7rgX0lrAyYLj1EcpSMl1C0hXUa0epinCUthFOGRyfCwgJQMZeeZhYVsfNIwg0pVGzQQTyzVEzZNsqCpJipqBCdDCuUZ6WTY5siZxcSHDquiikPBMqC062gKCEqhqjJKhwCgNAIlKEI4Jk6RyOz526iCGuFeFL8SoTynEcIMSkRkyaCitgIlg+VvacfAChvjB0QVTC0rMkWA2EIN3QitikSFS2TXmru7xO4yilORFGJMNEX7jwU6UJbphSzPojpkNQWS7oMmJjLZbGKBhxu8CiiNmUCjuKkkit49qsYOAbILwTARdakcKYudv4zP//Hh038Y9rA97yE1f3iBtcHlp//mHx5++vnTL395PH96PP/y6fT45dPj5384P/50+vLTwy/nn34+f/nl/OXz8nheTudxOun5NMaiqiDR3hTngVPBFANNzW1DLgmoUuAaqXVE8TjqNW8avHeh/avnejv+CPnzdgqqEAYylTP7KtKn5+7pPq+b+4x9I2Nr7vu+X/ftee5X9327XKdv83oJePgWFZ4zKzLIJi8Z2s4YqWJs/HYDqZN2Nn2RVV0caouWIiVq4XtUXJ6fv//+69Nvv8ZlrstwF4iIETvOOE5A9R+l8G4S/Sm2AGViqRUx1ayQA1rpNowgBQgkpCNDUnVu5Xt0BUNUwmOYZJHMzXfWQpnpwfmDIEuldDAAbzka1hZUynMAa1Tm5qqQWQrV0sjQxcayiOm6nEWkMqZWlCtKhiZHiopmptnSFTQBeoxA8zpRxzQYkKiJKok8CkoN4ap0kASFo23d1S7RFAYZqgIdKkYiG1oLjFaIzMc4ZIu596GS8GDjFBklCWhKsVze64291RLKnhRTAmIEiWyiEoUA5CkfCyJbkRlA1RHkkWxNuHaiJ6HoGCZAEhsUztRSB/2LFkSObMOPnJPNo0wQJpWJDAgKA2ohqlCpIDhXJEIiFR0/C1CeVJo0iBjjEeNYyjgghiUFiZYxHS2sSg54Ef0kEpUercivMJWqdM8oVWlmVEYFcRnMu2jeRDheV2QRkYMuZ8hgK5ySQEMSyGbVtcAGm70M1ZiMW1E2BU1MoYoHs2SWN8U5Y572pCoyhOVMGQVnESgz5Kj0Fg6OiKBdbkPPhLO/SikURQOnxu80VKXbD0iiv2T6zN01OIUAIlbK7oMIqZtZoJ45O0m0pcxSo5hqEY8kgogkpwFQMcm00EoVNXVIlJadlk+/PP7yj+vj4+ks67gsD08ynx5+Pi2n/3T+Ij///B/+8R//01++/OXnX/7y+dOnL59//vz46cunx8fP6/lxnNfzoz6e9XHVdcWyYDEMbaVlYl+lYIalIIJRbfdfDDwvyC2uf/XkdQ/35gPYyz3SvTbyebP7B8QnPZGawZKn5kyfWbunR7m7T99336++X33Ofbv6dtmj1fnnftm2a/geEe77nDNyj5wlPZOsm0CqHdWx8oJuRN70pFhaRAVtVrX6QwGlRv27WRXP35++/vr3b19/n/suWWI2jEX/I86nGWIOnRBtiVVOLqogjMNYNxVwKJwRrsOl0/PP2+4L0+yu33O4X7VFMhz4gkJaUzrsICMRAkJcEU/MZUm9FVOMTz/9d7/FV8zLdrn6lpo+JMfA47qYlJRLzUqdHumEvhSGpnaEJqIyChz7RQJL3eo9nAMo0qxXBJXWoFqpKHjmLD/UG1nhJVrOTFm7h0mRDkll1IOB2fKf2ZxnSa1CEEViUtASRGklZEhFCsmBailZIjIEJWNoKZahjPSRhG8iJaVgRTpppvCCZmY2oBAQNjdbNouKHJCjp5cdJPd8GhC06pyNTbh2crsx/qAGXEHKySkWEwt2ab3h/129Y/s6nN0CiVTPzAplCk5ib+XW1GmibBSCweQGYpVRMaRSWrDsiNlKKPAlDEF5PxVJdKFKsnctpsAEKE5ZakhPljV6zUrBTCxlgBqvVJMl8+9mg8HZ9k1vodc9bIeoKCMHOkVUNBVQCPPS1ggJQLXr6KhD5QYiIGI1JBMRrPihUXGAQHowHWQUsosccnRAUuSYh6HpBetOCTUq3J11BHjWXhmsM0rSfxcxS02+KS8tlEOqhHhQNL5X0dw6Ym1ZMhNyd0VKpQZsjDEWXdaxPq6PP3/56b/56fyPPy8//eXh9OmEYZ9q/e9X+28fHu2nzw9fPv3yy+PPvzz+/Onx88PD+fH8sC6n01jHGEPFsA6cDacWtYBJV/NfzLncKTG0tdYXi193xv1NpC8c2vn6xcO861HeyTx+kTcHkMjUYtTv/G/6zmkrO/9ld3fOZ/d9btfJss/cpl/cd/d97jPmDAJ5MyL3FBJ6WhShihy8g1TIedovUiikynLzyrHoiAdIRtimMr3mtj1/+/3r3//2/PVr+qa6tKhzwUxUBdqj1kQFxOFJz3g0UUjpsPSOQVUETXwFUMMI1McNFSBc3kl8vBZ1HtU8uyySpFrxyHaqBIRrcaqrQFU4qlAFKB+wKYKsVcb4H/7H/+2f6xn1Yz5/255+5L7ndr1enrbct+c9c46ZqhaZRpK5aUn14Ah+8eR3juQ8DUVyUQ9jwMMApxJDVQq6p0RZIsST6g2dFCo0VVRsYDBR5hXUJO1NqAYMVPGCtUWEQIST2ABode7BOKyiEJXkBWWUiLQqPeMAUSPyFKEkGZSKUG+Y9cFODaLY1GQtSg4Ul7A/yViC9fejFKEKQynR+wgz0go4b6arFVlo3ftsrCRpjpFZwrkNSGHVYegqunLKTlVWbc4LS6YoSrrvjpQhmeRroyr1wIRlAqPpyRhDRtdmS7WhwyXK2eKE6DjDH1VFhouVqVROeis7YQyB1BSxRJd/aIorlgGRUmXXsATVHH0Wzzy7ZOisrhe496wzJ3bYO0ULkKHTl9MQmdbZnWR15ezos3EPQbt54VBmEWSPC5R/WLeOm2iistQatQJUappSuBulGipERkOQgihTZGXN4MwcoXYhtAEapeVemSJlLP6kREGjmt7CtF6thP1AzaxURKjpgqFjWZex2FgeHj89ns+fPn95fPz8088///zlp8+nL788PP7y8Omnkz2eZD3Z6Twezp8eTufT+HRezo/LOpZ1LGNgaP/H28A5ukbOMShPiI5SCbgkhKSN+KEqfyvgvBj3G5inD+w9/KbUg1c/9lTXunV3CzlRifJMTzhi97nvGXvM3ffdr1v45nP37brvV5/XuV+iYu6Xbb96bO675wz3cGrVI4Okm6i2GbyJrZrVYLNKEQXhgl1b789OzeZ2aE37L4hEhUjF3J6+ff3221+fvv2WOZnshYtCUUG70eUOVeXVa+gw20gkn7uq4dDhBrNuRYHqJ1R3qBQgkvGsiRTHfPLoqpX5fgu6cJWC9QPqozDdba19gYqQM63oq0IA3vjf/9f/4/+aEfPr9fL96fp1u169nq5P356v3/enb8/fvz3//sMvm8kwVZamxgKKBa8QTWAvOBoZH1qmzIFKtXOsgJhCNQsmOgyEK8pJRYNgd7ZPFQJTHSJSkYgQNZO1wXsZjfERdkkVoGKVKoHPFZld6ytQzVyrPKtQltqky7ShYuTychaXCDkITrhftb4Dry/h71U42gttS9yr+zz1Aq3MtioHrq8OmBDUiui/csCksUXscUhKaXF6sCmxnzGDTBppSSHoIjqMzc+oyD1MFdSF0iyVLKhpVlXSDVYpHGVD2QeoQrmwlRwQUZQSEJU38qvp0OVkImUpkVlTFhEjXibH0KoUTZM0E1tIkoVB3AtNnylU9WgYhQwFkdPa7rrByVmTeJgJGO03qzhqi7V9YdEQArW2PoX0LIhpEaqMW1YBoAPqqkiIaFZCwUs2iqAV7abTC9oytQRlJRS+zeK8ndRkp6rEhrTadBC917TMiIxClikwqMXUUCVUoUwcImVEH5fYUAj56Yy+x4LlAFxYSsmy2sN4PD8+rJ8eHx9+Oj88PDz89OmnL58+ffn80+efPv/8+fNPnz/9dHr45XT6tJ7Pw07rWJd1jHXY49ChWFUpqUajL0eY388rOagBLYpEm3AU+eUuhL9FoG/s+IvJv3vcxBzk9Y+HIE0LCWfnRkgHxzHWREbGdEI557b79Tr96n6Jufs+9+cf4XPOjZF++DZj933znJ779OnhoBZU9+47bmCtuKGNUpADTs4QIRtt1ziGrvRz86DzUboK9oRVcvr18uPp62/f/v7X649v0gg0As5DTZA9i0EaAEMxFABSzQktQMbQmK7LyMzD7ktlu2Ma7EIaqozZIVRY3unGNa8y4WECAVKO1a8Cqr+DVGxud7ZvisxplhvZdYrxf/7P/9M//4ivlx9Plx/ft+fv++XJn77vl+e4XLbLtl2vzz++//6rb09PP37sl23bnq95FUyf8xRYUtVzuApzf7WEpElUaahICjVfpbPrJTFUamh4pMBQkU5ePyorHCa5oFKjRIeKKifFFFCZwolahySrLTZssFqXlO9haZf4dfSevpHaWFEwgy5scElGUoaPRWdJCqd2EKmmzN5YRgELVBlJjFTloRTUWUx2HRoFgHiOlJRaxiCbu1ITaGXuAwQpAxXRU+BVBAjPRjuZdgoJVFMumE2ilAlFiFFtIcgShEBMOXm4kKoQY5YpBUJNBRAdElVqSp3f1EoVW01Boh2pItU6/shMZ28nI83A4JV1r6oqZdeK4vipWgWnm4V00YyabJPJElrIkKE7WV3UlZUhpRgnraoINvlNimiQYtu8Z7OZsG2eBVJweYPzyKWA4nBIdvfEuvILOvMsoKi6pgS6ZRemmQlVVxRVtDI8yojKYX8liwtSWtqIdfLiJThAIzXYD+fmhBhU0oGyZT1rjlUexrLI0OVhXR6Wh8eHx/P5y+OnLw+PPz08/vzp8efHL395/Pmnz1++fPr86Xx+PJ8+nU7ndT3ZOmwZFOo5xHY4Lwm9TvWwxqw76lEhYPLe2+Heyr839HKcog+4R+7finV3buM+UajD/vPMLPWUw/csErj2RD+ZsW0Rm1+3fd98Xvbr88xrzH2/br5ffPeYW1R4uM+Lp0d4pAexhMGB6lIg5YergMRGqUyoKFp4TEitqjayYP4t3Rpil5WImWCvkgSuqJhzPj/9+Pbr77//y3b9IZkqMtQCrdeiqo2Yb2Uc6QVRBWB0cRhiIhAxEmZEtLW7ZKF918pI2hlG8Z4Eaw8By/cClIgpItIGqFzDggOje4JT6ibwlEEbZWADmGg7RZbqGP/Lf/7PP664/rhs+/7s16++/zq//749/epPT/v+I/dt356uT9t++bFffzxtP67PP56/XbZvlx/f5Ola3/br7091CfOSSAlJlUp4+ZnQ0KpUQA1a8KAqfkBblkELtaRkakqFJGBIVSgkiJNrJerGeRZMq0SyqAljqip5aErPqsgxSGGrglTCVGSIDXRHpWA6GDlmBEhMRYumqgBmJejpJCnUAeceb+RgUj1ftQ69YToext2s7WRKUWYroSMPGYMSJgnFRu6NfCJLzwLVsaBiUCVJhKNvVLSVwgVUjbZSl/7gWlDDQjkjouGVHSOaaYADIEwjKSAtMNiipmJGR1hACbO80ihhHyW8xlir9hKowtRoExtN17M1LYUNTuGgKCkEahkGVbUizgeC8koRDSmWu1jiE8CgJrKYKFvHVAmDmtYUHTAbCDBYyMiopLQ4C2rRfT1kpi5K4nibHYGaaFkVO3WKqmz2RqZnVprAlLPmWy9ZpMCNqgIrfmIweEqW8gXRgHF0KZb+5pjuQtvRYk4UqNOxnmVZbFltrOvp4WF9eByPj+unh4fz6XE9fV4fP5/Oj6cvjw+fH04/nx9/On/65fz40/nL5/Xh0/pwPp+WsZ7G2fR8M/rSFp88zPuY/L5FSwj1YbnvOrV5F+zfP27W/OYY6v6sd+d+UyC6HckAu+gAWBucyEB4xh4+Izf3uefM3H3u29wv+34N3/f9edsuc3/23OZ23eeec+ac7jOrgpO5JD12NMKKkPSimiMzUIioWpYQkypH4UoUUnVEBFzGNJkU0Dr4VQVkmahIqVpmqMi+bc/ffv/21396/vp3v04tVU6K5pwoJqIdLLbQE809iaiq3LsCQkyHmJEhKQBBlWWqVWUsHFdffl1HEcaS3TOg/6os09vEnEJRnlmZAKOz8QIATpYl+34xgusIPhNgnPH5vIr98hBeWXNWXur6w5+/xfVH7t99/tiv331/2q/f9vl9279v2/P18rxdvl+e/XKdT9frtx+Xb0/794s/X/anH3O/zm3D3IrdG69UswV6siy5StkwqgFSLkyqFKkailllJaGmtMUVZQKhin3UQthfS6YhQmpHMvaKEi+dAocUdJUeJmIqJsuialDtDUmRy0AFKYUzpbrtX90+JA+ukuI1aja0489WcRQqQyA5BYCLhQ1jRUtSFie8lyijDLSOsJhQ65MMj16OxxjPUBWOcAEAjq6RhqNXViOEOXZeJSeljsExAyhpYTz+YCqcGl2cUgOBHf2CIQYZxjJQ199SqUA1EyB7l4BogVBqMgIJHRKB8ioDBgvgIqXkDgt4jcUg6cF4rBIIgWpmSxxSvgpSpSY9pFfKDgiQdrtXxCg8jso0IimEkJKqgyJ7Mz2M8cCiA5r6YoKqISRtSUREY1FDS1TtYAhCQtSUVdI6vgrPVooIoJCzVBRpWQTjFtVkIEqEm6qVDLNFz6ehy3I6LaeH88PD+fHz6fywrg8Pp/Pj+eHL+vB5ffi8nh8ezg+f1vOn5eHTeDydPp3Pn9bz5+X8uJwfx3q282qn1VbVE7Aq1h6YhdFGLV96tveFe3kx7i8G+xb7vxx8Z8/r7sX7E37sJI7A/+UVHA2s7usCCd9vk1gyPcI9fPrO/66+bTHnfr3O/eq+7fvFfXPf9v155hZBQdVZkcF5mOnVWye6+HjUam5FHQDdyxI9MDRdNTmgZg03E5Fm+BaAMuUoCahqtqJZsqdfMfen71///tcfX3+dz8+DJGtoFSuKpRS8zVo4dE17BZNdAGTjCbimDOliL31GIetWjBp+jf6kyAm0c1MbmpWm1rUqaHf9QN6KypF3EDRZh+o9I5PKLBuVyZHg3DuiOs54CEA0bVUgT8AX5D9SUAYxM3fsW+7Pvj3l/D79x9wv+/bk2/O2bXO7XK/P+/Xp6bJdLtfny/X5ebs8bZfv+/Mzrpe6bE+XuFymb8/bvH6/bB6+u0DE4Y+nVYeWlkrIdBNaTjbzKGVWKonI3FK8RJI6MakMxzidjtqbiCjJRACZLolFM1JPoooylUFhTonIzIyoqMjoao6IVBYMuljXDTihh/DHOhCjKNY40ORNKUtdJOuFL4aAQTgXOAXC3DMLhdQqGJ1IFoFZSesPZGb2wJRO1omMrUyONWaW2gBEhVAabqhJlJBDjNaph8NOJpVkTjHsQdBNUI0YbMewLRLd4xCOdZakek+tLeGQoJB8VDpEJQrCIQuHVFk15KUqRYbBqlCRsMWAFCihaDE52B0ZkiU6oKzLqKoBIlBoY+8SOggJK1Rb7xCYZrhUdjv+8KAESRRFfQqQaoypSlUoY4RCRRVnNiTYBpUEoJlOgCl1KLrWmkpkHxUPxSHQogyLCiJVpVTGYssyRBez87J8enj49HD+dH54PD88nh9Oj4+Pp8eHh4f14byeTqd1WR6X0+dlfRwPX5bzp/V8Pq3raTmdltOqY6xnW1cZqy6LLgOrYlWcAU6eI4tFJe9GomtTbXEUHm92+a4Q/9p235l7+dMDbocdcf3L63fWP5HHKKZEOlAognkiYyL39M3Td9/38Om+he++X+d+ndfr7pvPfb9uc7+474Hd59V9mzEzMwjrcWd4HoTtV7LSq8MOuXw5SF7VVP2O5vgKAwNhWsw+zVEiRHB+BtpVsqR56zIq4B7Xpx+Xb78+ffvb9etXRZqOBES1ihJZKE55FAHqGBJQLOOgimDyLulkisqyDKDF3NWsCmPYofoAJq/DTEQiu5mnBsGoKoVmeyo5sEtshx3qb6ToKefisSEKmFUV1EqKk3dZFx6JAc0kQJF8bqQoVqCQD5qFrJG5esEL2OETsWVcfe45r3N/8v1p2573bd+36+V5Xp6vz0/X6/O87tfr9vw8r9s+ny/X/fnH9Xrdr5fLZbter5cnj3nZ54/NERmey2kJG/y2lROlNT3VkoLmUYbKjBqagpkcj+iRYkrCZzNL3EtPVmyDjdLTKMCLsvdAIPZEpkgJleWMCzo7hYSKakEKBiut4mxBz8xgjNGY9HrpPjbtPbt71OKjBenQv4BCJbU10bgpVoqlTyIiMOJ70MTrIo2rhKa/h2NCiP6c7Fx04ZFJXwKlgGoDy0sI6yzmGV2KJ2xqZI6iw6mk8kY5IFWe4lyqOUcYEso52IhkeUmkisL0dQRgWRDVclIiIKhKyaCaf38b6YRdIVAVW4Za89MhXcAFsV5eMlJU1LIEmRwTWcg0FOlwR08MIloEIR9lfVS764z2KQKUo7wkoHEosBXcUwwHlYNqC7z7dDmlsAqkWLkYuQA2dOgKWR7OdjqdHj49Pn56PH/6dP7y+fGnT4+fv5w/PZ4fT4+nh9P5/HBazuv5rJ9O+rjYaejnsa42Hux80tOqp6GLEKQJUSyGYRjAOGR2RkGRPeig7dZh3wVNuKv7V242mqvodbcWb833qxfvn9z/eNckYOuUcE5OygKicRIZiImYkTtlmd3dc/e5X8N3n7vv18g55+b7dcY2t23bL+lz3+fctsSM2DxnpGd4JHdDx7vN0SI9B2AZrwCgpY95zxo6DyEkTW7SZY3VPBQNj3RRgIZpsEmWoaKB6Limcl4vP77++vuvf708fzeDqA6tqiWzDFrti0VeyL24wR1UjN/AtMlZOoyWw8ySmg4EOxPQQIUrNC6RvUyKRzGRyYSghim/URJPb4ZCceDWoQLQmoDs/muXmzKd+gJKVFLWeLnTigaE4ZY68UhIqqhzEZwapli5FpBxrkDs6TN9jz0owr1fd9/2fXef1/163ffr9br7/uP6/Lxtz5fr9bpt2/W6b0/b9u1pf75c5vfv+3x62rfL9eL7dtmeM3PfrlEVVemI8mWIjgoRWazhHXBzBr3IlbGR6FBZDIuUJSzFIKLpKJHaU1qoQGQQbihlhSzFAOB62I9EkjqHRA9QTvfAobLKuq+RQdWGtkSyCRskFyHaDlcXaandRGUowQHNyEIW10oVsnos6xDVOAIC1UE+VFRElnNmGUBEW0kVvKgLVQAitVBiSLFiayIyA7VAREQNYpWSoiWqVeolyWvVH104E4m2VFApHiVlpQrOvRE1GV1gMjFBbZnuLHMVsTYlkTUItouKvSqQcpS2dNXVhJj4DH6LzGDyJwHtwfECldgzOTGxUqlyEQmm0WJVyCzS6RV68J1L1YRAzarcU5wVR4ix2a2GhtbJEBErMXCWG0TVTHRdz8NOasvQ08PpdB4PD+eH8/nhdD6fPn86fXo4P356eDh9Op0ez6fP66fHcfo81nWs57HyMZYxFjmvtqqezYaYYSy6CobpSpAsMVhoo487MP6Bwc+XF24Fery24zezrnfZwIePD//8/gyHrde75zfSVuuIt6YOpwRvmZEs8adHXPfwPX3f9913oneuc796zDm38H3GvvnVY/qcvu+5u9PoYyd7i5wbfijpLXTwa27l7coG5lJ6oYX/Osbvhg6Exr+kCeaEaTPFxoH9Z4uWDf3MaO2QyH27zufv1++/Xb5+zetlqEnVEAtgLA1DVrMi+9WUtYSqNOMyUxx6riZFJsyimgWqivTM3q7+c89187a/t0CAyFYgtxZpSVU7nAJAr4GiF+tIqxKFIxvq+zusp+BlkaiEAaBSW27huPfHMrrj9eVob0lZQmBAgWQl8lETmjWIAg8gHY3s9Xmd8zpj99gv+371/XK5XjmhYb8+b/uP6369zu356bJdnvft6fJ0vV6fn5627frj+fl5Xi++XffLnj7jakO28BL4zGVdxb2et3UZWekVYqKoZdGqyoHS4KT3TMksCVGHpBi0RldPRU30qK9EUUyDdQSJpJwcUAxBRxXbEsUSG8FUHCXJUKS5WtxLYp2cdIoJKn8y4iioWg9h5NJhkJOQEoMg0W6dTSQCmVKyAtl3V6UZ434oaBVPW0gOSWPWG4SnELVsilWk+cRFelVmRZmnZplUdlOAoH5NJq7JpKHEpFTNbIzF1AyRcAHKKxPo6XCca1iNgwWQVbObqLooZGhhQDUUHD/RAsghkerFVFy2FNVo0gxyOgX0ISVRyKxj17fsLSBJPXeIqC3sswtEPACXcuUgrRLTYVXjtK5mJ9GxjEXUxnLS01hOw4aty3Iay3k9n5fzOk4P68Pn8+Pj+vBwejg/PJzO5/XhvDycTufltNrDupyW5XGsjzoeWcaxdeiqNkxVTE2NiEx2bglC0xc9tZ6RwvJO77x8+4SEyhZg4CTBvE1jemvK//yV9wfUqycN32QXNytBEbDIDIqQIB2YhajcM3b33d333CnVcPW4xtz3fXfffd8i5vQtwufcUtx92+eeVT73SqoZBmWyWqeRn7HyMGAsljdqpyWPIaLaC726+8SSbov7F1gdyYb9dKePl1OsdXuVrPsqVe5AdtXDt+u8/Pjx298uP37LeRklOgRB5jC0CzgtO8C6AEv8IqPZWF297Ha0HnETURpomc4bP7JIlEIDkQoQZIpgqDY4sdA7qpLxK41OE4huJLKqqiJ7ACWiCeqRQKSQRTARRHLctJyOm96L44PlorfXOy6QV2CDV387kKoAsk5Zp1SQjuWee8Qe6XP6jN0pmj4ZJ1yefbvs27bvz9frvu9P18uz+5Nfn7fnp7n9mM+bPz9v2/Pz9XqZW4S7z3369B2BeRUJCVeJmj0rM7N0s6gsF4uibDon64FD2CUZLklWBEQ0ifPOTrGoox3sLaNhAgdOmDBfXstS9NSe7jXRfTtKkJTgFPJ9yA4scoSFguNJHS0ShLWiyitFxRKAmBG2FBU0s5wXw8iCRMSIFChMOG+EQkcELEqqpNGUm9ihZppewQ/fSYmIRqVwsh4dj4D1qNAKwCC6RJbokDFkmAxIlaZQV02ovZ1QFWPVqaQiUqRC4UBKBdRMR6Mr2YDNzJzFKybFPheGaEZWlEIiCgUTip+DGM1lDG20k1SCEp+SiIksmAg43TlKbTnJgMry8KCw5Xy2dawP5zFOn8+PJz2dlofT6bSM83o+LetYT7aexmldzus4L6fH9fSwnB6W88Pp/LA8npZ1HSdbFxvDxhgDY4xVdbGx6LLKsugwqiWrFoY24ldxxPXyMvlWXzYQU3/tJuqxxV5vw5snOHhYR3Dxatvev3Jv32+pw5vDaPGlVQxYqWuDQPZQumZmBcv61GVD7p57Euvhc/q2+bxGXn1uc9+mX9jFdWJ40j2nT3di/rG7e2WFB7M6UioyQ1XaeLNxxg/LbSK4STjw/xqBdziMqhurtIp5JsUetINq2k5pns/xJsL9K5TvNbWokAzkvPz49uO3v12//w7PdV0KKDFQ7pcxYG9hMPHQtg3MzROiYuhAXsCB8s0YYu7CEm51J5I/3yw45xSpIjON6uYZHKvbhza/oZQdZuDWFef1QBLlTNmdQpUotBpuDsiQ2+3PFyN+lwT0/7xPFd8sKdz9WwDYzDtOQH0daA7NMQooPGT3YBCZk75h5r7n9Dn38OnbNucW+8X33beLb89zu2zbj317fr48PW/P05+269Pl+nTdvu+Xb9cf+/XHdn3Gdg2Z2+bIzP0KVKT7PktRLraqqAhL+xEDVpUEdBczRNotzikOcFKSiIJjHDiYQ61MMyDHECFtgG1UFMXXjOpezpwVtXT7nnLHrHh3fB+FlOz+AJMPlsLLBqe0KgKJOsYOd+aa1QsALXnWfk2ATmoiiFmkRo2Y5iG7ioSOggtvjoESFJjRwqsNmfeqLE0oJDbIArUhVPIpFaoNl0rM2Iuyr+m36C1FgNSIMrUK1dKq0hwSEOEsBlQKAV1sBlcVyIeLFBk+gyw7U0RmkU6dpbZwwlcJqkTXrl5loYZk5FgXGyZiKHsget7OD6fHdZxPjw9jXR4eHx+W80+nx7Otj+vp/HA+LaeHh9O6LKfTYGP2vKznZT0ty2LLaSyLMqg3kyFGXMEh6AtVDHCQzgv3CnL3Ly0427bv99QbeKXcJeVt9G/bk8isbOt48wqvtufrF+83rLx9MW+nvf2bCXgWUK6ZGXtkZu4V4b7v6ZUeOd23a8zd5z63q/vu8+px9dj2efW5BcJ3z8iY08N7RnjuPr3EW1ZlhqpVJbQbmF12JKGnjq/R9ehu+nT36bgobQjlSFvCbQwcHbbOA5oLj/YE/E1Eoek4LBuaGrLK3eNyefr64/uvz0/f5r4bCixrihBnLZQ4qDbickvEKjspERLuGDkWc5E6+nYGJNFx6Crk8V1wfH/WeChGREwi7XZ1fiBZkZ0ONtjjCMy1VeU7KQF6pHVmtunnLAGMm62vWwzyGhl2Wyj3/3745H613V48nMRL4APcludBUFFAc0Ge+ldZrQfsCQK23Wsydtjd59y3eX2OeZnzeW5P+/X73J727bJdLtfr9fn5+jS/X7bLtl+v17ldr3O77M8ztn17mvOyz2v484wIn+VImU3spORdgMgYlvLda6zahBbVvrfG6URoEcEq5Wp1qGgFpTIkSwwFwIx4LGmJIKSaiCCpQ5tsOfA7981WUx0KpEC0gg41ZiHrlvcREMEaEYGhID9C6kggObYMPchME6YJV6AyZYLCb1ZKsE+mGCwiBakESs/SpsGJpsIlgWWYBlu2VqWZkBm1qVSunAm6cXp9EV1FB4qEAEYELfmglXCK0CF39jsJUVLI0U+J26CdgmiE6BgQCtWaqgGmouO02jLGGGNZxxjrOJ3Pp9NpWex0Hg+fTg+fzo8P68PDejotp9P5vJyXT+fTp3H+NNYHO52X05li+WMdOsY6dNgwU1G1MURVh+o4GLaCF/vea1vfVO1b3PtlF9zq8gDytb2+bZP7TXez/nkfZt29Qx4brT7sCryYzbsX71456vn924ZQsniYyEBOpKPcY7pPT5/uHvvm8xqbR/jcd9829833fc4tYve5zdw9rp7TfYtIf4HxRJJtU1Ecqi68qS1lIgBV3qU1GYtRjh5WtCv1XTspWkMTAzQJyOdap8ALrS26JMuaRxWbV/y+dbS3WkmNFODG0btvz9+ffvvr0+9/nZdnuMtYuhURkppEi4E1li42MQAChLqJjSlvJGZJ9wEpilweTJ+P28b2RUXQoZD+mdVznlB18zY48oTsL3xUyACSl3gfW7pSDlAwi/4ih8KOsBA9DpSe8o7cJD4+tO/vbf39Cv7wgHu3cedaeiHfBUFaB4vxOOFRkNIszUKC81SQgAdmpEfOmT5znz6vOa/u2+7bvu3X/ce+P+/zmZ2G/fo0L9u+XeZ2ndfr3Hy/bte5PZFjcvG57b7Ned3n1X2L2OaclRUZ2HcICpEREIQ4MoYOYQlaGsMJSEEyi8NvNGsMInFKhKg1iRIILWePDMkoSZVAREEksjJiUcJQCUbTLPVIKDcQd0n12/FeHlwC1CFEwMMglcojq0TNUrtWSDwsoCnKyeyYJWXpZJxnVRA+UUnSLdNxFpfUYRKSaYIRnlKQyQHCyW5vOsoRLlDhTAlbWftCFEZUzV1cGuFqa0JRKmJDhoSomInpaUBlPKrZOJ3Py2q2mthYT+fVltN6Oi/rYovpuox1WW2sYyxyWpZlrA+n83ldz+t6Yu1+PT+s59OynpfTaazrOsaqZxtnG6uMIavpqhiqQ2CAHYRhcukbfEnMFtDE2jbHL/tFb1b+w11QHz3/o531EsIfJf7321D+9Mf7N+onBw/sVtkvoByEJxc58Kz4e6Zn7p4ZsU/f95i7R4vvu2++bx5z32fsu88t3CNmxHTfI/eoGeEznSLMVdHDlltdsxUcS6Qq+PFEJSK6dkkYL3roCVNJVatgW1iI81Glzr6gQEJpHREyUWdydwFYKCJAIymK1o5AKkNgHZgXIkJin5enH7//9v33v29P3wSuo1OwKgiihVzaqAI9IYEmtwSSGc01YBcsmWYA2iAdgHwdYeMX0BsnDPyzYwAqemejNerQuT1TEZb/RZBVytG3ODwDWmALRQg0jX/rE9ogVeKgk7wABg6r+7J07rAEb2o+b5bs+2X9/pj7R70+4ZtN8hLnkNBz/LkB3XxWlLZycx15Q6AqGXn4JBDB972uW5Bv7jPmFj73mHtcruHum8+r71vMa+ybb1vMPbZt9xlzzn2yyRDT5z4zZuzhFCDJ2r0y5j59f670Pdxti0anuQzNrIzQclSoDUSGSIlNnyeVQkgGIgWS08nzRqgNDFXFKEECGfAZahYBFUsPGMsdCCr6CDM9DRSIkCkpaGFklC3WogdjqUxRhmKKlHLDsoZzXMwUGXskZCQQc67QQmWpmnmFQqMmZpSa5OIS20zZZW5xEikslmYSagOSsFSMOi2EzGlPF66xDh16GjpUxnLSZVlUbT2JjtNpXU/LspwHlrMtY1l0tbHYelrWZX04LafVbDEb4zxO6xgPdjrbsupp0XVRW9ehQ8zEzJaxLGar2dCx6rrqWHUdOlSN/1JdRHuylfFf0LgDNO51W4HHDnlZvcdqzLuNc1uffxIk3Zb9v2Wb3KT230Txf5J5f/BGtwAfx+A7Vl4Z4xfXJgU4E5U5IzLKqbtz5e6hxZ9+jdjdrz73Oa8zPWb6vsf0rEyfETNyes6SivAoTw6dzoBohguxtwR3SoqM6qCFlCsUIcoUQaFUpzZmuVpJts1qV220pT1xEP9VDJXjiJ4aH5ccK9owY1U9Yu7OQbpIIl0G3rft+9e/f//9r89fv5W7qlaA4pqUAZPjdh/VfhAOLQf/oF9R3AkECsBafN5o2wrJuDFLATrFaBFc6Rxf0d1vMhBZ6qlMqIGANwAKrR5g2iCMoqsFyWB1sCPaWoDih3qoQb6smGPNv5Rx8hWe7E98wJvFV3db4sNFX+/+ffOrvrj68uKbnXkTGT8+TBrQ42dH1wOP/zzhiZlwz8zIGTGdGje5l++Ze+Ve7pm7+54+PSLDfXpGeHhGZOwZ0+fuEdPTfc7c3ecMd9/2fbpvc9vgkTMzKiIY1YdX9Rwtj0zfBa7hue0xZ1XN61WHQJMwwBQUZHf3zOk+56YqEQEgMrVnJmtQWMfA6md4NAuhUClZKiOhiuFAyTIM0kz5gwkvWKDzyhNxRSxr5dCC6lDRMpWFcndmqNOiY11xXlKQJadxXkTOprBSqeVksj7YWIYuqy1jnNZlpWbNWNQWW4aOYavKMsYYq42xqJhyXOZYx1htWegATG2MMUyXYWMsw1aTMdR0LKqLjEV0YBiGYpiqKTXzVGBKlHOLYKmSrPxSi+f+l3uLf6w7fbX8DhvasRvuerBvjvnXkuO39Zk/CZheFb3/rfn3y98CjIC7lF+HuScEMVGBCmRoOqfd91C09HD3nFvEDCd05+r71f3q7r7v0zevLYOzd7cAwfqZe0YkOBYn3cshFRVZQV5eNYYIld4QzKoSVHg3wzqLbSAmXUCXNnqiN4uefWHkaP8KOkxunfaWWtD+1kTj03/Q6aghSalp+GdmqQ3iizNTq+Z2efrx9fvXv3///VffrqYtA3OQBkpuRZXDI3VFvtAKVEdeToZLZilaizlJl6CiPRAchVop2hPiju+K43hUHhNcChmcqc2rRIB69cXKkltx7Fhp3VSmAnZVyzFyUTNLIA/gtoLf5ox3lfpKfRUP3ZcgX6/125/LUfN5s1X0j/fM2+3xGgN3Y8FwQ96/y+GJW8fs/jw339CpIbIU0MLCJZYHar93yh3UjWWoSg5lqcwqr3BQiSpQzlx5RtKye8SePpHTPThgtLdYzAyPIL8lPJAuPsOZHQcQ0BKDVmpiluyR7u6RER7wRGSrWmtCkJIuSCIJjkIqjkvQuXZzUchRgQ1Vk4SmkXyNoYkIjYSLqmmNoSK2yHLK8aA2TGvRUogWZTsWEaP8KUpgq9qiZgodUqo2LG0RW5ayVcYyFlM1VRuqampdSjdRVTMIpIMxay6wGGxAhwwVNdUe6AwVHJTKBrf2f0flXXBAInrPd1T3KoR/vdb0jUW+X2m3H0VxHxK9Pce7pVsfbKJXB9z/+HEU9cdv8cHjjux7GP0W4ckjzG8cJ1kgXNnOqCcyPfdwd/c9c/rcw5uxtc+r+x7BH6dHuO+eOwetRMb0ILQ6nULsAXhkJAJSPYYleTMYkUBo3l64yiV6qwb17lTVpNiqgMRGPcoejPTb9nUPuMNhXu9mCwDHJB/pOkwrwNit4s4CE9CMp4zUMaQyYr8+ffvx21+ffv/bvP6QRPK9ssQaXtNY0qPj2mUZWNXRuT46GqWH1gTD9mzABoMvUy2Q28xf9UA7DtmDUNca/LQ85iVWrqqe86iH7ylAqK+iqsd8JoiAhdbMIl0JnSp1sjKAfEER5EsvuwOQ22TnuzD8zUKVu9V8n5++T4r/PAi6j5Je7ZbbW3/U12Jlud699btHb+XX+zDffJhmDb5CMPG9mj2YSBy6tsTKkxcDPTRKkM42D11ssrKakenI7mhnIrM8o8j7KMUx9LiAwvSaGdPd06vCWwC0Z54U57wTV9VfgCVMotCOvj/pTAIjJV0HRA0ipVrC8TGpkZiK1GFDldSnIWoYC1SVg3YKEl0IB6jroODUrMFx6W2i1aBDRYco5Zf60h+7seVp22TLnaUWvJh1lbsf39pJBiKdGuoLnCbfLrNX9+8O4NAZLSE0R3R/v8DuH29s9J8E+3WXULxf8G8WLe4Wf70+Eh/9yG/ei+zmZhr7rUd1vZ/00gwgkJ6IpmgVNTRzEocZ0zP2mIRqsoU73a9OH5BOTxAxMyMyfFJ907NiRkYEoNIqsZLpJZGI4rwpRrNQImH4j4AKaAxMiiNGiPviHJG6dTKBljNUQQPkDQTb3Eg1PUylrxM9hzSLtZcWbo1foAoqygoMlwHRltnSvoGK7fLj6fe/ff31X7bLD0SoWR3veYuvQbDSQdpvPI5QRIEFGT36tK0qRQZndzWyROx2k2mlVTm4QyKSY2QyOFQFVLYVlcN8aWUIu8zAoerTFwoqtx6h9sxEAarLSilAqhKmVFBU1HBPOfxpl7Tu7GwL+eqhBAsA9Nr6ZjN8GM5/uMpvq/99avxxUvzutK9OdR+gHQe88Ub3j9f7+QbkADrBv9mAW/mJpp+AHOaQbcBuR8pRSEjFjdYhjaquVsphKtfZRh3CFbdv0dq09EFHwpFZyXN6tgLRy0cjdBidKt/gzxwegCNaYgqrbVJNBEr9yqZIgnVKAicOPmrX1grU1O3bchcbtGcX8lYAQOXFoL+Pvl8ufvXBf/R4/atOwmiyXxv025O7sP1lvbFomS82/d64S75U2P/oUcfSuq23fL2o3viG+6V+58r7V3i3Ed5EOe/yg0Qeqe3N4TFBBQDNQ2EfyCQn68DpZyanFJRXzkj3dGefNmNP39ynz+lzz5i+T58+Y4ucEe5zEgER5ZmRRMpVZFGQzbOCim/ReqhVwbanFzjQLLvbIGKUsFcOjiWmIEWpydKDV+gqCFGUhtPIUfIBfQKfkevL66e9gLWltzu5sL4vwu5rAsKBWV2xQQKIZNAd7J4NNc8sd5/Xy9PX77/98/XH3zOmmSFFRKlcq5wlR28AEZjSOgMMvQoVlSKaEDKwonsV1V6ngCo1zciDxaAmlOi3hvOUCB1HdZWq3RiOgRQlJRqtOUNuWzXSXITfl3/WPqJK6kYYLDlG3PSgC2Bcv+2qUJPmWFI1UwDVklTC2/Lox+qLdXi/auWj52+SgDc58pvHR4HPx483sdKbZOLD88jrv7oZ4L6HBwjqOOCuE9gtkFdjNO5OqO++VM9nGDewHTBeUqjs3735Rq/AfHQheVgN6hvfguVDzer2HfSt55a39rebTvzl/ad9l4S9Ma6vP+QHZ8ZdS+a4jHrACt/9+cuRb1wtXgfjdyd/MdbZb3H7aLcrxquX6ND+9rk+WEsfvvj+c+Zr9/COcNur5T63uL1+PHm/I/7Y893eusE6ecs4HWQJVh6rh7NIg8oCbOFGemZUTs+M9Mp0UmsiZsQevsd0n3vkNWKfPue2ZbrvM9zd9xD3nqsVMechj53hzsUWDELIGhc5yEqEIQjZ4AAOUPIxrAcJ0u1Fboj1XnsvpXltiGLXSdBm/LiCXeur1vG/RfTJ6ER6CguPBLT7BwJQmo29VDloYrcPWsAhnBD7/uPb719//efL0+++X61ESNwhwOLWAKhmTHZTV4XCJ9keh7KDqFA0Dz6JZMq68UAPx5YcO4HDxwiZqEQNUcuiOOiCfpGXg36o4wCJAqSEYtHW3zWpTpsqbGzIoVoqR4+CaFsRAcbztx9qqsswVdExhrDIbOysWe8qEYVCMnt7KG4VFbn7l7f2tlPeZ9Bvfrw9tOPlf32HvNswbx9vrP+Hbun9G90Ute6PwevvdW+h3p9EXt5a7w+40+M9tve7ryCHQemp0tqdEjlKT3fm40jE+v+PE/ZzvD//+0e9e3LcxBdPcjspXt9lfHQZX351845H+PwmyXt/d+r1JX315J0Xq7vnt8Pq7rUbbObm4+WjRfLnDznWw31cL+8uFz3qPWz6Pvz/cCXfv377kwJuzoQxfau/o0WV290fMJ5MVGbMSM+KCs+cM9lcmpynGBmec/rcI/aYW6vu+J6xe2ye7vseGb7vhXDfUyo8IgLI8KDyTmVrgICJBvH0rXovVRUVKmRDRmWpcRJ0ALdcUrtKoyoFKY7kho0RmTTQ6DRWD+CjqLT2H2273uY5NYlJMxuSTDgcFRAPJsDLHaPmfU/uPvp8DYCQhEh6qqJ87pcfT7/+0/Nvf92fn43VmwRa5LfBTGqjW8BVvDNVRb1/UhBZchLSGXCQu1iJr6ym74LN57uiUqnKUUYr0zOPuGGcqjMNYTsHcreEGOXL0d9FVSbnlFEK9CXKqlvdv30rsUnjb//y60lXO5NXM2wZKiziDluU3TlwfAuHKykAyl8kmLv0hnmxHfkHq1/ePbl/3G/2Dzfeh77kZqDf2KY/cjyFt5sZ+CCU03cpRb0rAvyRr5KPfuyvcPTSbw3tUkimHHO32wq35KFKK1nmfdz9On7Xu3fU+uNPdf+4CQl89LE/Lt28/14fvvL+2t48x/vLUndL5Y3pf+OG+3HXrHpT/MHdu9wugr4+24dJD959gNuL7ZKP+t6b73v/XF+/WO8+//2nqiOOv32pQ9upjT8KFWyFamamQ3DA87OqkJ4ZkXMGxc33PeYezpIO4/o9Y6bPmIT0bI3y8T1iz/KI8JgRkZ5ZkTVLwMSBfPii9nILEoje2FnVzUgVCwQqRAdQkKNNWvmC5mTXk/YUXUrvwDvJqz+2I9UaWLKWxsNocy2U0CBeOMbPHY23/1WwwkJVheaS4WjVJtri1XGP2P+1iEClRM3r84+vf3/67V+25++a3hycKhErtI5HwbInyrCq072MbkvTPAtUjA3wTl/acgOtFQSWfjhaDuhRhFXIDGI26ijhk9XQlT7OAiu5TUGoAgWITDgTSSBApKlGpHYCFoAqNI5ORSfogoJmlkLGr3/721kXOy3Lsg5dFMsYOlazZdGFuDsRNR2mK2eNimrC1EwbgwioKSTZVgRAjn7Dh17tGa07W/ahjf4gpnsXcta7I+X1b99s+Hsn8cajvIo072o7bwpE948PYvh/w0OOmKRPe/detP51wG0Ft1LbLaJ8+4bvDc396//6h/kv/AL3X/mD5OnuX9xd4T+Pu++9qdzfi9cnefXQ10/efYt7d1KvT/s+6cEfv/Lyq/vOwR8flu8OyJeXX96i0JWlm+jCgYp/YUaHA8ic7RDSs0cfIsNbJTPnTHef10x3d59bzi3mlUj8cG/f4Ht6ROwR02PP8MyYsfMcHjMzqdESObNAt1PNq4JHY/PbbLXdTjUO3Yw2xNSiIqCrMZY9dAtC9A7/WhUQtR7b3egZyQiYUp7ldpFKSmjeXu6OALgVcIgjOJZZCQeK9jAAi8oW67+18HBYwGINNyuBSAW2y/P1++9P3/729O333K8mrdJyo1u+mI0EtBUoREpE2XbR4mhDoJBScpAf6SpVKDAjQEU0Oatr8dTw4dcSrSoO3GByiUL0kDK8FMeq10nrjYHHEOgJwj6WMSJxw0Q1C00qI+WQjCuDqlTU+Ns//T9nG+N0HmMMXcdYFGLrWNZ1rKuNoapjXW2YDhsr1WtVxyBb0lZVUbMuXIxFRSGSTBfuUXS8PtDX4Q+0KXEfRfHvN977A+7N+p8kAfL6sPvnf2Ia7k/yRx/v/nHve/4oBfngr7KPuTdo7xOg92/9bzT3/77HGx95/0nyQ+v87m///BW8+3Z/8o5//nhzx98kHDc/Wq/f4s0B98/v7/UfrYrX6+elL3E7nm17qLKuwwJEHkZfSjMyozIKQHplVUaijkauZwK5R1bk9Mwu0Jd7hhOnH7nPefGYGXvse6RnzsxMj9y8IgqV5XNeE1mI6rOESEV6RGXFYamK4X+CMoCSUlJHkF5GtR02btWIBCEjtlUvRYzWiZiDwsG9EuOLVYkkWB/tFajEXw1iP3q1BYDjlKrlkIkGSHnhAKBFoADQZAuqu7KlonnDrQtxmfy3EFUajUnNDJ/b5enb73/7/u03zgg1ToNg3UqbckumlvYEcanikKAkBj9umlwvIaf29AJKa1Sr95hKRNf2CtQIsO5MBOkCWhx12dhOigmL9jDBI5vg8LJMztuFQKvTLBWpkjHUM3r+Bx1MARV9A3m1VVJkfPvt/73IYstQM9XFdNhQU13Wk44xxmms67oOHasOG3ZSGTaG2VAbqrBlEZVhQ0TFMMZQo9i7ihITSI+hOgDFMUKto+yuENJLvez8P0GJfLz5/62R7/HkT6LC96Ho/evy+iT/xkhcPjr41Ye5ddn/4E3zj9/rv/LxYUR/M233sX++PuZPzvPeY/1R2vTmRvzJ8w8fb65J3n3yD99IPnol393TN27gzdsdj6zOaJm9cXpCR5dNzSCXpHExN+OO4pynrPDMSKByRmaGewFNAElPL5+zKtJJsg24h5Ope3XfvS4RW+ScPqm3UxWZUYGaAdaRJLxmaRZxx9TqIz659WoqMjoipwkXVElWmg0VSQGorwx5CbuZEaBNt7QeMkq0Ef+Qnk4hgpRovQSgSnV03lD5gugp1ofqQNa/XHKOQm1VRBwYy37vOrBDqIMK0zwYhpqqQEYEAZUikh5qqoqs3LfL8/fff/z+2/Xpm0SJjCqWORJ0aklasjIlUJW+rwJ0d0QbkFQalRH80lWVOoxKPnokM1kJkWw5rwTEM1DJOSEp3ekl61NMGB5XVWT0MuvrJNruJkU1K5dhFXFbocWpWlzj7BBEHdCjTorcQwTj27e/jhqycLKJqQ4ztaE2FlNb1rONdTktY1lFbYzV9GS6LOvKKczjdFKRMYaq6TAzs2EqZsNMleNtbSH7unSoqlBYg1MNldMz2Ug4YBbaYHx+eJV/Z9Hlzx73O/z+3zcH3IeNeG1u3oSQb558+I5/dIzoy3u9P54P/YPX/+sfH3yeuyf3vkretVX/KDt5f85/4/37MCp/f8D7m/VHTz58vEm23niFujvmvlR/76tIuQKOZm2hnNXaBAE0iXTyPEHObVUPted/fL1/IErTvSoz3CNizqwMdwRrNVuWu3s6FbA8Y4bvUVvWTLjPPSJoyjMTWVKVngJJILClZpYf/E/2L7NEqBPYDQfOVc+ihRIBxzRAgIb7SrOcslSpcm2mmtFR/804q2hRp1KUcWvbnu6G8gNEh+7CoK9ZwDcAKEljlFAW5TAsHNwudMeAepdUjxDt/sVRlwfod2+iOuruw7SAOffw/en779+//u36/A1zCrREIpKkcirl0MH3fheJl0Y8r4aYWSWikBliphydgZJCRkhfM7aNq0AfIpW0gJKRcoPnFZOsqoYbQVVKceg/NhpJqtS0eP5jkkxViookB6AuhaxKTplnCF5UVuWlOaZpI3JcLl81lW5Ch5kNzrVRNVOWfU7DFhuLiI6xjnFSsWU9mQ0dNsYqKjbGWBaoDrVhQ8SWdagNFVWzsSxmHIyhamaLCXMEExvWjRTjiChIg05VlKXqBF7pYR0thH+PV/jQwurxY929Unevvzkedwbo3x2Pfxha9osfdWj//3r8kam9ucb3X799w939eW/K/1VPeX+F3x//X3rN/+hb/Pntyzf/e2B0jifI9CZesXgeReZ3ZiHS56Q6S0X5nJVV6enp7ukJdAGfPsBnkG6VlREzfFYh0ivctxmYCY+InF4ZvpET7kn4J6aHU/8pPasCKojKPKYowmEVGR1dE9SCUjU0Tr2EtuawRgTKVFf+qUcgUJObJFmVmgEQqNF+QDNbaeGQdBZVamGyYp6AQq2O+EtUTYxY0sNm383qYu0IPZ9IoQ1laZ8rDVJvsQiqqhEapF3pBiqoRivMxARSJukzfLs8fX3++i+Xr3/z7VmrmgylUiosb/VwgfRU61i6AIjo4GwC8u+Im1LTm7YdDtgmpMDPn2iub2cDXcqyTim6c1JNB6IaqAKpymY5JdCNfggC0i8LEK3mQRDv303x9rtVaWYeqcLIW4CSKDOpLFtsbNt3hAnnzU4qqaotS+ucA7asNhZVU7Ohi9mqgK3rWBYdYyxDbbBtYGModNjgGWyY6VDVsawUdhk2+ETN1ExVbSVNlbK7TVCiVDJfZDdUR38YkSMNVBxkJdwjZO5AknmnPv1q57+p/LyxAvXRkfLut2/KQbff3qcUb/72zTHvf1vHZ3+fanxoB//Vx399veiPLC8/z3s/9WK49dVVkncX9o8ue7575c1telMfw90neZ/S3f72zWV/eV9W5BWNkb439wDhWIVWD04n97LCKUFT6VmJ8EBmRlZlTK/MCD/s+xG2u6c7NaXCvTKShzERmIEqd6+KiIiYELhPZPrcqzI1MoLNYd+8qsRyxjXDS8LDe/ZJz/EUOXDuGcF6fWXqMIAirOQ1UF1cU6KqqCrQaEUm3gRQdiP3Jl7MERiNuExkT3Mm3oYAyuOWs3hDEAtvk4hkZtts8BxHEaiASoFW3eBCt/tUgEgdPVh+jNsSkSMVE/T3Qr8LP1cBIiZIHfA5Ub5dvn377Z+evv11Xn+ITzVtcw/pBrAKkJIl0Aqqqb9wF1QtM40DNjh7HSkt3KadKwAFYqMo2UDsP7H8DMb5o7ZcqApHuLBMjkoxEZHKUmhZVYHTgEGBjaEZCbABXyYSUUgyvwTGazcyYzGOvAbfVqFQ4WiZcb1sWnRGBwYrSzdOsVdTqJiOYWrCEYBjQWGMRZchJsu6qKpCbF1MO+S3Hpxtw0zUFlvUxhiLjcWEiu2LjaFqtq42VMVEdCwLrbwuKhAdJsIakdgwM4MJSnrwkzZbrpfIi5op0whe/XwHyr4HTb4zjgnoW8Dlm4ccx92evPvt63D+4zN8/NuXF4/FjIPRKvj3lMH+dev/762u/fm3w0df883zP3Atr67/u1Pl7cePHLAe5U3e5ZfbRGQttPEZhxt4qep0jb4F6dthVCE8qATVglBRIhmeOYMyAtXiHsHiTUXE3DPcw8N395nu4ZMvJCv76eFemUmxwHSqxlbegD6RGYXKiKoK36sypSorpws0nW3H9LwWvNVwU0DpyypVRXHWKICC3Oo91Ics0WOUTIGlGIDGF8BBE2qEJbu/cqu/d6hehxnmCYGDWsRqjBwgGsGhJw4BJ6mYKqGiHZCy9E+9Mjm2dH+UZqN36HwT4jwow9U4y7otiVZorxukFFUU0/WqNBnImtvz89e/Pn39f/cff0dMrdIUzt3N7Ko6sgGlwnkc9YIlZTWP7lZVAllVpGHlMXeY7GPKc9JRmWkFcalVVQzcM3uGarOmW6sz6CCPFkoR/mQmdrhSCERy2O1aFbqITqpyFbp2JCqmTCKOwoaIAnGSQo4f1zgvlZVaS+VU1ZguVUNNVcxEC7YMiKod/9KNDIPKupgoUac2xpKFYSaCZQxRU1MVGWZjLGrLWFbVsSxnHetY12VZdVnNbCwnMxvLasMKMlYTURsDAhsGiI1hw9QMBR2cHGU6TEXFVA4dAjkg9sAx0OqtHXmxL+8jUNGPDfpd2Jivn/wX2M57y/g+MK9X7/IBAerfcv4/yjn+5HGTV7rZ039fqvFf9KiPL+nHr9zb6+O1dxSAl4e+fn5HHFA0kRVoFCCjrJu9B7KcMPeqzgyCU0wqMzKdyrAZc/d9VhapU7foPuYsd/eZc3Of4Xv4xjzgONIzo6oy6AAyMyICVREByO11Kvn6dDN4RA/MSiCo/aJREC2f10Ssp+EZqsbYvyrJ2UmBkC91kIDQZyHl6gDLt7EmcrwoKdVVBlZP0Ib/sLS03CIcNdF3SXFgP/VuNEJX6+s2j0tvYaa8rLgjEwBIm2IrAiKC1uEUgGXhA+xPTyNHpUfrtnJFGAUD2vhIMU5VMVuyspA/nn79/uv/ffn+O8rn7qsudZj7llYI4OBYhUAMKASDh0ai0udWeQ9xT/ZGBOhmeOezR9tM8lbal0OqIlHVF56fl6tPhXWuInWgADUgIUZhVVQkPTGvDAW3SQnmGExeM1GgQoVCFaWQ6VFqYgcIaVn/PykaIUVdmVwdAAAAAElFTkSuQmCC", + "text/plain": [ + "" + ] + }, + "execution_count": 1, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "from IPython.display import Image\n", + "Image(filename='ouput.png')" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.9.18" + }, + "orphan": true + }, + "nbformat": 4, + "nbformat_minor": 4 +} From 3240167967e5b05f97ac51fc82f527eed6ee4afe Mon Sep 17 00:00:00 2001 From: "Yang, Bo" Date: Thu, 30 May 2024 16:11:59 -0700 Subject: [PATCH 56/65] [Core] Handle `TypeError` when `RayTaskError.cause` is a `BaseExceptionGroup` (#45523) Signed-off-by: Yang, Bo --- python/ray/exceptions.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index daec24914f02..e35eee260d95 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -187,6 +187,7 @@ def as_instanceof_cause(self): try: dual_cls = self.make_dual_exception_type() + return dual_cls(self.cause) except TypeError as e: logger.warning( f"User exception type {type(self.cause)} in RayTaskError can't" @@ -196,8 +197,6 @@ def as_instanceof_cause(self): ) return self - return dual_cls(self.cause) - def __str__(self): """Format a RayTaskError as a string.""" lines = self.traceback_str.strip().split("\n") From b73e03768b0fca4bf566e8155e1fb767a84f9a7b Mon Sep 17 00:00:00 2001 From: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> Date: Thu, 30 May 2024 22:56:59 -0700 Subject: [PATCH 57/65] [Core] Fix worker column off-by-one in dashboard (#45648) Signed-off-by: Rui Qiao --- dashboard/client/src/pages/node/NodeRow.tsx | 1 + 1 file changed, 1 insertion(+) diff --git a/dashboard/client/src/pages/node/NodeRow.tsx b/dashboard/client/src/pages/node/NodeRow.tsx index bab7238a954f..3c5b89adb36d 100644 --- a/dashboard/client/src/pages/node/NodeRow.tsx +++ b/dashboard/client/src/pages/node/NodeRow.tsx @@ -279,6 +279,7 @@ export const WorkerRow = ({ node, worker }: WorkerRowProps) => { + N/A {coreWorker && ( From d6f97cc754a09f82a36da4795fd8280abd7d18b4 Mon Sep 17 00:00:00 2001 From: simonsays1980 Date: Fri, 31 May 2024 10:36:44 +0200 Subject: [PATCH 58/65] [RLlib] Fix bug: Target nets are not synched with main nets in SAC. (#45614) --- rllib/algorithms/sac/sac_rl_module.py | 7 ------- .../algorithms/sac/torch/sac_torch_rl_module.py | 16 ++++++++++++++++ 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/rllib/algorithms/sac/sac_rl_module.py b/rllib/algorithms/sac/sac_rl_module.py index 23527c957e59..3acd1e30a4c0 100644 --- a/rllib/algorithms/sac/sac_rl_module.py +++ b/rllib/algorithms/sac/sac_rl_module.py @@ -102,13 +102,6 @@ def setup(self): self.qf_twin = catalog.build_qf_head(framework=self.framework) self.qf_target_twin = catalog.build_qf_head(framework=self.framework) - # We do not want to train the target network. - self.qf_target_encoder.trainable = False - self.qf_target.trainable = False - if self.twin_q: - self.qf_target_twin_encoder.trainable = False - self.qf_target_twin.trainable = False - # Get the action distribution class. self.action_dist_cls = catalog.get_action_dist_cls(framework=self.framework) diff --git a/rllib/algorithms/sac/torch/sac_torch_rl_module.py b/rllib/algorithms/sac/torch/sac_torch_rl_module.py index a83303b8b948..9b30e5bbaf89 100644 --- a/rllib/algorithms/sac/torch/sac_torch_rl_module.py +++ b/rllib/algorithms/sac/torch/sac_torch_rl_module.py @@ -33,6 +33,22 @@ def setup(self): # parameter names to be removed or renamed when syncing from the state dict # when synching. if not self.inference_only: + # We do not want to train the target networks. Instead, we sync them + # with the actual (trained) ones. + self.qf_target_encoder.requires_grad_(False) + self.qf_target_encoder.load_state_dict(self.qf_encoder.state_dict()) + self.qf_target.requires_grad_(False) + self.qf_target.load_state_dict(self.qf.state_dict()) + + # If necessary, also synchronize the twin networks. + if self.twin_q: + self.qf_target_twin_encoder.requires_grad_(False) + self.qf_target_twin_encoder.load_state_dict( + self.qf_twin_encoder.state_dict() + ) + self.qf_target_twin.requires_grad_(False) + self.qf_target_twin.load_state_dict(self.qf_twin.state_dict()) + # Set the expected and unexpected keys for the inference-only module. self._set_inference_only_state_dict_keys() From f9ab43954e0f6ee5ab40db26e3d7bcf47a92a0a6 Mon Sep 17 00:00:00 2001 From: Jialing He Date: Sat, 1 Jun 2024 00:18:37 +0800 Subject: [PATCH 59/65] [Core] Fix the GIL deadlock issue caused by `list_named_actors`. (#45582) Signed-off-by: hejialing.hjl --- python/ray/_raylet.pyx | 5 +++-- python/ray/tests/test_list_actors_4.py | 31 ++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 8437edaa0122..407db534b013 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -4446,8 +4446,9 @@ cdef class CoreWorker: cdef: pair[c_vector[pair[c_string, c_string]], CRayStatus] result_pair - result_pair = CCoreWorkerProcess.GetCoreWorker().ListNamedActors( - all_namespaces) + with nogil: + result_pair = CCoreWorkerProcess.GetCoreWorker().ListNamedActors( + all_namespaces) check_status(result_pair.second) return [ (namespace.decode("utf-8"), diff --git a/python/ray/tests/test_list_actors_4.py b/python/ray/tests/test_list_actors_4.py index 0f302e428084..4bbbc819338d 100644 --- a/python/ray/tests/test_list_actors_4.py +++ b/python/ray/tests/test_list_actors_4.py @@ -1,5 +1,7 @@ +import asyncio import pytest import sys +import time import ray from ray._private.test_utils import run_string_as_driver @@ -52,6 +54,35 @@ class A: assert not ray.util.list_named_actors(all_namespaces=True) +@pytest.mark.asyncio +async def test_list_named_actors_with_normal_task(shutdown_only): + # The following parameters are all designed to increase the + # probability of reproducing the situation where + # `list_named_actors` gets hang. + # https://github.com/ray-project/ray/issues/45581 for more details. + TEST_RANGE = 10 + NORMAL_TASK_PER_ITEM = 100 + LIST_NAMED_ACTORS_PER_ITEM = 10 + for _ in range(TEST_RANGE): + time.sleep(1) + + @ray.remote + def test(): + return True + + res = [] + for i in range(NORMAL_TASK_PER_ITEM): + res.append(test.remote()) + + async def run(): + for i in range(LIST_NAMED_ACTORS_PER_ITEM): + await asyncio.sleep(0) + ray.util.list_named_actors(True) + + res.append(run()) + await asyncio.gather(*res) + + if __name__ == "__main__": import os From a95ec7fad786a3dfac995ccb126398f3f046e8df Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Fri, 31 May 2024 18:24:27 +0200 Subject: [PATCH 60/65] [RLlib] DreamerV3 on tf: Fix bug w/ `reduce_fn` still passed into `LearnerGroup.update_from_batch()`. (#45419) --- rllib/algorithms/dqn/dqn.py | 19 +- rllib/algorithms/dreamerv3/dreamerv3.py | 222 +++++++++------ .../algorithms/dreamerv3/dreamerv3_learner.py | 56 +--- .../dreamerv3/dreamerv3_rl_module.py | 86 ++++-- .../dreamerv3/tests/test_dreamerv3.py | 126 ++++----- .../dreamerv3/tf/dreamerv3_tf_learner.py | 33 ++- .../dreamerv3/tf/dreamerv3_tf_rl_module.py | 40 +-- .../dreamerv3/tf/models/actor_network.py | 8 +- .../tf/models/components/cnn_atari.py | 2 +- .../models/components/continue_predictor.py | 4 +- .../tf/models/components/reward_predictor.py | 4 +- .../tf/models/components/sequence_model.py | 5 +- .../tf/models/components/vector_decoder.py | 4 +- .../dreamerv3/tf/models/critic_network.py | 4 +- .../dreamerv3/tf/models/dreamer_model.py | 1 + rllib/algorithms/dreamerv3/utils/__init__.py | 2 +- .../algorithms/dreamerv3/utils/env_runner.py | 266 ++++++++++++------ rllib/algorithms/dreamerv3/utils/summaries.py | 253 ++++++++++------- rllib/algorithms/impala/impala.py | 7 + rllib/algorithms/ppo/ppo.py | 9 +- .../ppo/tests/test_ppo_with_env_runner.py | 8 +- .../ppo/tests/test_ppo_with_rl_module.py | 8 +- rllib/core/learner/learner.py | 121 ++++---- rllib/core/learner/learner_group.py | 38 ++- rllib/core/learner/torch/torch_learner.py | 2 +- rllib/core/models/torch/heads.py | 2 +- rllib/core/models/torch/primitives.py | 25 +- rllib/core/models/torch/utils.py | 23 +- rllib/core/rl_module/torch/torch_rl_module.py | 3 +- .../examples/catalogs/mobilenet_v2_encoder.py | 5 +- .../examples/evaluation/custom_evaluation.py | 1 + rllib/execution/rollout_ops.py | 9 +- rllib/tuned_examples/dreamerv3/atari_100k.py | 50 ++-- rllib/tuned_examples/dreamerv3/atari_200M.py | 53 ++-- .../dreamerv3/dm_control_suite_vision.py | 41 ++- rllib/utils/metrics/__init__.py | 1 + rllib/utils/metrics/metrics_logger.py | 30 +- rllib/utils/metrics/stats.py | 40 ++- 38 files changed, 951 insertions(+), 660 deletions(-) diff --git a/rllib/algorithms/dqn/dqn.py b/rllib/algorithms/dqn/dqn.py index 2674f99dbbd5..3bf51e0edd42 100644 --- a/rllib/algorithms/dqn/dqn.py +++ b/rllib/algorithms/dqn/dqn.py @@ -629,6 +629,11 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: env_runner_results, key=ENV_RUNNER_RESULTS ) + self.metrics.log_dict( + self.metrics.peek(ENV_RUNNER_RESULTS, NUM_AGENT_STEPS_SAMPLED, default={}), + key=NUM_AGENT_STEPS_SAMPLED_LIFETIME, + reduce="sum", + ) self.metrics.log_value( NUM_ENV_STEPS_SAMPLED_LIFETIME, self.metrics.peek(ENV_RUNNER_RESULTS, NUM_ENV_STEPS_SAMPLED, default=0), @@ -639,11 +644,6 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: self.metrics.peek(ENV_RUNNER_RESULTS, NUM_EPISODES, default=0), reduce="sum", ) - self.metrics.log_dict( - self.metrics.peek(ENV_RUNNER_RESULTS, NUM_AGENT_STEPS_SAMPLED, default={}), - key=NUM_AGENT_STEPS_SAMPLED_LIFETIME, - reduce="sum", - ) self.metrics.log_dict( self.metrics.peek(ENV_RUNNER_RESULTS, NUM_MODULE_STEPS_SAMPLED, default={}), key=NUM_MODULE_STEPS_SAMPLED_LIFETIME, @@ -680,6 +680,14 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: with self.metrics.log_time((TIMERS, LEARNER_UPDATE_TIMER)): learner_results = self.learner_group.update_from_episodes( episodes=episodes, + timesteps={ + NUM_ENV_STEPS_SAMPLED_LIFETIME: ( + self.metrics.peek(NUM_ENV_STEPS_SAMPLED_LIFETIME) + ), + NUM_AGENT_STEPS_SAMPLED_LIFETIME: ( + self.metrics.peek(NUM_AGENT_STEPS_SAMPLED_LIFETIME) + ), + }, ) # Isolate TD-errors from result dicts (we should not log these to # disk or WandB, they might be very large). @@ -730,6 +738,7 @@ def _training_step_new_api_stack(self, *, with_noise_reset) -> ResultDict: # Update the target networks, if necessary. with self.metrics.log_time((TIMERS, LEARNER_ADDITIONAL_UPDATE_TIMER)): modules_to_update = set(learner_results[0].keys()) - {ALL_MODULES} + # TODO (sven): Move to Learner._after_gradient_based_update(). additional_results = self.learner_group.additional_update( module_ids_to_update=modules_to_update, timestep=current_ts, diff --git a/rllib/algorithms/dreamerv3/dreamerv3.py b/rllib/algorithms/dreamerv3/dreamerv3.py index abb163f8b671..50bcce11ad90 100644 --- a/rllib/algorithms/dreamerv3/dreamerv3.py +++ b/rllib/algorithms/dreamerv3/dreamerv3.py @@ -10,11 +10,9 @@ import gc import logging -import tree # pip install dm_tree -from typing import Any, Dict, List, Optional, Union +from typing import Any, Dict, Optional, Union import gymnasium as gym -import numpy as np from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.algorithms.algorithm_config import AlgorithmConfig, NotProvided @@ -22,30 +20,36 @@ from ray.rllib.algorithms.dreamerv3.utils import do_symlog_obs from ray.rllib.algorithms.dreamerv3.utils.env_runner import DreamerV3EnvRunner from ray.rllib.algorithms.dreamerv3.utils.summaries import ( + report_dreamed_eval_trajectory_vs_samples, report_predicted_vs_sampled_obs, report_sampling_and_replay_buffer, ) from ray.rllib.core import DEFAULT_MODULE_ID from ray.rllib.core.columns import Columns from ray.rllib.core.rl_module.rl_module import SingleAgentRLModuleSpec +from ray.rllib.execution.rollout_ops import synchronous_parallel_sample from ray.rllib.policy.sample_batch import SampleBatch from ray.rllib.utils import deep_update from ray.rllib.utils.annotations import override, PublicAPI from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.numpy import one_hot from ray.rllib.utils.metrics import ( - ALL_MODULES, + ENV_RUNNER_RESULTS, GARBAGE_COLLECTION_TIMER, LEARN_ON_BATCH_TIMER, + LEARNER_RESULTS, NUM_AGENT_STEPS_SAMPLED, - NUM_AGENT_STEPS_TRAINED, + NUM_AGENT_STEPS_SAMPLED_LIFETIME, NUM_ENV_STEPS_SAMPLED, - NUM_ENV_STEPS_TRAINED, + NUM_ENV_STEPS_SAMPLED_LIFETIME, + NUM_ENV_STEPS_TRAINED_LIFETIME, + NUM_EPISODES, + NUM_EPISODES_LIFETIME, NUM_GRAD_UPDATES_LIFETIME, NUM_SYNCH_WORKER_WEIGHTS, - NUM_TRAINING_STEP_CALLS_SINCE_LAST_SYNCH_WORKER_WEIGHTS, SAMPLE_TIMER, SYNCH_WORKER_WEIGHTS_TIMER, + TIMERS, ) from ray.rllib.utils.replay_buffers.episode_replay_buffer import EpisodeReplayBuffer from ray.rllib.utils.typing import LearningRateOrSchedule, ResultDict @@ -145,10 +149,6 @@ def __init__(self, algo_class=None): self.env_runner_cls = DreamerV3EnvRunner self.num_env_runners = 0 self.rollout_fragment_length = 1 - # Since we are using a gymnasium-based EnvRunner, we can utilitze its - # vectorization capabilities w/o suffering performance losses (as we would - # with RLlib's `RemoteVectorEnv`). - self.remote_worker_envs = True # Dreamer only runs on the new API stack. self.enable_rl_module_and_learner = True self.enable_env_runner_and_connector_v2 = True @@ -506,8 +506,6 @@ def setup(self, config: AlgorithmConfig): @override(Algorithm) def training_step(self) -> ResultDict: - results = {} - env_runner = self.workers.local_worker() # Push enough samples into buffer initially before we start training. @@ -520,7 +518,7 @@ def training_step(self) -> ResultDict: # Have we sampled yet in this `training_step()` call? have_sampled = False - with self._timers[SAMPLE_TIMER]: + with self.metrics.log_time((TIMERS, SAMPLE_TIMER)): # Continue sampling from the actual environment (and add collected samples # to our replay buffer) as long as we: while ( @@ -535,45 +533,76 @@ def training_step(self) -> ResultDict: or not have_sampled ): # Sample using the env runner's module. - done_episodes, ongoing_episodes = env_runner.sample() + episodes, env_runner_results = synchronous_parallel_sample( + worker_set=self.workers, + max_agent_steps=( + self.config.rollout_fragment_length + * self.config.num_envs_per_env_runner + ), + sample_timeout_s=self.config.sample_timeout_s, + _uses_new_env_runners=True, + _return_metrics=True, + ) + self.metrics.merge_and_log_n_dicts( + env_runner_results, key=ENV_RUNNER_RESULTS + ) # Add ongoing and finished episodes into buffer. The buffer will # automatically take care of properly concatenating (by episode IDs) # the different chunks of the same episodes, even if they come in via # separate `add()` calls. - self.replay_buffer.add(episodes=done_episodes + ongoing_episodes) + self.replay_buffer.add(episodes=episodes) have_sampled = True # We took B x T env steps. - env_steps_last_regular_sample = sum( - len(eps) for eps in done_episodes + ongoing_episodes - ) + env_steps_last_regular_sample = sum(len(eps) for eps in episodes) total_sampled = env_steps_last_regular_sample # If we have never sampled before (just started the algo and not # recovered from a checkpoint), sample B random actions first. - if self._counters[NUM_AGENT_STEPS_SAMPLED] == 0: - d_, o_ = env_runner.sample( - num_timesteps=( + if self.metrics.peek(NUM_ENV_STEPS_SAMPLED_LIFETIME, default=0) == 0: + _episodes, _env_runner_results = synchronous_parallel_sample( + worker_set=self.workers, + max_agent_steps=( self.config.batch_size_B * self.config.batch_length_T - ) - - env_steps_last_regular_sample, + - env_steps_last_regular_sample + ), + sample_timeout_s=self.config.sample_timeout_s, random_actions=True, + _uses_new_env_runners=True, + _return_metrics=True, ) - self.replay_buffer.add(episodes=d_ + o_) - total_sampled += sum(len(eps) for eps in d_ + o_) - - self._counters[NUM_AGENT_STEPS_SAMPLED] += total_sampled - self._counters[NUM_ENV_STEPS_SAMPLED] += total_sampled + self.metrics.merge_and_log_n_dicts( + _env_runner_results, key=ENV_RUNNER_RESULTS + ) + self.replay_buffer.add(episodes=_episodes) + total_sampled += sum(len(eps) for eps in _episodes) + + # Update lifetime counts (now that we gathered results from all + # EnvRunners). + self.metrics.log_dict( + { + NUM_AGENT_STEPS_SAMPLED_LIFETIME: self.metrics.peek( + ENV_RUNNER_RESULTS, NUM_AGENT_STEPS_SAMPLED + ), + NUM_ENV_STEPS_SAMPLED_LIFETIME: self.metrics.peek( + ENV_RUNNER_RESULTS, NUM_ENV_STEPS_SAMPLED + ), + NUM_EPISODES_LIFETIME: self.metrics.peek( + ENV_RUNNER_RESULTS, NUM_EPISODES + ), + }, + reduce="sum", + ) # Summarize environment interaction and buffer data. - results[ALL_MODULES] = report_sampling_and_replay_buffer( - replay_buffer=self.replay_buffer, + report_sampling_and_replay_buffer( + metrics=self.metrics, replay_buffer=self.replay_buffer ) # Continue sampling batch_size_B x batch_length_T sized batches from the buffer - # and using these to update our models (`LearnerGroup.update()`) until the - # computed `training_ratio` is larger than the configured one, meaning we should - # go back and collect more samples again from the actual environment. + # and using these to update our models (`LearnerGroup.update_from_batch()`) + # until the computed `training_ratio` is larger than the configured one, meaning + # we should go back and collect more samples again from the actual environment. # However, when calculating the `training_ratio` here, we use only the # trained steps in this very `training_step()` call over the most recent sample # amount (`env_steps_last_regular_sample`), not the global values. This is to @@ -584,7 +613,7 @@ def training_step(self) -> ResultDict: replayed_steps_this_iter / env_steps_last_regular_sample ) < self.config.training_ratio: # Time individual batch updates. - with self._timers[LEARN_ON_BATCH_TIMER]: + with self.metrics.log_time((TIMERS, LEARN_ON_BATCH_TIMER)): logger.info(f"\tSub-iteration {self.training_iteration}/{sub_iter})") # Draw a new sample from the replay buffer. @@ -603,65 +632,76 @@ def training_step(self) -> ResultDict: ) # Perform the actual update via our learner group. - train_results = self.learner_group.update_from_batch( + learner_results = self.learner_group.update_from_batch( batch=SampleBatch(sample).as_multi_agent(), - reduce_fn=self._reduce_results, + # TODO(sven): Maybe we should do this broadcase of global timesteps + # at the end, like for EnvRunner global env step counts. Maybe when + # we request the state from the Learners, we can - at the same + # time - send the current globally summed/reduced-timesteps. + timesteps={ + NUM_ENV_STEPS_SAMPLED_LIFETIME: self.metrics.peek( + NUM_ENV_STEPS_SAMPLED_LIFETIME, default=0 + ) + }, ) - self._counters[NUM_AGENT_STEPS_TRAINED] += replayed_steps - self._counters[NUM_ENV_STEPS_TRAINED] += replayed_steps - - # Perform additional (non-gradient updates), such as the critic EMA-copy - # update. - with self._timers["critic_ema_update"]: - self.learner_group.additional_update( - timestep=self._counters[NUM_ENV_STEPS_SAMPLED], - reduce_fn=self._reduce_results, - ) - - if self.config.report_images_and_videos: - report_predicted_vs_sampled_obs( - # TODO (sven): DreamerV3 is single-agent only. - results=train_results[DEFAULT_MODULE_ID], - sample=sample, - batch_size_B=self.config.batch_size_B, - batch_length_T=self.config.batch_length_T, - symlog_obs=do_symlog_obs( - env_runner.env.single_observation_space, - self.config.symlog_obs, - ), - ) - - res = train_results[DEFAULT_MODULE_ID] - logger.info( - f"\t\tWORLD_MODEL_L_total={res['WORLD_MODEL_L_total']:.5f} (" - f"L_pred={res['WORLD_MODEL_L_prediction']:.5f} (" - f"decoder/obs={res['WORLD_MODEL_L_decoder']} " - f"L_rew={res['WORLD_MODEL_L_reward']} " - f"L_cont={res['WORLD_MODEL_L_continue']}); " - f"L_dyn/rep={res['WORLD_MODEL_L_dynamics']:.5f})" + self.metrics.merge_and_log_n_dicts(learner_results, key=LEARNER_RESULTS) + self.metrics.log_value( + NUM_ENV_STEPS_TRAINED_LIFETIME, replayed_steps, reduce="sum" ) - msg = "\t\t" - if self.config.train_actor: - msg += f"L_actor={res['ACTOR_L_total']:.5f} " - if self.config.train_critic: - msg += f"L_critic={res['CRITIC_L_total']:.5f} " - logger.info(msg) sub_iter += 1 - self._counters[NUM_GRAD_UPDATES_LIFETIME] += 1 + self.metrics.log_value(NUM_GRAD_UPDATES_LIFETIME, 1, reduce="sum") + + # Log videos showing how the decoder produces observation predictions + # from the posterior states. + # Only every n iterations and only for the first sampled batch row + # (videos are `config.batch_length_T` frames long). + report_predicted_vs_sampled_obs( + # TODO (sven): DreamerV3 is single-agent only. + metrics=self.metrics, + sample=sample, + batch_size_B=self.config.batch_size_B, + batch_length_T=self.config.batch_length_T, + symlog_obs=do_symlog_obs( + env_runner.env.single_observation_space, + self.config.symlog_obs, + ), + do_report=( + self.config.report_images_and_videos + and self.training_iteration % 100 == 0 + ), + ) + + # Log videos showing some of the dreamed trajectories and compare them with the + # actual trajectories from the train batch. + # Only every n iterations and only for the first sampled batch row AND first ts. + # (videos are `config.horizon_H` frames long originating from the observation + # at B=0 and T=0 in the train batch). + report_dreamed_eval_trajectory_vs_samples( + metrics=self.metrics, + sample=sample, + burn_in_T=0, + dreamed_T=self.config.horizon_H + 1, + dreamer_model=self.workers.local_worker().module.dreamer_model, + symlog_obs=do_symlog_obs( + env_runner.env.single_observation_space, + self.config.symlog_obs, + ), + do_report=( + self.config.report_dream_data and self.training_iteration % 100 == 0 + ), + ) # Update weights - after learning on the LearnerGroup - on all EnvRunner # workers. - with self._timers[SYNCH_WORKER_WEIGHTS_TIMER]: + with self.metrics.log_time((TIMERS, SYNCH_WORKER_WEIGHTS_TIMER)): # Only necessary if RLModule is not shared between (local) EnvRunner and # (local) Learner. if not self.config.share_module_between_env_runner_and_learner: - self._counters[ - NUM_TRAINING_STEP_CALLS_SINCE_LAST_SYNCH_WORKER_WEIGHTS - ] = 0 - self._counters[NUM_SYNCH_WORKER_WEIGHTS] += 1 + self.metrics.log_value(NUM_SYNCH_WORKER_WEIGHTS, 1, reduce="sum") self.workers.sync_weights( - from_worker_or_learner_group=self.learner_group + from_worker_or_learner_group=self.learner_group, + inference_only=True, ) # Try trick from https://medium.com/dive-into-ml-ai/dealing-with-memory-leak- @@ -669,33 +709,29 @@ def training_step(self) -> ResultDict: if self.config.gc_frequency_train_steps and ( self.training_iteration % self.config.gc_frequency_train_steps == 0 ): - with self._timers[GARBAGE_COLLECTION_TIMER]: + with self.metrics.log_time((TIMERS, GARBAGE_COLLECTION_TIMER)): gc.collect() # Add train results and the actual training ratio to stats. The latter should # be close to the configured `training_ratio`. - results.update(train_results) - results[ALL_MODULES]["actual_training_ratio"] = self.training_ratio + self.metrics.log_value("actual_training_ratio", self.training_ratio, window=1) # Return all results. - return results + return self.metrics.reduce() @property def training_ratio(self) -> float: - """Returns the actual training ratio of this Algorithm. + """Returns the actual training ratio of this Algorithm (not the configured one). The training ratio is copmuted by dividing the total number of steps trained thus far (replayed from the buffer) over the total number of actual env steps taken thus far. """ - return self._counters[NUM_ENV_STEPS_TRAINED] / ( - self._counters[NUM_ENV_STEPS_SAMPLED] + eps = 0.0001 + return self.metrics.peek(NUM_ENV_STEPS_TRAINED_LIFETIME, default=0) / ( + (self.metrics.peek(NUM_ENV_STEPS_SAMPLED_LIFETIME, default=eps) or eps) ) - @staticmethod - def _reduce_results(results: List[Dict[str, Any]]): - return tree.map_structure(lambda *s: np.mean(s, axis=0), *results) - # TODO (sven): Remove this once DreamerV3 is on the new SingleAgentEnvRunner. @PublicAPI def __setstate__(self, state) -> None: diff --git a/rllib/algorithms/dreamerv3/dreamerv3_learner.py b/rllib/algorithms/dreamerv3/dreamerv3_learner.py index 2ee6f4b16187..684829e3f194 100644 --- a/rllib/algorithms/dreamerv3/dreamerv3_learner.py +++ b/rllib/algorithms/dreamerv3/dreamerv3_learner.py @@ -7,61 +7,25 @@ D. Hafner, T. Lillicrap, M. Norouzi, J. Ba https://arxiv.org/pdf/2010.02193.pdf """ -from typing import Any, DefaultDict, Dict - -from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config from ray.rllib.core.learner.learner import Learner -from ray.rllib.policy.sample_batch import MultiAgentBatch -from ray.rllib.utils.annotations import override -from ray.rllib.utils.typing import ModuleID, TensorType +from ray.rllib.utils.annotations import ( + override, + OverrideToImplementCustomLogic_CallToSuperRecommended, +) class DreamerV3Learner(Learner): """DreamerV3 specific Learner class. - Only implements the `additional_update_for_module()` method to define the logic + Only implements the `_after_gradient_based_update()` method to define the logic for updating the critic EMA-copy after each training step. """ + @OverrideToImplementCustomLogic_CallToSuperRecommended @override(Learner) - def compile_results( - self, - *, - batch: MultiAgentBatch, - fwd_out: Dict[str, Any], - loss_per_module: Dict[str, TensorType], - metrics_per_module: DefaultDict[ModuleID, Dict[str, Any]], - ) -> Dict[str, Any]: - results = super().compile_results( - batch=batch, - fwd_out=fwd_out, - loss_per_module=loss_per_module, - metrics_per_module=metrics_per_module, - ) - - # Add the predicted obs distributions for possible (video) summarization. - if self.config.report_images_and_videos: - for module_id, res in results.items(): - if module_id in fwd_out: - res["WORLD_MODEL_fwd_out_obs_distribution_means_BxT"] = fwd_out[ - module_id - ]["obs_distribution_means_BxT"] - return results - - @override(Learner) - def additional_update_for_module( - self, - *, - module_id: ModuleID, - config: DreamerV3Config, - timestep: int, - ) -> None: - """Updates the EMA weights of the critic network.""" - - # Call the base class' method. - super().additional_update_for_module( - module_id=module_id, config=config, timestep=timestep - ) + def _after_gradient_based_update(self, timesteps): + super()._after_gradient_based_update(timesteps) # Update EMA weights of the critic. - self.module[module_id].critic.update_ema() + for module_id, module in self.module._rl_modules.items(): + module.critic.update_ema() diff --git a/rllib/algorithms/dreamerv3/dreamerv3_rl_module.py b/rllib/algorithms/dreamerv3/dreamerv3_rl_module.py index 9d5bf2605529..c95363eaa907 100644 --- a/rllib/algorithms/dreamerv3/dreamerv3_rl_module.py +++ b/rllib/algorithms/dreamerv3/dreamerv3_rl_module.py @@ -3,6 +3,7 @@ """ import abc +from typing import Any, Dict import gymnasium as gym import numpy as np @@ -29,6 +30,8 @@ class DreamerV3RLModule(RLModule, abc.ABC): @override(RLModule) def setup(self): + super().setup() + # Gather model-relevant settings. B = 1 T = self.config.model_config_dict["batch_length_T"] @@ -79,35 +82,39 @@ def setup(self): self.action_dist_cls = catalog.get_action_dist_cls(framework=self.framework) # Perform a test `call()` to force building the dreamer model's variables. - test_obs = np.tile( - np.expand_dims(self.config.observation_space.sample(), (0, 1)), - reps=(B, T) + (1,) * len(self.config.observation_space.shape), - ) - if isinstance(self.config.action_space, gym.spaces.Discrete): - test_actions = np.tile( - np.expand_dims( - one_hot( - self.config.action_space.sample(), - depth=self.config.action_space.n, + if self.framework == "tf2": + test_obs = np.tile( + np.expand_dims(self.config.observation_space.sample(), (0, 1)), + reps=(B, T) + (1,) * len(self.config.observation_space.shape), + ) + if isinstance(self.config.action_space, gym.spaces.Discrete): + test_actions = np.tile( + np.expand_dims( + one_hot( + self.config.action_space.sample(), + depth=self.config.action_space.n, + ), + (0, 1), ), - (0, 1), + reps=(B, T, 1), + ) + else: + test_actions = np.tile( + np.expand_dims(self.config.action_space.sample(), (0, 1)), + reps=(B, T, 1), + ) + + self.dreamer_model( + inputs=None, + observations=_convert_to_tf(test_obs, dtype=tf.float32), + actions=_convert_to_tf(test_actions, dtype=tf.float32), + is_first=_convert_to_tf(np.ones((B, T)), dtype=tf.bool), + start_is_terminated_BxT=_convert_to_tf( + np.zeros((B * T,)), dtype=tf.bool ), - reps=(B, T, 1), - ) - else: - test_actions = np.tile( - np.expand_dims(self.config.action_space.sample(), (0, 1)), - reps=(B, T, 1), + gamma=gamma, ) - self.dreamer_model( - None, - _convert_to_tf(test_obs, dtype=tf.float32), - _convert_to_tf(test_actions, dtype=tf.float32), - _convert_to_tf(np.ones((B, T)), dtype=tf.bool), - _convert_to_tf(np.zeros((B * T,)), dtype=tf.bool), - ) - # Initialize the critic EMA net: self.critic.init_ema() @@ -152,3 +159,32 @@ def output_specs_train(self) -> SpecDict: # Deterministic, continuous h-states (t1 to T). "h_states_BxT", ] + + @override(RLModule) + def _forward_inference(self, batch: NestedDict) -> Dict[str, Any]: + # Call the Dreamer-Model's forward_inference method and return a dict. + actions, next_state = self.dreamer_model.forward_inference( + observations=batch[Columns.OBS], + previous_states=batch[Columns.STATE_IN], + is_first=batch["is_first"], + ) + return {Columns.ACTIONS: actions, Columns.STATE_OUT: next_state} + + @override(RLModule) + def _forward_exploration(self, batch: NestedDict) -> Dict[str, Any]: + # Call the Dreamer-Model's forward_exploration method and return a dict. + actions, next_state = self.dreamer_model.forward_exploration( + observations=batch[Columns.OBS], + previous_states=batch[Columns.STATE_IN], + is_first=batch["is_first"], + ) + return {Columns.ACTIONS: actions, Columns.STATE_OUT: next_state} + + @override(RLModule) + def _forward_train(self, batch: NestedDict): + # Call the Dreamer-Model's forward_train method and return its outputs as-is. + return self.dreamer_model.forward_train( + observations=batch[Columns.OBS], + actions=batch[Columns.ACTIONS], + is_first=batch["is_first"], + ) diff --git a/rllib/algorithms/dreamerv3/tests/test_dreamerv3.py b/rllib/algorithms/dreamerv3/tests/test_dreamerv3.py index 92bb33dda483..0c8875b54f10 100644 --- a/rllib/algorithms/dreamerv3/tests/test_dreamerv3.py +++ b/rllib/algorithms/dreamerv3/tests/test_dreamerv3.py @@ -40,6 +40,7 @@ def test_dreamerv3_compilation(self): # Build a DreamerV3Config object. config = ( dreamerv3.DreamerV3Config() + .framework(eager_tracing=False) .training( # Keep things simple. Especially the long dream rollouts seem # to take an enormous amount of time (initially). @@ -51,7 +52,7 @@ def test_dreamerv3_compilation(self): use_float16=False, ) .learners( - num_learners=2, # Try with 2 Learners. + num_learners=0, # TODO 2 # Try with 2 Learners. num_cpus_per_learner=1, num_gpus_per_learner=0, ) @@ -59,70 +60,69 @@ def test_dreamerv3_compilation(self): num_iterations = 2 - for _ in framework_iterator(config, frameworks="tf2"): - for env in [ - "FrozenLake-v1", - "CartPole-v1", - "ALE/MsPacman-v5", - "Pendulum-v1", - ]: - print("Env={}".format(env)) - # Add one-hot observations for FrozenLake env. - if env == "FrozenLake-v1": - - def env_creator(ctx): - import gymnasium as gym - from ray.rllib.algorithms.dreamerv3.utils.env_runner import ( - OneHot, - ) - - return OneHot(gym.make("FrozenLake-v1")) - - tune.register_env("frozen-lake-one-hot", env_creator) - env = "frozen-lake-one-hot" - - config.environment(env) - algo = config.build() - obs_space = algo.workers.local_worker().env.single_observation_space - act_space = algo.workers.local_worker().env.single_action_space - rl_module = algo.workers.local_worker().module - - for i in range(num_iterations): - results = algo.train() - print(results) - # Test dream trajectory w/ recreated observations. - sample = algo.replay_buffer.sample() - dream = rl_module.dreamer_model.dream_trajectory_with_burn_in( - start_states=rl_module.dreamer_model.get_initial_state(), - timesteps_burn_in=5, - timesteps_H=45, - observations=sample["obs"][:1], # B=1 - actions=( - one_hot( - sample["actions"], - depth=act_space.n, - ) - if isinstance(act_space, gym.spaces.Discrete) - else sample["actions"] - )[ - :1 - ], # B=1 - ) - self.assertTrue( - dream["actions_dreamed_t0_to_H_BxT"].shape - == (46, 1) - + ( - (act_space.n,) - if isinstance(act_space, gym.spaces.Discrete) - else tuple(act_space.shape) + for env in [ + "FrozenLake-v1", + "CartPole-v1", + "ALE/MsPacman-v5", + "Pendulum-v1", + ]: + print("Env={}".format(env)) + # Add one-hot observations for FrozenLake env. + if env == "FrozenLake-v1": + + def env_creator(ctx): + import gymnasium as gym + from ray.rllib.algorithms.dreamerv3.utils.env_runner import ( + OneHot, ) + + return OneHot(gym.make("FrozenLake-v1")) + + tune.register_env("frozen-lake-one-hot", env_creator) + env = "frozen-lake-one-hot" + + config.environment(env) + algo = config.build() + obs_space = algo.workers.local_worker().env.single_observation_space + act_space = algo.workers.local_worker().env.single_action_space + rl_module = algo.workers.local_worker().module + + for i in range(num_iterations): + results = algo.train() + print(results) + # Test dream trajectory w/ recreated observations. + sample = algo.replay_buffer.sample() + dream = rl_module.dreamer_model.dream_trajectory_with_burn_in( + start_states=rl_module.dreamer_model.get_initial_state(), + timesteps_burn_in=5, + timesteps_H=45, + observations=sample["obs"][:1], # B=1 + actions=( + one_hot( + sample["actions"], + depth=act_space.n, + ) + if isinstance(act_space, gym.spaces.Discrete) + else sample["actions"] + )[ + :1 + ], # B=1 + ) + self.assertTrue( + dream["actions_dreamed_t0_to_H_BxT"].shape + == (46, 1) + + ( + (act_space.n,) + if isinstance(act_space, gym.spaces.Discrete) + else tuple(act_space.shape) ) - self.assertTrue(dream["continues_dreamed_t0_to_H_BxT"].shape == (46, 1)) - self.assertTrue( - dream["observations_dreamed_t0_to_H_BxT"].shape - == [46, 1] + list(obs_space.shape) - ) - algo.stop() + ) + self.assertTrue(dream["continues_dreamed_t0_to_H_BxT"].shape == (46, 1)) + self.assertTrue( + dream["observations_dreamed_t0_to_H_BxT"].shape + == [46, 1] + list(obs_space.shape) + ) + algo.stop() def test_dreamerv3_dreamer_model_sizes(self): """Tests, whether the different model sizes match the ones reported in [1].""" diff --git a/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_learner.py b/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_learner.py index 2fa6dce2f02d..d35717e4aa44 100644 --- a/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_learner.py +++ b/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_learner.py @@ -18,7 +18,6 @@ from ray.rllib.core.learner.learner import ParamDict from ray.rllib.core.learner.tf.tf_learner import TfLearner from ray.rllib.utils.annotations import override -from ray.rllib.utils.deprecation import deprecation_warning from ray.rllib.utils.framework import try_import_tf, try_import_tfp from ray.rllib.utils.tf_utils import symlog, two_hot, clip_gradients from ray.rllib.utils.typing import ModuleID, TensorType @@ -40,7 +39,7 @@ class DreamerV3TfLearner(DreamerV3Learner, TfLearner): @override(TfLearner) def configure_optimizers_for_module( - self, module_id: ModuleID, config: DreamerV3Config = None, hps=None + self, module_id: ModuleID, config: DreamerV3Config = None ): """Create the 3 optimizers for Dreamer learning: world_model, actor, critic. @@ -48,12 +47,6 @@ def configure_optimizers_for_module( - albeit probably not that important - are used by the author's own implementation. """ - if hps is not None: - deprecation_warning( - old="Learner.configure_optimizers_for_module(.., hps=..)", - help="Deprecated argument. Use `config` (AlgorithmConfig) instead.", - error=True, - ) dreamerv3_module = self._module[module_id] @@ -242,10 +235,20 @@ def compute_loss_for_module( key=module_id, window=1, # <- single items (should not be mean/ema-reduced over time). ) + + # Add the predicted obs distributions for possible (video) summarization. + if config.report_images_and_videos: + self.metrics.log_value( + (module_id, "WORLD_MODEL_fwd_out_obs_distribution_means_b0xT"), + fwd_out["obs_distribution_means_BxT"][: self.config.batch_length_T], + reduce=None, # No reduction, we want the tensor to stay in-tact. + window=1, # <- single items (should not be mean/ema-reduced over time). + ) + if config.report_individual_batch_item_stats: # Log important world-model loss stats. self.metrics.log_dict( - metrics_dict={ + { "WORLD_MODEL_L_decoder_B_T": prediction_losses["L_decoder_B_T"], "WORLD_MODEL_L_reward_B_T": prediction_losses["L_reward_B_T"], "WORLD_MODEL_L_continue_B_T": prediction_losses["L_continue_B_T"], @@ -270,10 +273,10 @@ def compute_loss_for_module( "h": fwd_out["h_states_BxT"], "z": fwd_out["z_posterior_states_BxT"], }, - start_is_terminated=tf.reshape(batch["is_terminated"], [-1]), # ->BxT + start_is_terminated=tf.reshape(batch["is_terminated"], [-1]), # -> BxT ) if config.report_dream_data: - # To reduce this massive mount of data a little, slice out a T=1 piece + # To reduce this massive amount of data a little, slice out a T=1 piece # from each stats that has the shape (H, BxT), meaning convert e.g. # `rewards_dreamed_t0_to_H_BxT` into `rewards_dreamed_t0_to_H_Bx1`. # This will reduce the amount of data to be transferred and reported @@ -281,9 +284,9 @@ def compute_loss_for_module( self.metrics.log_dict( { # Replace 'T' with '1'. - "DREAM_DATA_" - + key[:-1] - + "1": (value[:, config.batch_size_B_per_learner]) + f"DREAM_DATA_{key[:-1]}1": ( + value[:, config.batch_size_B_per_learner] + ) for key, value in dream_data.items() if key.endswith("H_BxT") }, @@ -733,7 +736,7 @@ def _compute_critic_loss( :-1 ] - # Reduce over H- (time) axis (sum) and then B-axis (mean). + # Reduce over both H- (time) axis and B-axis (mean). L_critic = tf.reduce_mean(L_critic_H_B) # Log important critic loss stats. diff --git a/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_rl_module.py b/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_rl_module.py index a05c516d29bd..44952e829741 100644 --- a/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_rl_module.py +++ b/rllib/algorithms/dreamerv3/tf/dreamerv3_tf_rl_module.py @@ -7,17 +7,8 @@ D. Hafner, T. Lillicrap, M. Norouzi, J. Ba https://arxiv.org/pdf/2010.02193.pdf """ -from typing import Any, Dict - from ray.rllib.algorithms.dreamerv3.dreamerv3_rl_module import DreamerV3RLModule -from ray.rllib.core.columns import Columns -from ray.rllib.core.rl_module.rl_module import RLModule from ray.rllib.core.rl_module.tf.tf_rl_module import TfRLModule -from ray.rllib.utils.annotations import override -from ray.rllib.utils.framework import try_import_tf -from ray.rllib.utils.nested_dict import NestedDict - -tf1, tf, _ = try_import_tf() class DreamerV3TfRLModule(TfRLModule, DreamerV3RLModule): @@ -26,33 +17,4 @@ class DreamerV3TfRLModule(TfRLModule, DreamerV3RLModule): Serves mainly as a thin-wrapper around the `DreamerModel` (a tf.keras.Model) class. """ - framework: str = "tf2" - - @override(RLModule) - def _forward_inference(self, batch: NestedDict) -> Dict[str, Any]: - # Call the Dreamer-Model's forward_inference method and return a dict. - actions, next_state = self.dreamer_model.forward_inference( - observations=batch[Columns.OBS], - previous_states=batch[Columns.STATE_IN], - is_first=batch["is_first"], - ) - return {Columns.ACTIONS: actions, Columns.STATE_OUT: next_state} - - @override(RLModule) - def _forward_exploration(self, batch: NestedDict) -> Dict[str, Any]: - # Call the Dreamer-Model's forward_exploration method and return a dict. - actions, next_state = self.dreamer_model.forward_exploration( - observations=batch[Columns.OBS], - previous_states=batch[Columns.STATE_IN], - is_first=batch["is_first"], - ) - return {Columns.ACTIONS: actions, Columns.STATE_OUT: next_state} - - @override(RLModule) - def _forward_train(self, batch: NestedDict): - # Call the Dreamer-Model's forward_train method and return its outputs as-is. - return self.dreamer_model.forward_train( - observations=batch[Columns.OBS], - actions=batch[Columns.ACTIONS], - is_first=batch["is_first"], - ) + framework = "tf2" diff --git a/rllib/algorithms/dreamerv3/tf/models/actor_network.py b/rllib/algorithms/dreamerv3/tf/models/actor_network.py index 44785323711f..6fe1a7ef5b71 100644 --- a/rllib/algorithms/dreamerv3/tf/models/actor_network.py +++ b/rllib/algorithms/dreamerv3/tf/models/actor_network.py @@ -3,8 +3,6 @@ D. Hafner, J. Pasukonis, J. Ba, T. Lillicrap https://arxiv.org/pdf/2301.04104v1.pdf """ -from typing import Optional - import gymnasium as gym from gymnasium.spaces import Box, Discrete import numpy as np @@ -35,15 +33,15 @@ class ActorNetwork(tf.keras.Model): def __init__( self, *, - model_size: Optional[str] = "XS", + model_size: str = "XS", action_space: gym.Space, ): """Initializes an ActorNetwork instance. Args: - model_size: The "Model Size" used according to [1] Appendinx B. + model_size: The "Model Size" used according to [1] Appendix B. Use None for manually setting the different network sizes. - action_space: The action space the our environment used. + action_space: The action space the our environment used. """ super().__init__(name="actor") diff --git a/rllib/algorithms/dreamerv3/tf/models/components/cnn_atari.py b/rllib/algorithms/dreamerv3/tf/models/components/cnn_atari.py index 16e733ce4b17..c0f7ee09b092 100644 --- a/rllib/algorithms/dreamerv3/tf/models/components/cnn_atari.py +++ b/rllib/algorithms/dreamerv3/tf/models/components/cnn_atari.py @@ -23,7 +23,7 @@ def __init__( """Initializes a CNNAtari instance. Args: - model_size: The "Model Size" used according to [1] Appendinx B. + model_size: The "Model Size" used according to [1] Appendix B. Use None for manually setting the `cnn_multiplier`. cnn_multiplier: Optional override for the additional factor used to multiply the number of filters with each CNN layer. Starting with diff --git a/rllib/algorithms/dreamerv3/tf/models/components/continue_predictor.py b/rllib/algorithms/dreamerv3/tf/models/components/continue_predictor.py index dd948b9951f0..d5434d8aca31 100644 --- a/rllib/algorithms/dreamerv3/tf/models/components/continue_predictor.py +++ b/rllib/algorithms/dreamerv3/tf/models/components/continue_predictor.py @@ -3,8 +3,6 @@ D. Hafner, J. Pasukonis, J. Ba, T. Lillicrap https://arxiv.org/pdf/2301.04104v1.pdf """ -from typing import Optional - from ray.rllib.algorithms.dreamerv3.tf.models.components.mlp import MLP from ray.rllib.algorithms.dreamerv3.utils import ( get_gru_units, @@ -29,7 +27,7 @@ class ContinuePredictor(tf.keras.Model): terminal. """ - def __init__(self, *, model_size: Optional[str] = "XS"): + def __init__(self, *, model_size: str = "XS"): """Initializes a ContinuePredictor instance. Args: diff --git a/rllib/algorithms/dreamerv3/tf/models/components/reward_predictor.py b/rllib/algorithms/dreamerv3/tf/models/components/reward_predictor.py index c281565897cb..3e7cb6de93f9 100644 --- a/rllib/algorithms/dreamerv3/tf/models/components/reward_predictor.py +++ b/rllib/algorithms/dreamerv3/tf/models/components/reward_predictor.py @@ -3,8 +3,6 @@ D. Hafner, J. Pasukonis, J. Ba, T. Lillicrap https://arxiv.org/pdf/2301.04104v1.pdf """ -from typing import Optional - from ray.rllib.algorithms.dreamerv3.tf.models.components.mlp import MLP from ray.rllib.algorithms.dreamerv3.tf.models.components.reward_predictor_layer import ( RewardPredictorLayer, @@ -28,7 +26,7 @@ class RewardPredictor(tf.keras.Model): def __init__( self, *, - model_size: Optional[str] = "XS", + model_size: str = "XS", num_buckets: int = 255, lower_bound: float = -20.0, upper_bound: float = 20.0, diff --git a/rllib/algorithms/dreamerv3/tf/models/components/sequence_model.py b/rllib/algorithms/dreamerv3/tf/models/components/sequence_model.py index 0d2de1970471..7e21c9860578 100644 --- a/rllib/algorithms/dreamerv3/tf/models/components/sequence_model.py +++ b/rllib/algorithms/dreamerv3/tf/models/components/sequence_model.py @@ -76,7 +76,6 @@ def __init__( num_gru_units, return_sequences=False, return_state=False, - time_major=True, # Note: Changing these activations is most likely a bad idea! # In experiments, setting one of both of them to silu deteriorated # performance significantly. @@ -139,7 +138,7 @@ def call(self, a, h, z): ) # Pass through pre-GRU layer. out = self.pre_gru_layer(out) - # Pass through (time-major) GRU. - h_next = self.gru_unit(tf.expand_dims(out, axis=0), initial_state=h) + # Pass through (batch-major) GRU (expand axis=1 as the time axis). + h_next = self.gru_unit(tf.expand_dims(out, axis=1), initial_state=h) # Return the GRU's output (the next h-state). return h_next diff --git a/rllib/algorithms/dreamerv3/tf/models/components/vector_decoder.py b/rllib/algorithms/dreamerv3/tf/models/components/vector_decoder.py index a384d1473bda..e183561f9217 100644 --- a/rllib/algorithms/dreamerv3/tf/models/components/vector_decoder.py +++ b/rllib/algorithms/dreamerv3/tf/models/components/vector_decoder.py @@ -3,8 +3,6 @@ D. Hafner, J. Pasukonis, J. Ba, T. Lillicrap https://arxiv.org/pdf/2301.04104v1.pdf """ -from typing import Optional - import gymnasium as gym from ray.rllib.algorithms.dreamerv3.tf.models.components.mlp import MLP @@ -28,7 +26,7 @@ class VectorDecoder(tf.keras.Model): def __init__( self, *, - model_size: Optional[str] = "XS", + model_size: str = "XS", observation_space: gym.Space, ): """Initializes a VectorDecoder instance. diff --git a/rllib/algorithms/dreamerv3/tf/models/critic_network.py b/rllib/algorithms/dreamerv3/tf/models/critic_network.py index e2b2d45d9435..4eb9b9940133 100644 --- a/rllib/algorithms/dreamerv3/tf/models/critic_network.py +++ b/rllib/algorithms/dreamerv3/tf/models/critic_network.py @@ -3,8 +3,6 @@ D. Hafner, J. Pasukonis, J. Ba, T. Lillicrap https://arxiv.org/pdf/2301.04104v1.pdf """ -from typing import Optional - from ray.rllib.algorithms.dreamerv3.tf.models.components.mlp import MLP from ray.rllib.algorithms.dreamerv3.tf.models.components.reward_predictor_layer import ( RewardPredictorLayer, @@ -33,7 +31,7 @@ class CriticNetwork(tf.keras.Model): def __init__( self, *, - model_size: Optional[str] = "XS", + model_size: str = "XS", num_buckets: int = 255, lower_bound: float = -20.0, upper_bound: float = 20.0, diff --git a/rllib/algorithms/dreamerv3/tf/models/dreamer_model.py b/rllib/algorithms/dreamerv3/tf/models/dreamer_model.py index 26a44d1fb3b0..dc4eec8579ae 100644 --- a/rllib/algorithms/dreamerv3/tf/models/dreamer_model.py +++ b/rllib/algorithms/dreamerv3/tf/models/dreamer_model.py @@ -113,6 +113,7 @@ def call( actions, is_first, start_is_terminated_BxT, + gamma, ): """Main call method for building this model in order to generate its variables. diff --git a/rllib/algorithms/dreamerv3/utils/__init__.py b/rllib/algorithms/dreamerv3/utils/__init__.py index 592bbf9b32e8..fe7b58cf515e 100644 --- a/rllib/algorithms/dreamerv3/utils/__init__.py +++ b/rllib/algorithms/dreamerv3/utils/__init__.py @@ -124,7 +124,7 @@ def get_num_curiosity_nets(model_size, override=None): num_curiosity_nets = { "nano": 8, "micro": 8, - "mini": 16, + "mini": 8, "XXS": 8, "XS": 8, "S": 8, diff --git a/rllib/algorithms/dreamerv3/utils/env_runner.py b/rllib/algorithms/dreamerv3/utils/env_runner.py index c0b73ef824fd..93ed2beb6240 100644 --- a/rllib/algorithms/dreamerv3/utils/env_runner.py +++ b/rllib/algorithms/dreamerv3/utils/env_runner.py @@ -15,21 +15,42 @@ import numpy as np import tree # pip install dm_tree +import ray from ray.rllib.algorithms.algorithm_config import AlgorithmConfig -from ray.rllib.core import DEFAULT_MODULE_ID +from ray.rllib.core import DEFAULT_AGENT_ID, DEFAULT_MODULE_ID from ray.rllib.core.columns import Columns from ray.rllib.env.env_runner import EnvRunner +from ray.rllib.env.single_agent_episode import SingleAgentEpisode from ray.rllib.env.wrappers.atari_wrappers import NoopResetEnv, MaxAndSkipEnv from ray.rllib.env.wrappers.dm_control_wrapper import DMCEnv from ray.rllib.env.utils import _gym_env_creator -from ray.rllib.evaluation.metrics import RolloutMetrics from ray.rllib.utils.annotations import override -from ray.rllib.utils.framework import try_import_tf -from ray.rllib.env.single_agent_episode import SingleAgentEpisode -from ray.rllib.utils.numpy import one_hot +from ray.rllib.utils.framework import try_import_tf, try_import_torch +from ray.rllib.utils.metrics import ( + EPISODE_DURATION_SEC_MEAN, + EPISODE_LEN_MAX, + EPISODE_LEN_MEAN, + EPISODE_LEN_MIN, + EPISODE_RETURN_MAX, + EPISODE_RETURN_MEAN, + EPISODE_RETURN_MIN, + NUM_AGENT_STEPS_SAMPLED, + NUM_AGENT_STEPS_SAMPLED_LIFETIME, + NUM_EPISODES, + NUM_ENV_STEPS_SAMPLED, + NUM_ENV_STEPS_SAMPLED_LIFETIME, + NUM_MODULE_STEPS_SAMPLED, + NUM_MODULE_STEPS_SAMPLED_LIFETIME, +) +from ray.rllib.utils.metrics.metrics_logger import MetricsLogger +from ray.rllib.utils.numpy import convert_to_numpy, one_hot +from ray.rllib.utils.spaces.space_utils import batch, unbatch +from ray.rllib.utils.torch_utils import convert_to_torch_tensor +from ray.rllib.utils.typing import ResultDict from ray.tune.registry import ENV_CREATOR, _global_registry _, tf, _ = try_import_tf() +torch, _ = try_import_torch() # TODO (sven): Use SingleAgentEnvRunner instead of this as soon as we have the new @@ -70,23 +91,38 @@ def __init__( # However, in Danijar's repo, Atari100k experiments are configured as: # noop=30, 64x64x3 (no grayscaling), sticky actions=False, # full action space=False, - wrappers = [ - partial(gym.wrappers.TimeLimit, max_episode_steps=108000), - partial(resize_v1, x_size=64, y_size=64), # resize to 64x64 - NormalizedImageEnv, - NoopResetEnv, - MaxAndSkipEnv, - ] + + def _entry_point(): + return gym.make( + self.config.env, + **dict( + self.config.env_config, + **{ + # "sticky actions" but not according to Danijar's 100k + # configs. + "repeat_action_probability": 0.0, + # "full action space" but not according to Danijar's 100k + # configs. + "full_action_space": False, + # Already done by MaxAndSkip wrapper: "action repeat" == 4. + "frameskip": 1, + }, + ), + ) + + gym.register("rllib-single-agent-env-v0", entry_point=_entry_point) self.env = gym.vector.make( - "GymV26Environment-v0", - env_id=self.config.env, - wrappers=wrappers, + "rllib-single-agent-env-v0", num_envs=self.config.num_envs_per_env_runner, asynchronous=self.config.remote_worker_envs, - make_kwargs=dict( - self.config.env_config, **{"render_mode": "rgb_array"} - ), + wrappers=[ + partial(gym.wrappers.TimeLimit, max_episode_steps=108000), + partial(resize_v1, x_size=64, y_size=64), # resize to 64x64 + NormalizedImageEnv, + NoopResetEnv, + MaxAndSkipEnv, + ], ) # DeepMind Control. elif self.config.env.startswith("DMC/"): @@ -147,6 +183,24 @@ def __init__( # TODO (sven): DreamerV3 is currently single-agent only. self.module = self.marl_module_spec.build()[DEFAULT_MODULE_ID] + self.metrics = MetricsLogger() + + self._device = None + if ( + torch + and torch.cuda.is_available() + and self.config.framework_str == "torch" + and self.config.share_module_between_env_runner_and_learner + and self.config.num_gpus_per_learner > 0 + ): + gpu_ids = ray.get_gpu_ids() + self._device = f"cuda:{gpu_ids[0]}" + self.convert_to_tensor = ( + partial(convert_to_torch_tensor, device=self._device) + if self.config.framework_str == "torch" + else tf.convert_to_tensor + ) + self._needs_initial_reset = True self._episodes = [None for _ in range(self.num_envs)] self._states = [None for _ in range(self.num_envs)] @@ -158,7 +212,6 @@ def __init__( # via its replay buffer, etc..). self._done_episodes_for_metrics = [] self._ongoing_episodes_for_metrics = defaultdict(list) - self._ts_since_last_metrics = 0 @override(EnvRunner) def sample( @@ -228,7 +281,7 @@ def _sample_timesteps( explore: bool = True, random_actions: bool = False, force_reset: bool = False, - ) -> Tuple[List[SingleAgentEpisode], List[SingleAgentEpisode]]: + ) -> List[SingleAgentEpisode]: """Helper method to run n timesteps. See docstring of self.sample() for more details. @@ -238,47 +291,25 @@ def _sample_timesteps( # Get initial states for all `batch_size_B` rows in the forward batch. initial_states = tree.map_structure( lambda s: np.repeat(s, self.num_envs, axis=0), - self.module.get_initial_state(), + convert_to_numpy(self.module.get_initial_state()), ) # Have to reset the env (on all vector sub-envs). if force_reset or self._needs_initial_reset: obs, _ = self.env.reset() + self._needs_initial_reset = False self._episodes = [SingleAgentEpisode() for _ in range(self.num_envs)] - states = initial_states - # Set is_first to True for all rows (all sub-envs just got reset). - is_first = np.ones((self.num_envs,)) - self._needs_initial_reset = False # Set initial obs and states in the episodes. for i in range(self.num_envs): self._episodes[i].add_env_reset(observation=obs[i]) - self._states[i] = {k: s[i] for k, s in states.items()} + self._states[i] = None + # Don't reset existing envs; continue in already started episodes. else: # Pick up stored observations and states from previous timesteps. obs = np.stack([eps.observations[-1] for eps in self._episodes]) - # Compile the initial state for each batch row: If episode just started, use - # model's initial state, if not, use state stored last in - # SingleAgentEpisode. - states = { - k: np.stack( - [ - initial_states[k][i] - if self._states[i] is None - else self._states[i][k] - for i, eps in enumerate(self._episodes) - ] - ) - for k in initial_states.keys() - } - # If a batch row is at the beginning of an episode, set its `is_first` flag - # to 1.0, otherwise 0.0. - is_first = np.zeros((self.num_envs,)) - for i, eps in enumerate(self._episodes): - if len(eps) == 0: - is_first[i] = 1.0 # Loop through env for n timesteps. ts = 0 @@ -288,33 +319,35 @@ def _sample_timesteps( actions = self.env.action_space.sample() # Compute an action using our RLModule. else: - batch = { + is_first = np.zeros((self.num_envs,)) + for i, eps in enumerate(self._episodes): + if self._states[i] is None: + is_first[i] = 1.0 + self._states[i] = {k: s[i] for k, s in initial_states.items()} + to_module = { Columns.STATE_IN: tree.map_structure( - lambda s: tf.convert_to_tensor(s), states + lambda s: self.convert_to_tensor(s), batch(self._states) ), - Columns.OBS: tf.convert_to_tensor(obs), - "is_first": tf.convert_to_tensor(is_first), + Columns.OBS: self.convert_to_tensor(obs), + "is_first": self.convert_to_tensor(is_first), } # Explore or not. if explore: - outs = self.module.forward_exploration(batch) + outs = self.module.forward_exploration(to_module) else: - outs = self.module.forward_inference(batch) + outs = self.module.forward_inference(to_module) # Model outputs one-hot actions (if discrete). Convert to int actions # as well. - actions = outs[Columns.ACTIONS].numpy() + actions = convert_to_numpy(outs[Columns.ACTIONS]) if isinstance(self.env.single_action_space, gym.spaces.Discrete): actions = np.argmax(actions, axis=-1) - states = tree.map_structure( - lambda s: s.numpy(), outs[Columns.STATE_OUT] - ) + self._states = unbatch(convert_to_numpy(outs[Columns.STATE_OUT])) obs, rewards, terminateds, truncateds, infos = self.env.step(actions) ts += self.num_envs for i in range(self.num_envs): - s = {k: s[i] for k, s in states.items()} # The last entry in self.observations[i] is already the reset # obs of the new episode. if terminateds[i] or truncateds[i]: @@ -327,12 +360,7 @@ def _sample_timesteps( terminated=terminateds[i], truncated=truncateds[i], ) - self._states[i] = s - # Reset h-states to the model's initial ones b/c we are starting a - # new episode. - for k, v in self.module.get_initial_state().items(): - states[k][i] = v.numpy() - is_first[i] = True + self._states[i] = None done_episodes_to_return.append(self._episodes[i]) # Create a new episode object. self._episodes[i] = SingleAgentEpisode(observations=[obs[i]]) @@ -342,9 +370,6 @@ def _sample_timesteps( action=actions[i], reward=rewards[i], ) - is_first[i] = False - - self._states[i] = s # Return done episodes ... self._done_episodes_for_metrics.extend(done_episodes_to_return) @@ -356,9 +381,9 @@ def _sample_timesteps( for eps in ongoing_episodes: self._ongoing_episodes_for_metrics[eps.id_].append(eps) - self._ts_since_last_metrics += ts + self._increase_sampled_metrics(ts) - return done_episodes_to_return, ongoing_episodes + return done_episodes_to_return + ongoing_episodes def _sample_episodes( self, @@ -378,7 +403,7 @@ def _sample_episodes( # Multiply states n times according to our vector env batch size (num_envs). states = tree.map_structure( lambda s: np.repeat(s, self.num_envs, axis=0), - self.module.get_initial_state(), + convert_to_numpy(self.module.get_initial_state()), ) is_first = np.ones((self.num_envs,)) @@ -392,10 +417,10 @@ def _sample_episodes( else: batch = { Columns.STATE_IN: tree.map_structure( - lambda s: tf.convert_to_tensor(s), states + lambda s: self.convert_to_tensor(s), states ), - Columns.OBS: tf.convert_to_tensor(obs), - "is_first": tf.convert_to_tensor(is_first), + Columns.OBS: self.convert_to_tensor(obs), + "is_first": self.convert_to_tensor(is_first), } if explore: @@ -403,12 +428,10 @@ def _sample_episodes( else: outs = self.module.forward_inference(batch) - actions = outs[Columns.ACTIONS].numpy() + actions = convert_to_numpy(outs[Columns.ACTIONS]) if isinstance(self.env.single_action_space, gym.spaces.Discrete): actions = np.argmax(actions, axis=-1) - states = tree.map_structure( - lambda s: s.numpy(), outs[Columns.STATE_OUT] - ) + states = convert_to_numpy(outs[Columns.STATE_OUT]) obs, rewards, terminateds, truncateds, infos = self.env.step(actions) @@ -434,8 +457,10 @@ def _sample_episodes( # Reset h-states to the model's initial ones b/c we are starting a # new episode. - for k, v in self.module.get_initial_state().items(): - states[k][i] = v.numpy() + for k, v in convert_to_numpy( + self.module.get_initial_state() + ).items(): + states[k][i] = v is_first[i] = True episodes[i] = SingleAgentEpisode(observations=[obs[i]]) @@ -448,41 +473,50 @@ def _sample_episodes( is_first[i] = False self._done_episodes_for_metrics.extend(done_episodes_to_return) - self._ts_since_last_metrics += sum(len(eps) for eps in done_episodes_to_return) # If user calls sample(num_timesteps=..) after this, we must reset again # at the beginning. self._needs_initial_reset = True + ts = sum(map(len, done_episodes_to_return)) + self._increase_sampled_metrics(ts) + return done_episodes_to_return - # TODO (sven): Remove the requirement for EnvRunners/RolloutWorkers to have this - # API. Instead Algorithm should compile episode metrics itself via its local - # buffer. - def get_metrics(self) -> List[RolloutMetrics]: + def get_metrics(self) -> ResultDict: # Compute per-episode metrics (only on already completed episodes). - metrics = [] for eps in self._done_episodes_for_metrics: + assert eps.is_done + episode_length = len(eps) - episode_reward = eps.get_return() + episode_return = eps.get_return() + episode_duration_s = eps.get_duration_s() + # Don't forget about the already returned chunks of this episode. if eps.id_ in self._ongoing_episodes_for_metrics: for eps2 in self._ongoing_episodes_for_metrics[eps.id_]: episode_length += len(eps2) - episode_reward += eps2.get_return() + episode_return += eps2.get_return() del self._ongoing_episodes_for_metrics[eps.id_] - metrics.append( - RolloutMetrics( - episode_length=episode_length, - episode_reward=episode_reward, - ) + self._log_episode_metrics( + episode_length, episode_return, episode_duration_s ) + # Log num episodes counter for this iteration. + self.metrics.log_value( + NUM_EPISODES, + len(self._done_episodes_for_metrics), + reduce="sum", + # Reset internal data on `reduce()` call below (not a lifetime count). + clear_on_reduce=True, + ) + + # Now that we have logged everything, clear cache of done episodes. self._done_episodes_for_metrics.clear() - self._ts_since_last_metrics = 0 - return metrics + # Return reduced metrics. + return self.metrics.reduce() # TODO (sven): Remove the requirement for EnvRunners/RolloutWorkers to have this # API. Replace by proper state overriding via `EnvRunner.set_state()` @@ -503,6 +537,52 @@ def stop(self): # Close our env object via gymnasium's API. self.env.close() + def _increase_sampled_metrics(self, num_steps): + # Per sample cycle stats. + self.metrics.log_value( + NUM_ENV_STEPS_SAMPLED, num_steps, reduce="sum", clear_on_reduce=True + ) + self.metrics.log_value( + (NUM_AGENT_STEPS_SAMPLED, DEFAULT_AGENT_ID), + num_steps, + reduce="sum", + clear_on_reduce=True, + ) + self.metrics.log_value( + (NUM_MODULE_STEPS_SAMPLED, DEFAULT_MODULE_ID), + num_steps, + reduce="sum", + clear_on_reduce=True, + ) + # Lifetime stats. + self.metrics.log_value(NUM_ENV_STEPS_SAMPLED_LIFETIME, num_steps, reduce="sum") + self.metrics.log_value( + (NUM_AGENT_STEPS_SAMPLED_LIFETIME, DEFAULT_AGENT_ID), + num_steps, + reduce="sum", + ) + self.metrics.log_value( + (NUM_MODULE_STEPS_SAMPLED_LIFETIME, DEFAULT_MODULE_ID), + num_steps, + reduce="sum", + ) + return num_steps + + def _log_episode_metrics(self, length, ret, sec): + # Log general episode metrics. + # To mimick the old API stack behavior, we'll use `window` here for + # these particular stats (instead of the default EMA). + win = self.config.metrics_num_episodes_for_smoothing + self.metrics.log_value(EPISODE_LEN_MEAN, length, window=win) + self.metrics.log_value(EPISODE_RETURN_MEAN, ret, window=win) + self.metrics.log_value(EPISODE_DURATION_SEC_MEAN, sec, window=win) + + # For some metrics, log min/max as well. + self.metrics.log_value(EPISODE_LEN_MIN, length, reduce="min") + self.metrics.log_value(EPISODE_RETURN_MIN, ret, reduce="min") + self.metrics.log_value(EPISODE_LEN_MAX, length, reduce="max") + self.metrics.log_value(EPISODE_RETURN_MAX, ret, reduce="max") + class NormalizedImageEnv(gym.ObservationWrapper): def __init__(self, *args, **kwargs): diff --git a/rllib/algorithms/dreamerv3/utils/summaries.py b/rllib/algorithms/dreamerv3/utils/summaries.py index f78876c83fe7..dd36adbb3160 100644 --- a/rllib/algorithms/dreamerv3/utils/summaries.py +++ b/rllib/algorithms/dreamerv3/utils/summaries.py @@ -13,16 +13,16 @@ create_cartpole_dream_image, create_frozenlake_dream_image, ) +from ray.rllib.core import DEFAULT_MODULE_ID from ray.rllib.core.columns import Columns +from ray.rllib.utils.framework import try_import_torch +from ray.rllib.utils.metrics import ( + LEARNER_RESULTS, + REPLAY_BUFFER_RESULTS, +) from ray.rllib.utils.tf_utils import inverse_symlog - -def _summarize(*, results, data_to_summarize, keys_to_log, include_histograms=False): - for k in keys_to_log: - if data_to_summarize[k].shape == (): - results.update({k: data_to_summarize[k]}) - elif include_histograms: - results.update({k: data_to_summarize[k]}) +torch, _ = try_import_torch() def reconstruct_obs_from_h_and_z( @@ -39,10 +39,18 @@ def reconstruct_obs_from_h_and_z( # Note that the last h-state (T+1) is NOT used here as it's already part of # a new trajectory. # Use mean() of the Gaussian, no sample! -> No need to construct dist object here. - reconstructed_obs_distr_means_TxB = dreamer_model.world_model.decoder( - # Fold time rank. - h=np.reshape(h_t0_to_H, (T * B, -1)), - z=np.reshape(z_t0_to_H, (T * B,) + z_t0_to_H.shape[2:]), + device = next(iter(dreamer_model.world_model.decoder.parameters())).device + reconstructed_obs_distr_means_TxB = ( + dreamer_model.world_model.decoder( + # Fold time rank. + h=torch.from_numpy(h_t0_to_H).reshape((T * B, -1)).to(device), + z=torch.from_numpy(z_t0_to_H) + .reshape((T * B,) + z_t0_to_H.shape[2:]) + .to(device), + ) + .detach() + .cpu() + .numpy() ) # Unfold time rank again. reconstructed_obs_T_B = np.reshape( @@ -115,11 +123,12 @@ def report_dreamed_trajectory( def report_predicted_vs_sampled_obs( *, - results, + metrics, sample, batch_size_B, batch_length_T, symlog_obs: bool = True, + do_report: bool = True, ): """Summarizes sampled data (from the replay buffer) vs world-model predictions. @@ -133,127 +142,188 @@ def report_predicted_vs_sampled_obs( Continues: Compute MSE (sampled vs predicted). Args: - results: The results dict that was returned by - `LearnerGroup.update_from_batch()`. + metrics: The MetricsLogger object of the DreamerV3 algo. sample: The sampled data (dict) from the replay buffer. Already tf-tensor converted. batch_size_B: The batch size (B). This is the number of trajectories sampled from the buffer. batch_length_T: The batch length (T). This is the length of an individual trajectory sampled from the buffer. + do_report: Whether to actually log the report (default). If this is set to + False, this function serves as a clean-up on the given metrics, making sure + they do NOT contain anymore any (spacious) data relevant for producing + the report/videos. """ - predicted_observation_means_BxT = results[ - "WORLD_MODEL_fwd_out_obs_distribution_means_BxT" - ] + fwd_output_key = ( + LEARNER_RESULTS, + DEFAULT_MODULE_ID, + "WORLD_MODEL_fwd_out_obs_distribution_means_b0xT", + ) + # logged as a non-reduced item (still a list) + predicted_observation_means_single_example = metrics.peek( + fwd_output_key, default=[None] + )[-1] + metrics.delete(fwd_output_key, key_error=False) + + final_result_key = ( + f"WORLD_MODEL_sampled_vs_predicted_posterior_b0x{batch_length_T}_videos" + ) + if not do_report: + metrics.delete(final_result_key, key_error=False) + return + _report_obs( - results=results, + metrics=metrics, computed_float_obs_B_T_dims=np.reshape( - predicted_observation_means_BxT, - (batch_size_B, batch_length_T) + sample[Columns.OBS].shape[2:], + predicted_observation_means_single_example, + # WandB videos need to be channels first. + (1, batch_length_T) + sample[Columns.OBS].shape[2:], ), - sampled_obs_B_T_dims=sample[Columns.OBS], - descr_prefix="WORLD_MODEL", - descr_obs=f"predicted_posterior_T{batch_length_T}", + sampled_obs_B_T_dims=sample[Columns.OBS][0:1], + metrics_key=final_result_key, symlog_obs=symlog_obs, ) def report_dreamed_eval_trajectory_vs_samples( *, - results, - dream_data, + metrics, sample, burn_in_T, dreamed_T, dreamer_model, symlog_obs: bool = True, -): + do_report: bool = True, +) -> None: + """Logs dreamed observations, rewards, continues and compares them vs sampled data. + + For obs, we'll try to create videos (side-by-side comparison) of the dreamed, + recreated-from-prior obs vs the sampled ones (over dreamed_T timesteps). + + Args: + metrics: The MetricsLogger object of the DreamerV3 algo. + sample: The sampled data (dict) from the replay buffer. Already tf-tensor + converted. + burn_in_T: The number of burn-in timesteps (these will be skipped over in the + reported video comparisons and MSEs). + dreamed_T: The number of timesteps to produce dreamed data for. + dreamer_model: The DreamerModel to use to create observation vectors/images + from dreamed h- and (prior) z-states. + symlog_obs: Whether to inverse-symlog the computed observations or not. Set this + to True for environments, in which we should symlog the observations. + do_report: Whether to actually log the report (default). If this is set to + False, this function serves as a clean-up on the given metrics, making sure + they do NOT contain anymore any (spacious) data relevant for producing + the report/videos. + """ + dream_data = metrics.peek( + LEARNER_RESULTS, + DEFAULT_MODULE_ID, + "dream_data", + default={}, + ) + metrics.delete(LEARNER_RESULTS, DEFAULT_MODULE_ID, "dream_data", key_error=False) + + final_result_key_obs = f"EVALUATION_sampled_vs_dreamed_prior_H{dreamed_T}_obs" + final_result_key_rew = ( + f"EVALUATION_sampled_vs_dreamed_prior_H{dreamed_T}_rewards_MSE" + ) + final_result_key_cont = ( + f"EVALUATION_sampled_vs_dreamed_prior_H{dreamed_T}_continues_MSE" + ) + if not do_report: + metrics.delete(final_result_key_obs, key_error=False) + metrics.delete(final_result_key_rew, key_error=False) + metrics.delete(final_result_key_cont, key_error=False) + return + # Obs MSE. - dreamed_obs_T_B = reconstruct_obs_from_h_and_z( - h_t0_to_H=dream_data["h_states_t0_to_H_BxT"], - z_t0_to_H=dream_data["z_states_prior_t0_to_H_BxT"], + dreamed_obs_H_B = reconstruct_obs_from_h_and_z( + h_t0_to_H=dream_data["h_states_t0_to_H_Bx1"][0], + z_t0_to_H=dream_data["z_states_prior_t0_to_H_Bx1"][0], dreamer_model=dreamer_model, obs_dims_shape=sample[Columns.OBS].shape[2:], ) - t0 = burn_in_T - 1 + t0 = burn_in_T tH = t0 + dreamed_T # Observation MSE and - if applicable - images comparisons. - mse_sampled_vs_dreamed_obs = _report_obs( - results=results, - # Have to transpose b/c dreamed data is time-major. - computed_float_obs_B_T_dims=np.transpose( - dreamed_obs_T_B, - axes=[1, 0] + list(range(2, len(dreamed_obs_T_B.shape))), - ), - sampled_obs_B_T_dims=sample[Columns.OBS][:, t0 : tH + 1], - descr_prefix="EVALUATION", - descr_obs=f"dreamed_prior_H{dreamed_T}", + _report_obs( + metrics=metrics, + # WandB videos need to be 5D (B, L, c, h, w) -> transpose/swap H and B axes. + computed_float_obs_B_T_dims=np.swapaxes(dreamed_obs_H_B, 0, 1)[ + 0:1 + ], # for now: only B=1 + sampled_obs_B_T_dims=sample[Columns.OBS][0:1, t0:tH], + metrics_key=final_result_key_obs, symlog_obs=symlog_obs, ) # Reward MSE. _report_rewards( - results=results, - computed_rewards=dream_data["rewards_dreamed_t0_to_H_BxT"], - sampled_rewards=sample[Columns.REWARDS][:, t0 : tH + 1], - descr_prefix="EVALUATION", - descr_reward=f"dreamed_prior_H{dreamed_T}", + metrics=metrics, + computed_rewards=dream_data["rewards_dreamed_t0_to_H_Bx1"][0], + sampled_rewards=sample[Columns.REWARDS][:, t0:tH], + metrics_key=final_result_key_rew, ) # Continues MSE. _report_continues( - results=results, - computed_continues=dream_data["continues_dreamed_t0_to_H_BxT"], - sampled_continues=(1.0 - sample["is_terminated"])[:, t0 : tH + 1], - descr_prefix="EVALUATION", - descr_cont=f"dreamed_prior_H{dreamed_T}", + metrics=metrics, + computed_continues=dream_data["continues_dreamed_t0_to_H_Bx1"][0], + sampled_continues=(1.0 - sample["is_terminated"])[:, t0:tH], + metrics_key=final_result_key_cont, ) - return mse_sampled_vs_dreamed_obs -def report_sampling_and_replay_buffer(*, replay_buffer): +def report_sampling_and_replay_buffer(*, metrics, replay_buffer): episodes_in_buffer = replay_buffer.get_num_episodes() ts_in_buffer = replay_buffer.get_num_timesteps() replayed_steps = replay_buffer.get_sampled_timesteps() added_steps = replay_buffer.get_added_timesteps() # Summarize buffer, sampling, and train ratio stats. - return { - "BUFFER_capacity": replay_buffer.capacity, - "BUFFER_size_num_episodes": episodes_in_buffer, - "BUFFER_size_timesteps": ts_in_buffer, - "BUFFER_replayed_steps": replayed_steps, - "BUFFER_added_steps": added_steps, - } + metrics.log_dict( + { + "capacity": replay_buffer.capacity, + "size_num_episodes": episodes_in_buffer, + "size_timesteps": ts_in_buffer, + "replayed_steps": replayed_steps, + "added_steps": added_steps, + }, + key=REPLAY_BUFFER_RESULTS, + window=1, + ) # window=1 b/c these are current (total count/state) values. def _report_obs( *, - results, + metrics, computed_float_obs_B_T_dims, sampled_obs_B_T_dims, - descr_prefix=None, - descr_obs, + metrics_key, symlog_obs, ): """Summarizes computed- vs sampled observations: MSE and (if applicable) images. Args: + metrics: The MetricsLogger object of the DreamerV3 algo. computed_float_obs_B_T_dims: Computed float observations (not clipped, not cast'd). Shape=(B, T, [dims ...]). sampled_obs_B_T_dims: Sampled observations (as-is from the environment, meaning this could be uint8, 0-255 clipped images). Shape=(B, T, [dims ...]). - B: The batch size B (see shapes of `computed_float_obs_B_T_dims` and - `sampled_obs_B_T_dims` above). - T: The batch length T (see shapes of `computed_float_obs_B_T_dims` and - `sampled_obs_B_T_dims` above). - descr: A string used to describe the computed data to be used in the TB - summaries. + metrics_key: The metrics key (or key sequence) under which to log ths resulting + video sequence. + symlog_obs: Whether to inverse-symlog the computed observations or not. Set this + to True for environments, in which we should symlog the observations. + """ # Videos: Create summary, comparing computed images with actual sampled ones. # 4=[B, T, w, h] grayscale image; 5=[B, T, w, h, C] RGB image. if len(sampled_obs_B_T_dims.shape) in [4, 5]: - descr_prefix = (descr_prefix + "_") if descr_prefix else "" + # WandB videos need to be channels first. + transpose_axes = ( + (0, 1, 4, 2, 3) if len(sampled_obs_B_T_dims.shape) == 5 else (0, 3, 1, 2) + ) if symlog_obs: computed_float_obs_B_T_dims = inverse_symlog(computed_float_obs_B_T_dims) @@ -265,68 +335,63 @@ def _report_obs( sampled_obs_B_T_dims = np.clip(sampled_obs_B_T_dims, 0.0, 255.0).astype( np.uint8 ) + sampled_obs_B_T_dims = np.transpose(sampled_obs_B_T_dims, transpose_axes) computed_images = np.clip(computed_float_obs_B_T_dims, 0.0, 255.0).astype( np.uint8 ) + computed_images = np.transpose(computed_images, transpose_axes) # Concat sampled and computed images along the height axis (3) such that # real images show below respective predicted ones. # (B, T, C, h, w) sampled_vs_computed_images = np.concatenate( [computed_images, sampled_obs_B_T_dims], - axis=3, + axis=-1, # concat on width axis (looks nicer) ) # Add grayscale dim, if necessary. if len(sampled_obs_B_T_dims.shape) == 2 + 2: sampled_vs_computed_images = np.expand_dims(sampled_vs_computed_images, -1) - results.update( - {f"{descr_prefix}sampled_vs_{descr_obs}_videos": sampled_vs_computed_images} + metrics.log_value( + metrics_key, + sampled_vs_computed_images, + reduce=None, # No reduction, we want the obs tensor to stay in-tact. + window=1, ) - # return mse_sampled_vs_computed_obs - def _report_rewards( *, - results, + metrics, computed_rewards, sampled_rewards, - descr_prefix=None, - descr_reward, + metrics_key, ): - descr_prefix = (descr_prefix + "_") if descr_prefix else "" mse_sampled_vs_computed_rewards = np.mean( np.square(computed_rewards - sampled_rewards) ) mse_sampled_vs_computed_rewards = np.mean(mse_sampled_vs_computed_rewards) - results.update( - { - f"{descr_prefix}sampled_vs_{descr_reward}_rewards_mse": ( - mse_sampled_vs_computed_rewards - ), - } + metrics.log_value( + metrics_key, + mse_sampled_vs_computed_rewards, + window=1, ) def _report_continues( *, - results, + metrics, computed_continues, sampled_continues, - descr_prefix=None, - descr_cont, + metrics_key, ): - descr_prefix = (descr_prefix + "_") if descr_prefix else "" # Continue MSE. mse_sampled_vs_computed_continues = np.mean( np.square( computed_continues - sampled_continues.astype(computed_continues.dtype) ) ) - results.update( - { - f"{descr_prefix}sampled_vs_{descr_cont}_continues_mse": ( - mse_sampled_vs_computed_continues - ), - } + metrics.log_value( + metrics_key, + mse_sampled_vs_computed_continues, + window=1, ) diff --git a/rllib/algorithms/impala/impala.py b/rllib/algorithms/impala/impala.py index e1d414c3179b..b506dcf546aa 100644 --- a/rllib/algorithms/impala/impala.py +++ b/rllib/algorithms/impala/impala.py @@ -37,6 +37,7 @@ NUM_AGENT_STEPS_SAMPLED, NUM_AGENT_STEPS_TRAINED, NUM_ENV_STEPS_SAMPLED, + NUM_ENV_STEPS_SAMPLED_LIFETIME, NUM_ENV_STEPS_TRAINED, NUM_MODULE_STEPS_TRAINED, NUM_SYNCH_WORKER_WEIGHTS, @@ -720,6 +721,7 @@ def training_step(self) -> ResultDict: if self.config.enable_rl_module_and_learner: train_results = self.learn_on_processed_samples() module_ids_to_update = set(train_results.keys()) - {ALL_MODULES} + # TODO (sven): Move to Learner._after_gradient_based_update(). additional_results = self.learner_group.additional_update( module_ids_to_update=module_ids_to_update, timestep=self._counters[ @@ -959,6 +961,11 @@ def learn_on_processed_samples(self) -> ResultDict: for batch in batches: result = self.learner_group.update_from_batch( batch=batch, + timesteps={ + NUM_ENV_STEPS_SAMPLED_LIFETIME: ( + self.metrics.peek(NUM_ENV_STEPS_SAMPLED_LIFETIME) + ), + }, async_update=async_update, num_iters=self.config.num_sgd_iter, minibatch_size=self.config.minibatch_size, diff --git a/rllib/algorithms/ppo/ppo.py b/rllib/algorithms/ppo/ppo.py index ce491f9ca09d..60dfe4b6eed6 100644 --- a/rllib/algorithms/ppo/ppo.py +++ b/rllib/algorithms/ppo/ppo.py @@ -479,6 +479,11 @@ def _training_step_new_api_stack(self) -> ResultDict: with self.metrics.log_time((TIMERS, LEARNER_UPDATE_TIMER)): learner_results = self.learner_group.update_from_episodes( episodes=episodes, + timesteps={ + NUM_ENV_STEPS_SAMPLED_LIFETIME: ( + self.metrics.peek(NUM_ENV_STEPS_SAMPLED_LIFETIME) + ), + }, minibatch_size=( self.config.mini_batch_size_per_learner or self.config.sgd_minibatch_size @@ -515,7 +520,6 @@ def _training_step_new_api_stack(self) -> ResultDict: # Sync weights from learner_group to all rollout workers. from_worker_or_learner_group=self.learner_group, policies=modules_to_update, - global_vars=None, inference_only=True, ) else: @@ -542,7 +546,8 @@ def _training_step_new_api_stack(self) -> ResultDict: ) kl_dict[mid] = kl - # triggers a special update method on RLOptimizer to update the KL values. + # TODO (sven): Move to Learner._after_gradient_based_update(). + # Triggers a special update method on RLOptimizer to update the KL values. additional_results = self.learner_group.additional_update( module_ids_to_update=modules_to_update, sampled_kl_values=kl_dict, diff --git a/rllib/algorithms/ppo/tests/test_ppo_with_env_runner.py b/rllib/algorithms/ppo/tests/test_ppo_with_env_runner.py index f3defd5f7520..f7c89f167f8b 100644 --- a/rllib/algorithms/ppo/tests/test_ppo_with_env_runner.py +++ b/rllib/algorithms/ppo/tests/test_ppo_with_env_runner.py @@ -7,9 +7,7 @@ ) from ray.rllib.algorithms.callbacks import DefaultCallbacks from ray.rllib.core import DEFAULT_MODULE_ID -from ray.rllib.core.learner.learner import ( - LEARNER_RESULTS_CURR_LR_KEY, -) +from ray.rllib.core.learner.learner import DEFAULT_OPTIMIZER, LR_KEY from ray.rllib.utils.metrics import LEARNER_RESULTS from ray.rllib.utils.test_utils import ( @@ -44,7 +42,7 @@ def on_train_result(self, *, algorithm, result: dict, **kwargs): # Learning rate should decrease by 0.0001/4 per iteration. check( - stats[LEARNER_RESULTS_CURR_LR_KEY], + stats[DEFAULT_OPTIMIZER + "_" + LR_KEY], 0.0000075 if algorithm.iteration == 1 else 0.000005, ) # Compare reported curr lr vs the actual lr found in the optimizer object. @@ -54,7 +52,7 @@ def on_train_result(self, *, algorithm, result: dict, **kwargs): if algorithm.config.framework_str == "torch" else optim.lr ) - check(stats[LEARNER_RESULTS_CURR_LR_KEY], actual_optimizer_lr) + check(stats[DEFAULT_OPTIMIZER + "_" + LR_KEY], actual_optimizer_lr) class TestPPO(unittest.TestCase): diff --git a/rllib/algorithms/ppo/tests/test_ppo_with_rl_module.py b/rllib/algorithms/ppo/tests/test_ppo_with_rl_module.py index 7c89be47b189..4f9b32041829 100644 --- a/rllib/algorithms/ppo/tests/test_ppo_with_rl_module.py +++ b/rllib/algorithms/ppo/tests/test_ppo_with_rl_module.py @@ -10,9 +10,7 @@ from ray.rllib.algorithms.callbacks import DefaultCallbacks from ray.rllib.algorithms.ppo.tests.test_ppo import PENDULUM_FAKE_BATCH from ray.rllib.core import DEFAULT_MODULE_ID -from ray.rllib.core.learner.learner import ( - LEARNER_RESULTS_CURR_LR_KEY, -) +from ray.rllib.core.learner.learner import DEFAULT_OPTIMIZER, LR_KEY from ray.rllib.evaluation.postprocessing import ( compute_gae_for_sample_batch, ) @@ -50,7 +48,7 @@ def on_train_result(self, *, algorithm, result: dict, **kwargs): # Learning rate should decrease by 0.0001/4 per iteration. check( - stats[LEARNER_RESULTS_CURR_LR_KEY], + stats[DEFAULT_OPTIMIZER + "_" + LR_KEY], 0.0000075 if algorithm.iteration == 1 else 0.000005, ) # Compare reported curr lr vs the actual lr found in the optimizer object. @@ -60,7 +58,7 @@ def on_train_result(self, *, algorithm, result: dict, **kwargs): if algorithm.config.framework_str == "torch" else optim.lr ) - check(stats[LEARNER_RESULTS_CURR_LR_KEY], actual_optimizer_lr) + check(stats[DEFAULT_OPTIMIZER + "_" + LR_KEY], actual_optimizer_lr) class TestPPO(unittest.TestCase): diff --git a/rllib/core/learner/learner.py b/rllib/core/learner/learner.py index 887be4f2bf92..b71877fdf892 100644 --- a/rllib/core/learner/learner.py +++ b/rllib/core/learner/learner.py @@ -43,6 +43,7 @@ from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.metrics import ( ALL_MODULES, + NUM_ENV_STEPS_SAMPLED_LIFETIME, NUM_ENV_STEPS_TRAINED, NUM_MODULE_STEPS_TRAINED, ) @@ -85,7 +86,7 @@ ENTROPY_KEY = "entropy" # Additional update keys -LEARNER_RESULTS_CURR_LR_KEY = "curr_lr" +LR_KEY = "learning_rate" @dataclass @@ -287,6 +288,10 @@ def __init__( # and return the resulting (reduced) dict. self.metrics = MetricsLogger() + # TODO (sven): Do we really need this API? It seems like LearnerGroup constructs + # all Learner workers and then immediately builds them any ways? Seems to make + # thing more complicated. Unless there is a reason related to Train worker group + # setup. @OverrideToImplementCustomLogic_CallToSuperRecommended def build(self) -> None: """Builds the Learner. @@ -528,7 +533,6 @@ def postprocess_gradients_for_module( module_id: ModuleID, config: Optional["AlgorithmConfig"] = None, module_gradients_dict: ParamDict, - hps=None, ) -> ParamDict: """Applies postprocessing operations on the gradients of the given module. @@ -547,13 +551,6 @@ def postprocess_gradients_for_module( A dictionary with the updated gradients and the exact same (flat) structure as the incoming `module_gradients_dict` arg. """ - if hps is not None: - deprecation_warning( - old="Learner.postprocess_gradients_for_module(.., hps=..)", - help="Deprecated argument. Use `config` (AlgorithmConfig) instead.", - error=True, - ) - postprocessed_grads = {} if config.grad_clip is None: @@ -1029,7 +1026,6 @@ def additional_update_for_module( module_id: ModuleID, config: Optional["AlgorithmConfig"] = None, timestep: int, - hps=None, **kwargs, ) -> None: """Apply additional non-gradient based updates for a single module. @@ -1045,36 +1041,14 @@ def additional_update_for_module( Returns: A dictionary of results from the update """ - if hps is not None: - deprecation_warning( - old="Learner.additional_update_for_module(.., hps=..)", - help="Deprecated argument. Use `config` (AlgorithmConfig) instead.", - error=True, - ) - - # Only cover the optimizer mapped to this particular module. - for optimizer_name, optimizer in self.get_optimizers_for_module(module_id): - # Only update this optimizer's lr, if a scheduler has been registered - # along with it. - if optimizer in self._optimizer_lr_schedules: - new_lr = self._optimizer_lr_schedules[optimizer].update( - timestep=timestep - ) - self._set_optimizer_lr(optimizer, lr=new_lr) - - # Make sure our returned results differentiate by optimizer name - # (if not the default name). - stats_name = LEARNER_RESULTS_CURR_LR_KEY - if optimizer_name != DEFAULT_OPTIMIZER: - stats_name += "_" + optimizer_name - self.metrics.log_value( - key=(module_id, stats_name), value=new_lr, window=1 - ) + pass def update_from_batch( self, batch: MultiAgentBatch, *, + # TODO (sven): Make this a more formal structure with its own type. + timesteps: Optional[Dict[str, Any]] = None, # TODO (sven): Deprecate these in favor of config attributes for only those # algos that actually need (and know how) to do minibatching. minibatch_size: Optional[int] = None, @@ -1090,6 +1064,9 @@ def update_from_batch( Args: batch: A batch of training data to update from. + timesteps: Timesteps dict, which must have the key + `NUM_ENV_STEPS_SAMPLED_LIFETIME`. + # TODO (sven): Make this a more formal structure with its own type. minibatch_size: The size of the minibatch to use for each update. num_iters: The number of complete passes over all the sub-batches in the input multi-agent batch. @@ -1113,7 +1090,7 @@ def update_from_batch( ) return self._update_from_batch_or_episodes( batch=batch, - episodes=None, + timesteps=timesteps, minibatch_size=minibatch_size, num_iters=num_iters, ) @@ -1122,6 +1099,8 @@ def update_from_episodes( self, episodes: List[EpisodeType], *, + # TODO (sven): Make this a more formal structure with its own type. + timesteps: Optional[Dict[str, Any]] = None, # TODO (sven): Deprecate these in favor of config attributes for only those # algos that actually need (and know how) to do minibatching. minibatch_size: Optional[int] = None, @@ -1138,6 +1117,9 @@ def update_from_episodes( Args: episodes: An list of episode objects to update from. + timesteps: Timesteps dict, which must have the key + `NUM_ENV_STEPS_SAMPLED_LIFETIME`. + # TODO (sven): Make this a more formal structure with its own type. minibatch_size: The size of the minibatch to use for each update. num_iters: The number of complete passes over all the sub-batches in the input multi-agent batch. @@ -1167,8 +1149,8 @@ def update_from_episodes( error=True, ) return self._update_from_batch_or_episodes( - batch=None, episodes=episodes, + timesteps=timesteps, minibatch_size=minibatch_size, num_iters=num_iters, min_total_mini_batches=min_total_mini_batches, @@ -1270,17 +1252,16 @@ def _update_from_batch_or_episodes( # as well for simplicity. batch: Optional[MultiAgentBatch] = None, episodes: Optional[List[EpisodeType]] = None, + # TODO (sven): Make this a more formal structure with its own type. + timesteps: Optional[Dict[str, Any]] = None, # TODO (sven): Deprecate these in favor of config attributes for only those # algos that actually need (and know how) to do minibatching. minibatch_size: Optional[int] = None, num_iters: int = 1, min_total_mini_batches: int = 0, ) -> Union[Dict[str, Any], List[Dict[str, Any]]]: - self._check_is_built() - if num_iters < 1: - # We must do at least one pass on the batch for training. - raise ValueError("`num_iters` must be >= 1") + self._check_is_built() # Call the learner connector. if self._learner_connector is not None and episodes is not None: @@ -1310,6 +1291,7 @@ def _update_from_batch_or_episodes( f"Found IDs: {unknown_module_ids}" ) + # TODO: Move this into LearnerConnector pipeline? # Filter out those RLModules from the final train batch that should not be # updated. for module_id in list(batch.policy_batches.keys()): @@ -1326,14 +1308,10 @@ def _update_from_batch_or_episodes( { (ALL_MODULES, NUM_ENV_STEPS_TRAINED): batch.env_steps(), (ALL_MODULES, NUM_MODULE_STEPS_TRAINED): batch.agent_steps(), - }, - reduce="sum", - clear_on_reduce=True, - ) - self.metrics.log_dict( - { - (mid, NUM_MODULE_STEPS_TRAINED): len(b) - for mid, b in batch.policy_batches.items() + **{ + (mid, NUM_MODULE_STEPS_TRAINED): len(b) + for mid, b in batch.policy_batches.items() + }, }, reduce="sum", clear_on_reduce=True, @@ -1388,11 +1366,48 @@ def _update_from_batch_or_episodes( self._set_slicing_by_batch_id(batch, value=False) + # Call `_after_gradient_based_update` to allow for non-gradient based + # cleanups-, logging-, and update logic to happen. + self._after_gradient_based_update(timesteps) + + # Reduce results across all minibatch update steps. + return self.metrics.reduce() + + @OverrideToImplementCustomLogic_CallToSuperRecommended + def _after_gradient_based_update(self, timesteps: Dict[str, Any]) -> None: + """Called after gradient-based updates are completed. + + Should be overridden to implement custom cleanup-, logging-, or non-gradient- + based Learner/RLModule update logic after(!) gradient-based updates have been + completed. + + Args: + timesteps: Timesteps dict, which must have the key + `NUM_ENV_STEPS_SAMPLED_LIFETIME`. + # TODO (sven): Make this a more formal structure with its own type. + """ + timesteps = timesteps or {} + + # Only update this optimizer's lr, if a scheduler has been registered + # along with it. + for module_id, optimizer_names in self._module_optimizers.items(): + for optimizer_name in optimizer_names: + optimizer = self._named_optimizers[optimizer_name] + lr_schedule = self._optimizer_lr_schedules.get(optimizer) + if lr_schedule is None: + continue + new_lr = lr_schedule.update( + timestep=timesteps.get(NUM_ENV_STEPS_SAMPLED_LIFETIME, 0) + ) + self._set_optimizer_lr(optimizer, lr=new_lr) + # Log all current learning rates of all our optimizers (registered under the # different ModuleIDs). self.metrics.log_dict( { - (mid, f"{full_name[len(mid) + 1 :]}_lr"): convert_to_numpy( + # Cut out the module ID from the beginning since it's already part of + # the key sequence: (ModuleID, "[optim name]_lr"). + (mid, f"{full_name[len(mid) + 1:]}_{LR_KEY}"): convert_to_numpy( self._get_optimizer_lr(self._named_optimizers[full_name]) ) for mid, full_names in self._module_optimizers.items() @@ -1401,9 +1416,6 @@ def _update_from_batch_or_episodes( window=1, ) - # Reduce results across all minibatch update steps. - return self.metrics.reduce() - def _set_slicing_by_batch_id( self, batch: MultiAgentBatch, *, value: bool ) -> MultiAgentBatch: @@ -1456,7 +1468,6 @@ def _save_optimizers(self, path: Union[str, pathlib.Path]) -> None: Args: path: The path to the directory to save the state to. - """ pass @@ -1465,7 +1476,6 @@ def _load_optimizers(self, path: Union[str, pathlib.Path]) -> None: Args: path: The path to the directory to load the state from. - """ pass @@ -1491,7 +1501,6 @@ def save_state(self, path: Union[str, pathlib.Path]) -> None: Args: path: The path to the directory to save the state to. - """ self._check_is_built() path = pathlib.Path(path) diff --git a/rllib/core/learner/learner_group.py b/rllib/core/learner/learner_group.py index acc42ef83dea..5496cb0ce5ec 100644 --- a/rllib/core/learner/learner_group.py +++ b/rllib/core/learner/learner_group.py @@ -220,6 +220,7 @@ def update_from_batch( self, batch: MultiAgentBatch, *, + timesteps: Optional[Dict[str, Any]] = None, async_update: bool = False, # TODO (sven): Deprecate the following args. They should be extracted from # self.config of those specific algorithms that actually require these @@ -266,7 +267,7 @@ def update_from_batch( ) return self._update( batch=batch, - episodes=None, + timesteps=timesteps, async_update=async_update, minibatch_size=minibatch_size, num_iters=num_iters, @@ -276,6 +277,7 @@ def update_from_episodes( self, episodes: List[EpisodeType], *, + timesteps: Optional[Dict[str, Any]] = None, async_update: bool = False, # TODO (sven): Deprecate the following args. They should be extracted from # self.config of those specific algorithms that actually require these @@ -322,8 +324,8 @@ def update_from_episodes( ) return self._update( - batch=None, episodes=episodes, + timesteps=timesteps, async_update=async_update, minibatch_size=minibatch_size, num_iters=num_iters, @@ -334,6 +336,7 @@ def _update( *, batch: Optional[MultiAgentBatch] = None, episodes: Optional[List[EpisodeType]] = None, + timesteps: Optional[Dict[str, Any]] = None, async_update: bool = False, minibatch_size: Optional[int] = None, num_iters: int = 1, @@ -342,22 +345,25 @@ def _update( # Define function to be called on all Learner actors (or the local learner). def _learner_update( learner: Learner, - batch_shard=None, - episodes_shard=None, - min_total_mini_batches=0, + _batch_shard=None, + _episodes_shard=None, + _timesteps=None, + _min_total_mini_batches=0, ): - if batch_shard is not None: + if _batch_shard is not None: return learner.update_from_batch( - batch=batch_shard, + batch=_batch_shard, + timesteps=_timesteps, minibatch_size=minibatch_size, num_iters=num_iters, ) else: return learner.update_from_episodes( - episodes=episodes_shard, + episodes=_episodes_shard, + timesteps=_timesteps, minibatch_size=minibatch_size, num_iters=num_iters, - min_total_mini_batches=min_total_mini_batches, + min_total_mini_batches=_min_total_mini_batches, ) # Local Learner worker: Don't shard batch/episodes, just run data as-is through @@ -372,8 +378,9 @@ def _learner_update( results = [ _learner_update( learner=self._learner, - batch_shard=batch, - episodes_shard=episodes, + _batch_shard=batch, + _episodes_shard=episodes, + _timesteps=timesteps, ) ] # One or more remote Learners: Shard batch/episodes into equal pieces (roughly @@ -387,7 +394,9 @@ def _learner_update( # "lockstep"), the `ShardBatchIterator` should not be used. if episodes is None: partials = [ - partial(_learner_update, batch_shard=batch_shard) + partial( + _learner_update, _batch_shard=batch_shard, _timesteps=timesteps + ) for batch_shard in ShardBatchIterator(batch, len(self._workers)) ] # Single- or MultiAgentEpisodes: Shard into equal pieces (only roughly equal @@ -424,8 +433,9 @@ def _learner_update( partials = [ partial( _learner_update, - episodes_shard=eps_shard, - min_total_mini_batches=min_total_mini_batches, + _episodes_shard=eps_shard, + _timesteps=timesteps, + _min_total_mini_batches=min_total_mini_batches, ) for eps_shard in eps_shards ] diff --git a/rllib/core/learner/torch/torch_learner.py b/rllib/core/learner/torch/torch_learner.py index cacce6f90b06..6dc586c12077 100644 --- a/rllib/core/learner/torch/torch_learner.py +++ b/rllib/core/learner/torch/torch_learner.py @@ -136,7 +136,7 @@ def compute_gradients( self, loss_per_module: Dict[ModuleID, TensorType], **kwargs ) -> ParamDict: for optim in self._optimizer_parameters: - # set_to_none is a faster way to zero out the gradients + # `set_to_none=True` is a faster way to zero out the gradients. optim.zero_grad(set_to_none=True) loss_per_module[ALL_MODULES].backward() grads = {pid: p.grad for pid, p in self._params.items()} diff --git a/rllib/core/models/torch/heads.py b/rllib/core/models/torch/heads.py index 95f61e58c4fd..d634ff2ef4c8 100644 --- a/rllib/core/models/torch/heads.py +++ b/rllib/core/models/torch/heads.py @@ -218,7 +218,7 @@ def __init__(self, config: CNNTransposeHeadConfig) -> None: if initial_dense_weights_initializer: initial_dense_weights_initializer( self.initial_dense.weight, - **config.initial_dense_initializer_config or {}, + **config.initial_dense_weights_initializer_config or {}, ) # Initialized dense layer bais, if necessary. if initial_dense_bias_initializer: diff --git a/rllib/core/models/torch/primitives.py b/rllib/core/models/torch/primitives.py index a8b8d9b27fd2..0be093730279 100644 --- a/rllib/core/models/torch/primitives.py +++ b/rllib/core/models/torch/primitives.py @@ -161,7 +161,8 @@ def __init__( # Insert a layer normalization in between layer's output and # the activation. if hidden_layer_use_layernorm: - layers.append(nn.LayerNorm(dims[i + 1])) + # We use an epsilon of 0.001 here to mimick the Tf default behavior. + layers.append(nn.LayerNorm(dims[i + 1], eps=0.001)) # Add the activation function. if hidden_activation is not None: layers.append(hidden_activation()) @@ -294,7 +295,8 @@ def __init__( # Layernorm. if cnn_use_layernorm: - layers.append(nn.LayerNorm((out_depth, out_size[0], out_size[1]))) + # We use an epsilon of 0.001 here to mimick the Tf default behavior. + layers.append(LayerNorm1D(out_depth, eps=0.001)) # Activation. if cnn_activation is not None: layers.append(cnn_activation()) @@ -446,7 +448,7 @@ def __init__( layers.append(layer) # Layernorm (never for final layer). if cnn_transpose_use_layernorm and not is_final_layer: - layers.append(nn.LayerNorm((out_depth, out_size[0], out_size[1]))) + layers.append(LayerNorm1D(out_depth, eps=0.001)) # Last layer is never activated (regardless of config). if cnn_transpose_activation is not None and not is_final_layer: layers.append(cnn_transpose_activation()) @@ -464,3 +466,20 @@ def forward(self, inputs): out = inputs.permute(0, 3, 1, 2) out = self.cnn_transpose(out.type(self.expected_input_dtype)) return out.permute(0, 2, 3, 1) + + +class LayerNorm1D(nn.Module): + def __init__(self, num_features, **kwargs): + super().__init__() + self.layer_norm = nn.LayerNorm(num_features, **kwargs) + + def forward(self, x): + # x shape: (B, dim, dim, channels). + batch_size, channels, h, w = x.size() + # Reshape to (batch_size * height * width, channels) for LayerNorm + x = x.permute(0, 2, 3, 1).reshape(-1, channels) + # Apply LayerNorm + x = self.layer_norm(x) + # Reshape back to (batch_size, dim, dim, channels) + x = x.reshape(batch_size, h, w, channels).permute(0, 3, 1, 2) + return x diff --git a/rllib/core/models/torch/utils.py b/rllib/core/models/torch/utils.py index f9da0adab31a..1bdbdef016f4 100644 --- a/rllib/core/models/torch/utils.py +++ b/rllib/core/models/torch/utils.py @@ -45,28 +45,33 @@ def __init__(self, width, height, stride_w, stride_h): self.stride_w = stride_w self.stride_h = stride_h - self.zeros = torch.zeros( - size=( - self.width * self.stride_w - (self.stride_w - 1), - self.height * self.stride_h - (self.stride_h - 1), + self.register_buffer( + "zeros", + torch.zeros( + size=( + self.width * self.stride_w - (self.stride_w - 1), + self.height * self.stride_h - (self.stride_h - 1), + ), + dtype=torch.float32, ), - dtype=torch.float32, ) + self.out_width, self.out_height = self.zeros.shape[0], self.zeros.shape[1] # Squeeze in batch and channel dims. self.zeros = self.zeros.unsqueeze(0).unsqueeze(0) - self.where_template = torch.zeros( + where_template = torch.zeros( (self.stride_w, self.stride_h), dtype=torch.float32 ) # Set upper/left corner to 1.0. - self.where_template[0][0] = 1.0 + where_template[0][0] = 1.0 # then tile across the entire (strided) image size. - self.where_template = self.where_template.repeat((self.height, self.width))[ + where_template = where_template.repeat((self.height, self.width))[ : -(self.stride_w - 1), : -(self.stride_h - 1) ] # Squeeze in batch and channel dims and convert to bool. - self.where_template = self.where_template.unsqueeze(0).unsqueeze(0).bool() + where_template = where_template.unsqueeze(0).unsqueeze(0).bool() + self.register_buffer("where_template", where_template) def forward(self, x): # Repeat incoming image stride(w/h) times to match the strided output template. diff --git a/rllib/core/rl_module/torch/torch_rl_module.py b/rllib/core/rl_module/torch/torch_rl_module.py index 9cb4d2bda6c4..0ced41878552 100644 --- a/rllib/core/rl_module/torch/torch_rl_module.py +++ b/rllib/core/rl_module/torch/torch_rl_module.py @@ -110,7 +110,8 @@ def get_state(self, inference_only: bool = False) -> Mapping[str, Any]: @override(RLModule) def set_state(self, state_dict: Mapping[str, Any]) -> None: - self.load_state_dict(convert_to_torch_tensor(state_dict)) + state_dict = convert_to_torch_tensor(state_dict) + self.load_state_dict(state_dict) def _module_state_file_name(self) -> pathlib.Path: return pathlib.Path("module_state.pt") diff --git a/rllib/examples/catalogs/mobilenet_v2_encoder.py b/rllib/examples/catalogs/mobilenet_v2_encoder.py index beebdb79f773..ca44215b8bef 100644 --- a/rllib/examples/catalogs/mobilenet_v2_encoder.py +++ b/rllib/examples/catalogs/mobilenet_v2_encoder.py @@ -44,7 +44,10 @@ def _get_encoder_config( # Create a generic config with our enhanced Catalog ppo_config = ( PPOConfig() - .api_stack(enable_rl_module_and_learner=True) + .api_stack( + enable_rl_module_and_learner=True, + enable_env_runner_and_connector_v2=True, + ) .rl_module( rl_module_spec=SingleAgentRLModuleSpec( catalog_class=MobileNetEnhancedPPOCatalog diff --git a/rllib/examples/evaluation/custom_evaluation.py b/rllib/examples/evaluation/custom_evaluation.py index d396ffee04df..76aad3eccdf4 100644 --- a/rllib/examples/evaluation/custom_evaluation.py +++ b/rllib/examples/evaluation/custom_evaluation.py @@ -149,6 +149,7 @@ def custom_eval_function( eval_results = algorithm.metrics.reduce( key=(EVALUATION_RESULTS, ENV_RUNNER_RESULTS) ) + # Alternatively, you could manually reduce over the n returned `env_runner_metrics` # dicts, but this would be much harder as you might not know, which metrics # to sum up, which ones to average over, etc.. diff --git a/rllib/execution/rollout_ops.py b/rllib/execution/rollout_ops.py index 4b3582b9968b..a42a1991453d 100644 --- a/rllib/execution/rollout_ops.py +++ b/rllib/execution/rollout_ops.py @@ -24,6 +24,7 @@ def synchronous_parallel_sample( max_env_steps: Optional[int] = None, concat: bool = True, sample_timeout_s: Optional[float] = 60.0, + random_actions: bool = False, _uses_new_env_runners: bool = False, _return_metrics: bool = False, ) -> Union[List[SampleBatchType], SampleBatchType, List[EpisodeType], EpisodeType]: @@ -81,6 +82,8 @@ def synchronous_parallel_sample( sample_batches_or_episodes = [] all_stats_dicts = [] + random_action_kwargs = {} if not random_actions else {"random_actions": True} + # Stop collecting batches as soon as one criterium is met. while (max_agent_or_env_steps is None and agent_or_env_steps == 0) or ( max_agent_or_env_steps is not None @@ -89,16 +92,16 @@ def synchronous_parallel_sample( # No remote workers in the set -> Use local worker for collecting # samples. if worker_set.num_remote_workers() <= 0: - sampled_data = [worker_set.local_worker().sample()] + sampled_data = [worker_set.local_worker().sample(**random_action_kwargs)] if _return_metrics: stats_dicts = [worker_set.local_worker().get_metrics()] # Loop over remote workers' `sample()` method in parallel. else: sampled_data = worker_set.foreach_worker( ( - (lambda w: w.sample()) + (lambda w: w.sample(**random_action_kwargs)) if not _return_metrics - else (lambda w: (w.sample(), w.get_metrics())) + else (lambda w: (w.sample(**random_action_kwargs), w.get_metrics())) ), local_worker=False, timeout_seconds=sample_timeout_s, diff --git a/rllib/tuned_examples/dreamerv3/atari_100k.py b/rllib/tuned_examples/dreamerv3/atari_100k.py index 23a46fcbf3e7..68bdc9745136 100644 --- a/rllib/tuned_examples/dreamerv3/atari_100k.py +++ b/rllib/tuned_examples/dreamerv3/atari_100k.py @@ -9,17 +9,27 @@ """ # Run with: -# python run_regression_tests.py --dir [this file] --env ALE/[gym ID e.g. Pong-v5] +# python [this script name].py --env ALE/[gym ID e.g. Pong-v5] -from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config +# To see all available options: +# python [this script name].py --help +from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config +from ray.rllib.utils.test_utils import add_rllib_example_script_args -# Number of GPUs to run on. -num_gpus = 1 +parser = add_rllib_example_script_args( + default_iters=1000000, + default_reward=20.0, + default_timesteps=1000000, +) +# Use `parser` to add your own custom command line options to this script +# and (if needed) use their values toset up `config` below. +args = parser.parse_args() config = ( DreamerV3Config() .environment( + env=args.env, # [2]: "We follow the evaluation protocol of Machado et al. (2018) with 200M # environment steps, action repeat of 4, a time limit of 108,000 steps per # episode that correspond to 30 minutes of game play, no access to life @@ -34,31 +44,35 @@ "full_action_space": False, # Already done by MaxAndSkip wrapper: "action repeat" == 4. "frameskip": 1, - } - ) - .resources( - num_cpus_for_main_process=1, - ) - .learners( - num_learners=0 if num_gpus == 1 else num_gpus, - num_gpus_per_learner=1 if num_gpus else 0, + }, ) .env_runners( + num_env_runners=(args.num_env_runners or 0), # If we use >1 GPU and increase the batch size accordingly, we should also # increase the number of envs per worker. - num_envs_per_env_runner=(num_gpus or 1), - remote_worker_envs=True, + num_envs_per_env_runner=(args.num_gpus or 1), + remote_worker_envs=(args.num_gpus > 1), + ) + .learners( + num_learners=0 if args.num_gpus == 1 else args.num_gpus, + num_gpus_per_learner=1 if args.num_gpus else 0, ) .reporting( - metrics_num_episodes_for_smoothing=(num_gpus or 1), - report_images_and_videos=False, - report_dream_data=False, + metrics_num_episodes_for_smoothing=(args.num_gpus or 1), + report_images_and_videos=True, + report_dream_data=True, report_individual_batch_item_stats=False, ) # See Appendix A. .training( model_size="S", training_ratio=1024, - batch_size_B=16 * (num_gpus or 1), + batch_size_B=16 * (args.num_gpus or 1), ) ) + + +if __name__ == "__main__": + from ray.rllib.utils.test_utils import run_rllib_example_script_experiment + + run_rllib_example_script_experiment(config, args, stop={}, keep_config=True) diff --git a/rllib/tuned_examples/dreamerv3/atari_200M.py b/rllib/tuned_examples/dreamerv3/atari_200M.py index 2fb1e48b0929..2339d345d2f8 100644 --- a/rllib/tuned_examples/dreamerv3/atari_200M.py +++ b/rllib/tuned_examples/dreamerv3/atari_200M.py @@ -9,13 +9,22 @@ """ # Run with: -# python run_regression_tests.py --dir [this file] --env ALE/[gym ID e.g. Pong-v5] +# python [this script name].py --env ALE/[gym ID e.g. Pong-v5] -from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config +# To see all available options: +# python [this script name].py --help +from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config +from ray.rllib.utils.test_utils import add_rllib_example_script_args -# Number of GPUs to run on. -num_gpus = 1 +parser = add_rllib_example_script_args( + default_iters=1000000, + default_reward=20.0, + default_timesteps=1000000, +) +# Use `parser` to add your own custom command line options to this script +# and (if needed) use their values toset up `config` below. +args = parser.parse_args() config = ( DreamerV3Config() @@ -23,19 +32,10 @@ # For each (parallelized) env, we should provide a CPU. Lower this number # if you don't have enough CPUs. num_cpus_for_main_process=8 - * (num_gpus or 1), - ) - .learners( - num_learners=0 if num_gpus == 1 else num_gpus, - num_gpus_per_learner=1 if num_gpus else 0, - ) - .env_runners( - # If we use >1 GPU and increase the batch size accordingly, we should also - # increase the number of envs per worker. - num_envs_per_env_runner=8 * (num_gpus or 1), - remote_worker_envs=True, + * (args.num_gpus or 1), ) .environment( + env=args.env, # [2]: "We follow the evaluation protocol of Machado et al. (2018) with 200M # environment steps, action repeat of 4, a time limit of 108,000 steps per # episode that correspond to 30 minutes of game play, no access to life @@ -50,10 +50,21 @@ "full_action_space": False, # Already done by MaxAndSkip wrapper: "action repeat" == 4. "frameskip": 1, - } + }, + ) + .env_runners( + num_env_runners=(args.num_env_runners or 0), + # If we use >1 GPU and increase the batch size accordingly, we should also + # increase the number of envs per worker. + num_envs_per_env_runner=8 * (args.num_gpus or 1), + remote_worker_envs=True, + ) + .learners( + num_learners=0 if args.num_gpus == 1 else args.num_gpus, + num_gpus_per_learner=1 if args.num_gpus else 0, ) .reporting( - metrics_num_episodes_for_smoothing=(num_gpus or 1), + metrics_num_episodes_for_smoothing=(args.num_gpus or 1), report_images_and_videos=False, report_dream_data=False, report_individual_batch_item_stats=False, @@ -62,6 +73,12 @@ .training( model_size="XL", training_ratio=64, - batch_size_B=16 * (num_gpus or 1), + batch_size_B=16 * (args.num_gpus or 1), ) ) + + +if __name__ == "__main__": + from ray.rllib.utils.test_utils import run_rllib_example_script_experiment + + run_rllib_example_script_experiment(config, args, keep_config=True) diff --git a/rllib/tuned_examples/dreamerv3/dm_control_suite_vision.py b/rllib/tuned_examples/dreamerv3/dm_control_suite_vision.py index b201900da5f6..21c1a435a034 100644 --- a/rllib/tuned_examples/dreamerv3/dm_control_suite_vision.py +++ b/rllib/tuned_examples/dreamerv3/dm_control_suite_vision.py @@ -7,30 +7,45 @@ D. Hafner, T. Lillicrap, M. Norouzi, J. Ba https://arxiv.org/pdf/2010.02193.pdf """ + # Run with: -# python run_regression_tests.py --dir [this file] --env DMC/[task]/[domain] -# e.g. --env=DMC/cartpole/swingup +# python [this script name].py --env DMC/[task]/[domain] (e.g. DMC/cartpole/swingup) -from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config +# To see all available options: +# python [this script name].py --help +from ray.rllib.algorithms.dreamerv3.dreamerv3 import DreamerV3Config +from ray.rllib.utils.test_utils import add_rllib_example_script_args -# Number of GPUs to run on. -num_gpus = 1 +parser = add_rllib_example_script_args( + default_iters=1000000, + default_reward=800.0, + default_timesteps=1000000, +) +# Use `parser` to add your own custom command line options to this script +# and (if needed) use their values toset up `config` below. +args = parser.parse_args() config = ( DreamerV3Config() # Use image observations. - .environment(env_config={"from_pixels": True}) - .resources( - num_cpus_for_main_process=1, + .environment( + env=args.env, + env_config={"from_pixels": True}, ) .learners( - num_learners=0 if num_gpus == 1 else num_gpus, - num_gpus_per_learner=1 if num_gpus else 0, + num_learners=0 if args.num_gpus == 1 else args.num_gpus, + num_gpus_per_learner=1 if args.num_gpus else 0, + ) + .env_runners( + num_env_runners=(args.num_env_runners or 0), + # If we use >1 GPU and increase the batch size accordingly, we should also + # increase the number of envs per worker. + num_envs_per_env_runner=4 * (args.num_gpus or 1), + remote_worker_envs=True, ) - .env_runners(num_envs_per_env_runner=4 * (num_gpus or 1), remote_worker_envs=True) .reporting( - metrics_num_episodes_for_smoothing=(num_gpus or 1), + metrics_num_episodes_for_smoothing=(args.num_gpus or 1), report_images_and_videos=False, report_dream_data=False, report_individual_batch_item_stats=False, @@ -39,6 +54,6 @@ .training( model_size="S", training_ratio=512, - batch_size_B=16 * (num_gpus or 1), + batch_size_B=16 * (args.num_gpus or 1), ) ) diff --git a/rllib/utils/metrics/__init__.py b/rllib/utils/metrics/__init__.py index 39e087da9434..764edabdb8a2 100644 --- a/rllib/utils/metrics/__init__.py +++ b/rllib/utils/metrics/__init__.py @@ -1,6 +1,7 @@ # Algorithm ResultDict keys. EVALUATION_RESULTS = "evaluation" ENV_RUNNER_RESULTS = "env_runners" +REPLAY_BUFFER_RESULTS = "replay_buffer" LEARNER_RESULTS = "learners" FAULT_TOLERANCE_STATS = "fault_tolerance" TIMERS = "timers" diff --git a/rllib/utils/metrics/metrics_logger.py b/rllib/utils/metrics/metrics_logger.py index 8de81deb0038..611d6f7d9b8c 100644 --- a/rllib/utils/metrics/metrics_logger.py +++ b/rllib/utils/metrics/metrics_logger.py @@ -596,9 +596,9 @@ def deactivate_tensor_mode(self): def tensors_to_numpy(self, tensor_metrics): """Converts all previously logged and returned tensors back to numpy values.""" - for key, value in tensor_metrics.items(): + for key, values in tensor_metrics.items(): assert self._key_in_stats(key) - self._get_key(key).numpy(value) + self._get_key(key).set_to_numpy_values(values) @property def tensor_mode(self): @@ -727,6 +727,19 @@ def set_value( clear_on_reduce=clear_on_reduce, ) + def delete(self, *key: Tuple[str], key_error: bool = True) -> None: + """Deletes th egiven `key` from this metrics logger's stats. + + Args: + key: The key or key sequence (for nested location within self.stats), + to delete from this MetricsLogger's stats. + key_error: Whether to throw a KeyError if `key` cannot be found in `self`. + + Raises: + KeyError: If `key` cannot be found in `self` AND `key_error` is True. + """ + self._del_key(key, key_error) + def reduce( self, key: Optional[Union[str, Tuple[str]]] = None, @@ -894,6 +907,19 @@ def _set_key(self, flat_key, stats): _dict[key] = {} _dict = _dict[key] + def _del_key(self, flat_key, key_error=False): + flat_key = force_tuple(tree.flatten(flat_key)) + _dict = self.stats + try: + for i, key in enumerate(flat_key): + if i == len(flat_key) - 1: + del _dict[key] + return + _dict = _dict[key] + except KeyError as e: + if key_error: + raise e + @Deprecated(new="MetricsLogger.merge_and_log_n_dicts()", error=True) def log_n_dicts(self, *args, **kwargs): pass diff --git a/rllib/utils/metrics/stats.py b/rllib/utils/metrics/stats.py index eec5845fd1a9..5a7358fde45e 100644 --- a/rllib/utils/metrics/stats.py +++ b/rllib/utils/metrics/stats.py @@ -472,18 +472,19 @@ def merge_in_parallel(self, *others: "Stats") -> None: self.values = list(reversed(new_values)) - def numpy(self, value: Any = None) -> "Stats": - """Converts all of self's internal values to numpy (if a tensor).""" - if value is not None: - if self._reduce_method is None: - assert isinstance(value, list) and len(self.values) >= len(value) - self.values = convert_to_numpy(value) - else: - assert len(self.values) > 0 - self.values = [convert_to_numpy(value)] + def set_to_numpy_values(self, values) -> None: + """Converts `self.values` from tensors to actual numpy values. + + Args: + values: The (numpy) values to set `self.values` to. + """ + numpy_values = convert_to_numpy(values) + if self._reduce_method is None: + assert isinstance(values, list) and len(self.values) >= len(values) + self.values = numpy_values else: - self.values = convert_to_numpy(self.values) - return self + assert len(self.values) > 0 + self.values = [numpy_values] def __len__(self) -> int: """Returns the length of the internal values list.""" @@ -613,8 +614,21 @@ def _reduced_values(self, values=None, window=None) -> Tuple[Any, Any]: reduce_in = reduce_in.float() reduced = reduce_meth(reduce_in) elif tf and tf.is_tensor(values[0]): - reduce_meth = getattr(tf, "reduce_" + self._reduce_method) - reduced = reduce_meth(values) + # TODO (sven): Currently, tensor metrics only work with window=1. + # We might want o enforce it more formally, b/c it's probably not a + # good idea to have MetricsLogger or Stats tinker with the actual + # computation graph that users are trying to build in their loss + # functions. + assert len(values) == 1 + # TODO (sven) If the shape is (), do NOT even use the reduce method. + # Using `tf.reduce_mean()` here actually lead to a completely broken + # DreamerV3 (for a still unknown exact reason). + if len(values[0].shape) == 0: + reduced = values[0] + else: + reduce_meth = getattr(tf, "reduce_" + self._reduce_method) + reduced = reduce_meth(values) + else: reduce_meth = getattr(np, "nan" + self._reduce_method) reduced = reduce_meth(values) From c94d8c3d2b2e8324005110023188837a07bb9651 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Fri, 31 May 2024 11:33:31 -0700 Subject: [PATCH 61/65] [ci][microcheck] add a few cheap tests to microcheck (#45657) Add keys to a few cheap builds and tests that I noticed failed on people's PR so we can include them in microcheck. These tests are not covered in the scope of test_in_docker. Test: - CI Signed-off-by: can --- .buildkite/build.rayci.yml | 1 + .buildkite/kuberay.rayci.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.buildkite/build.rayci.yml b/.buildkite/build.rayci.yml index 06e96da332f8..582f0d497b87 100644 --- a/.buildkite/build.rayci.yml +++ b/.buildkite/build.rayci.yml @@ -28,6 +28,7 @@ steps: - forge - label: ":tapioca: build: jar" + key: java_wheels tags: - java - oss diff --git a/.buildkite/kuberay.rayci.yml b/.buildkite/kuberay.rayci.yml index cbdbab5219ee..655a2a573f64 100644 --- a/.buildkite/kuberay.rayci.yml +++ b/.buildkite/kuberay.rayci.yml @@ -20,6 +20,7 @@ steps: - raycpubase - label: ":kubernetes: chaos {{matrix.workload}} under {{matrix.fault}}" + key: kuberay_tests tags: - python - docker From a30630a7451d9cc458f1973def33349468120608 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Fri, 31 May 2024 13:20:26 -0700 Subject: [PATCH 62/65] [Core] Ray c++ backend structured logging (#44468) Signed-off-by: Jiajun Yao --- src/ray/gcs/gcs_server/gcs_actor_manager.cc | 6 +- src/ray/raylet/node_manager.cc | 2 +- src/ray/util/logging.cc | 186 ++++++++++---------- src/ray/util/logging.h | 125 ++++++++----- src/ray/util/tests/logging_test.cc | 62 ++++++- 5 files changed, 237 insertions(+), 144 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.cc b/src/ray/gcs/gcs_server/gcs_actor_manager.cc index 33183a0489a4..008b6c13e3a2 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.cc @@ -259,8 +259,10 @@ void GcsActorManager::HandleRegisterActor(rpc::RegisterActorRequest request, auto actor_id = ActorID::FromBinary(request.task_spec().actor_creation_task_spec().actor_id()); - RAY_LOG(INFO) << "Registering actor, job id = " << actor_id.JobId() - << ", actor id = " << actor_id; + RAY_LOG(INFO) + .WithField(kLogKeyJobID, actor_id.JobId()) + .WithField(kLogKeyActorID, actor_id) + << "Registering actor"; Status status = RegisterActor(request, [reply, send_reply_callback, actor_id]( diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index a4be13e6b533..29099231237d 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -284,7 +284,7 @@ NodeManager::NodeManager( RayConfig::instance().min_memory_free_bytes(), RayConfig::instance().memory_monitor_refresh_ms(), CreateMemoryUsageRefreshCallback())) { - RAY_LOG(INFO) << "Initializing NodeManager with ID " << self_node_id_; + RAY_LOG(INFO).WithField(kLogKeyNodeID, self_node_id_) << "Initializing NodeManager"; cluster_resource_scheduler_ = std::make_shared( io_service, scheduling::NodeID(self_node_id_.Binary()), diff --git a/src/ray/util/logging.cc b/src/ray/util/logging.cc index fdcf5d22b56c..5e3d55e46e79 100644 --- a/src/ray/util/logging.cc +++ b/src/ray/util/logging.cc @@ -36,6 +36,7 @@ #include "absl/debugging/stacktrace.h" #include "absl/debugging/symbolize.h" #include "absl/strings/str_format.h" +#include "nlohmann/json.hpp" #include "ray/util/event_label.h" #include "ray/util/filesystem.h" #include "ray/util/util.h" @@ -46,12 +47,19 @@ namespace ray { +// Format pattern is 2020-08-21 17:00:00,000 I 100 1001 msg. +// %L is loglevel, %P is process id, %t for thread id. +constexpr char kLogFormatTextPattern[] = "[%Y-%m-%d %H:%M:%S,%e %L %P %t] %v"; +constexpr char kLogFormatJsonPattern[] = + "{\"asctime\":\"%Y-%m-%d %H:%M:%S,%e\",\"levelname\":\"%L\"%v}"; + RayLogLevel RayLog::severity_threshold_ = RayLogLevel::INFO; std::string RayLog::app_name_ = ""; +std::string RayLog::component_name_ = ""; std::string RayLog::log_dir_ = ""; -// Format pattern is 2020-08-21 17:00:00,000 I 100 1001 msg. -// %L is loglevel, %P is process id, %t for thread id. -std::string RayLog::log_format_pattern_ = "[%Y-%m-%d %H:%M:%S,%e %L %P %t] %v"; +bool RayLog::log_format_json_ = false; +std::string RayLog::log_format_pattern_ = kLogFormatTextPattern; + std::string RayLog::logger_name_ = "ray_log_sink"; long RayLog::log_rotation_max_size_ = 1 << 29; long RayLog::log_rotation_file_num_ = 10; @@ -141,51 +149,8 @@ class DefaultStdErrLogger final { std::shared_ptr default_stderr_logger_; }; -class SpdLogMessage final { - public: - explicit SpdLogMessage(const char *file, - int line, - int loglevel, - std::shared_ptr expose_osstream) - : loglevel_(loglevel), expose_osstream_(expose_osstream) { - stream() << ConstBasename(file) << ":" << line << ": "; - } - - inline void Flush() { - auto logger = spdlog::get(RayLog::GetLoggerName()); - if (!logger) { - logger = DefaultStdErrLogger::Instance().GetDefaultLogger(); - } - - if (loglevel_ == static_cast(spdlog::level::critical)) { - stream() << "\n*** StackTrace Information ***\n" << ray::StackTrace(); - } - if (expose_osstream_) { - *expose_osstream_ << "\n*** StackTrace Information ***\n" << ray::StackTrace(); - } - // NOTE(lingxuan.zlx): See more fmt by visiting https://github.com/fmtlib/fmt. - logger->log( - static_cast(loglevel_), /*fmt*/ "{}", str_.str()); - logger->flush(); - } - - ~SpdLogMessage() { Flush(); } - inline std::ostream &stream() { return str_; } - - private: - SpdLogMessage(const SpdLogMessage &) = delete; - SpdLogMessage &operator=(const SpdLogMessage &) = delete; - - private: - std::ostringstream str_; - int loglevel_; - std::shared_ptr expose_osstream_; -}; - -typedef ray::SpdLogMessage LoggingProvider; - // Spdlog's severity map. -static int GetMappedSeverity(RayLogLevel severity) { +static spdlog::level::level_enum GetMappedSeverity(RayLogLevel severity) { switch (severity) { case RayLogLevel::TRACE: return spdlog::level::trace; @@ -208,9 +173,7 @@ static int GetMappedSeverity(RayLogLevel severity) { std::vector RayLog::fatal_log_callbacks_; -void RayLog::StartRayLog(const std::string &app_name, - RayLogLevel severity_threshold, - const std::string &log_dir) { +void RayLog::InitSeverityThreshold(RayLogLevel severity_threshold) { const char *var_value = std::getenv("RAY_BACKEND_LOG_LEVEL"); if (var_value != nullptr) { std::string data = var_value; @@ -234,12 +197,35 @@ void RayLog::StartRayLog(const std::string &app_name, << " to " << static_cast(severity_threshold); } severity_threshold_ = severity_threshold; +} + +void RayLog::InitLogFormat() { + // Default is plain text + log_format_json_ = false; + log_format_pattern_ = kLogFormatTextPattern; + + const char *var_value = std::getenv("RAY_BACKEND_LOG_JSON"); + if (var_value != nullptr) { + std::string data = var_value; + if (data == "1") { + log_format_json_ = true; + log_format_pattern_ = kLogFormatJsonPattern; + } + } +} + +void RayLog::StartRayLog(const std::string &app_name, + RayLogLevel severity_threshold, + const std::string &log_dir) { + InitSeverityThreshold(severity_threshold); + InitLogFormat(); + app_name_ = app_name; log_dir_ = log_dir; // All the logging sinks to add. std::vector sinks; - auto level = static_cast(severity_threshold_); + auto level = GetMappedSeverity(severity_threshold_); std::string app_name_without_path = app_name; if (app_name.empty()) { app_name_without_path = "DefaultApp"; @@ -274,8 +260,6 @@ void RayLog::StartRayLog(const std::string &app_name, log_rotation_file_num_ = file_num; } } - spdlog::set_pattern(log_format_pattern_); - spdlog::set_level(static_cast(severity_threshold_)); // Sink all log stuff to default file logger we defined here. We may need // multiple sinks for different files or loglevel. auto file_logger = spdlog::get(RayLog::GetLoggerName()); @@ -288,14 +272,11 @@ void RayLog::StartRayLog(const std::string &app_name, JoinPaths(log_dir_, app_name_without_path + "_" + std::to_string(pid) + ".log"), log_rotation_max_size_, log_rotation_file_num_); + file_sink->set_level(level); sinks.push_back(file_sink); } else { - // Format pattern is 2020-08-21 17:00:00,000 I 100 1001 msg. - // %L is loglevel, %P is process id, %t for thread id. - log_format_pattern_ = - "[%Y-%m-%d %H:%M:%S,%e %L %P %t] (" + app_name_without_path + ") %v"; + component_name_ = app_name_without_path; auto console_sink = std::make_shared(); - console_sink->set_pattern(log_format_pattern_); console_sink->set_level(level); sinks.push_back(console_sink); } @@ -303,7 +284,6 @@ void RayLog::StartRayLog(const std::string &app_name, // In all cases, log errors to the console log so they are in driver logs. // https://github.com/ray-project/ray/issues/12893 auto err_sink = std::make_shared(); - err_sink->set_pattern(log_format_pattern_); err_sink->set_level(spdlog::level::err); sinks.push_back(err_sink); @@ -311,9 +291,8 @@ void RayLog::StartRayLog(const std::string &app_name, auto logger = std::make_shared( RayLog::GetLoggerName(), sinks.begin(), sinks.end()); logger->set_level(level); + // Set the pattern of all sinks. logger->set_pattern(log_format_pattern_); - spdlog::set_level(static_cast(severity_threshold_)); - spdlog::set_pattern(log_format_pattern_); spdlog::set_default_logger(logger); initialized_ = true; @@ -418,58 +397,87 @@ void RayLog::AddFatalLogCallbacks( } RayLog::RayLog(const char *file_name, int line_number, RayLogLevel severity) - : logging_provider_(nullptr), - is_enabled_(severity >= severity_threshold_), + : is_enabled_(severity >= severity_threshold_), severity_(severity), is_fatal_(severity == RayLogLevel::FATAL) { if (is_fatal_) { - expose_osstream_ = std::make_shared(); - #ifdef _WIN32 int pid = _getpid(); #else pid_t pid = getpid(); #endif - *expose_osstream_ << absl::StrFormat("%s:%d (PID: %d, TID: %s, errno: %d (%s)):", - file_name, - line_number, - pid, - std::to_string(GetTid()), - errno, - strerror(errno)); + expose_fatal_osstream_ << absl::StrFormat("%s:%d (PID: %d, TID: %s, errno: %d (%s)):", + file_name, + line_number, + pid, + std::to_string(GetTid()), + errno, + strerror(errno)); } if (is_enabled_) { - logging_provider_ = new LoggingProvider( - file_name, line_number, GetMappedSeverity(severity), expose_osstream_); + if (log_format_json_) { + if (!component_name_.empty()) { + WithField(kLogKeyComponent, component_name_); + } + WithField(kLogKeyFilename, ConstBasename(file_name)); + WithField(kLogKeyLineno, line_number); + } else { + if (!component_name_.empty()) { + msg_osstream_ << "(" << component_name_ << ") "; + } + msg_osstream_ << ConstBasename(file_name) << ":" << line_number << ": "; + } } } -std::ostream &RayLog::Stream() { - auto logging_provider = reinterpret_cast(logging_provider_); - // Before calling this function, user should check IsEnabled. - // When IsEnabled == false, logging_provider_ will be empty. - return logging_provider->stream(); -} - bool RayLog::IsEnabled() const { return is_enabled_; } bool RayLog::IsFatal() const { return is_fatal_; } -std::ostream &RayLog::ExposeStream() { return *expose_osstream_; } - RayLog::~RayLog() { - if (logging_provider_ != nullptr) { - delete reinterpret_cast(logging_provider_); - logging_provider_ = nullptr; - } - if (expose_osstream_ != nullptr) { + if (IsFatal()) { + msg_osstream_ << "\n*** StackTrace Information ***\n" << ray::StackTrace(); + expose_fatal_osstream_ << "\n*** StackTrace Information ***\n" << ray::StackTrace(); for (const auto &callback : fatal_log_callbacks_) { - callback(EL_RAY_FATAL_CHECK_FAILED, expose_osstream_->str()); + callback(EL_RAY_FATAL_CHECK_FAILED, expose_fatal_osstream_.str()); } } + + auto logger = spdlog::get(RayLog::GetLoggerName()); + if (!logger) { + logger = DefaultStdErrLogger::Instance().GetDefaultLogger(); + } + // NOTE(lingxuan.zlx): See more fmt by visiting https://github.com/fmtlib/fmt. + if (log_format_json_) { + logger->log(GetMappedSeverity(severity_), + /*fmt*/ ",\"{}\":{}{}", + kLogKeyMessage, + nlohmann::json(msg_osstream_.str()).dump(), + context_osstream_.str()); + } else { + logger->log(GetMappedSeverity(severity_), + /*fmt*/ "{}{}", + msg_osstream_.str(), + context_osstream_.str()); + } + logger->flush(); + if (severity_ == RayLogLevel::FATAL) { std::_Exit(EXIT_FAILURE); } } +template <> +RayLog &RayLog::WithFieldJsonFormat(std::string_view key, + const std::string &value) { + context_osstream_ << ",\"" << key << "\":" << nlohmann::json(value).dump(); + return *this; +} + +template <> +RayLog &RayLog::WithFieldJsonFormat(std::string_view key, const int &value) { + context_osstream_ << ",\"" << key << "\":" << value; + return *this; +} + } // namespace ray diff --git a/src/ray/util/logging.h b/src/ray/util/logging.h index 2adf4b26f617..16abe4272509 100644 --- a/src/ray/util/logging.h +++ b/src/ray/util/logging.h @@ -85,6 +85,19 @@ enum { ERROR = 0 }; #endif namespace ray { +/// Sync with ray._private.structured_logging.constants.LogKey +constexpr std::string_view kLogKeyAsctime = "asctime"; +constexpr std::string_view kLogKeyLevelname = "levelname"; +constexpr std::string_view kLogKeyMessage = "message"; +constexpr std::string_view kLogKeyFilename = "filename"; +constexpr std::string_view kLogKeyLineno = "lineno"; +constexpr std::string_view kLogKeyComponent = "component"; +constexpr std::string_view kLogKeyJobID = "job_id"; +constexpr std::string_view kLogKeyWorkerID = "worker_id"; +constexpr std::string_view kLogKeyNodeID = "node_id"; +constexpr std::string_view kLogKeyActorID = "actor_id"; +constexpr std::string_view kLogKeyTaskID = "task_id"; + class StackTrace { /// This dumps the current stack trace information. friend std::ostream &operator<<(std::ostream &os, const StackTrace &stack_trace); @@ -208,50 +221,24 @@ enum class RayLogLevel { // which hide the implementation into logging.cc file. // In logging.cc, we can choose different log libs using different macros. -// This is also a null log which does not output anything. -class RayLogBase { - public: - virtual ~RayLogBase(){}; - - // By default, this class is a null log because it return false here. - virtual bool IsEnabled() const { return false; }; - - // This function to judge whether current log is fatal or not. - virtual bool IsFatal() const { return false; }; - - template - RayLogBase &operator<<(const T &t) { - if (IsEnabled()) { - Stream() << t; - } - if (IsFatal()) { - ExposeStream() << t; - } - return *this; - } - - protected: - virtual std::ostream &Stream() { return std::cerr; }; - virtual std::ostream &ExposeStream() { return std::cerr; }; -}; - /// Callback function which will be triggered to expose fatal log. /// The first argument: a string representing log type or label. /// The second argument: log content. using FatalLogCallback = std::function; -class RayLog : public RayLogBase { +class RayLog { public: RayLog(const char *file_name, int line_number, RayLogLevel severity); - virtual ~RayLog(); + ~RayLog(); /// Return whether or not current logging instance is enabled. /// /// \return True if logging is enabled and false otherwise. - virtual bool IsEnabled() const; + bool IsEnabled() const; - virtual bool IsFatal() const; + /// This function to judge whether current log is fatal or not. + bool IsFatal() const; /// The init function of ray log for a program which should be called only once. /// @@ -296,9 +283,6 @@ class RayLog : public RayLogBase { /// To check failure signal handler enabled or not. static bool IsFailureSignalHandlerEnabled(); - /// Get the log level from environment variable. - static RayLogLevel GetLogLevelFromEnv(); - static std::string GetLogFormatPattern(); static std::string GetLoggerName(); @@ -307,35 +291,81 @@ class RayLog : public RayLogBase { static void AddFatalLogCallbacks( const std::vector &expose_log_callbacks); + template + RayLog &operator<<(const T &t) { + if (IsEnabled()) { + msg_osstream_ << t; + } + if (IsFatal()) { + expose_fatal_osstream_ << t; + } + return *this; + } + + /// Add log context to the log. + /// Caller should make sure key is not duplicated + /// and doesn't conflict with system keys like levelname. + template + RayLog &WithField(std::string_view key, const T &value) { + if (log_format_json_) { + return WithFieldJsonFormat(key, value); + } else { + return WithFieldTextFormat(key, value); + } + } + private: FRIEND_TEST(PrintLogTest, TestRayLogEveryNOrDebug); FRIEND_TEST(PrintLogTest, TestRayLogEveryN); - // Hide the implementation of log provider by void *. - // Otherwise, lib user may define the same macro to use the correct header file. - void *logging_provider_; + + template + RayLog &WithFieldTextFormat(std::string_view key, const T &value) { + context_osstream_ << " " << key << "=" << value; + return *this; + } + + template + RayLog &WithFieldJsonFormat(std::string_view key, const T &value) { + std::stringstream ss; + ss << value; + return WithFieldJsonFormat(key, ss.str()); + } + + static void InitSeverityThreshold(RayLogLevel severity_threshold); + static void InitLogFormat(); + /// True if log messages should be logged and false if they should be ignored. bool is_enabled_; /// log level. RayLogLevel severity_; /// Whether current log is fatal or not. bool is_fatal_ = false; - /// String stream of exposed log content. - std::shared_ptr expose_osstream_ = nullptr; + /// String stream of the log message + std::ostringstream msg_osstream_; + /// String stream of the log context: a list of key-value pairs. + std::ostringstream context_osstream_; + /// String stream of exposed fatal log content. + std::ostringstream expose_fatal_osstream_; + /// Whether or not the log is initialized. static std::atomic initialized_; /// Callback functions which will be triggered to expose fatal log. static std::vector fatal_log_callbacks_; static RayLogLevel severity_threshold_; - // In InitGoogleLogging, it simply keeps the pointer. - // We need to make sure the app name passed to InitGoogleLogging exist. static std::string app_name_; + /// This is used when we log to stderr + /// to indicate which component generates the log. + /// This is empty if we log to file. + static std::string component_name_; /// The directory where the log files are stored. /// If this is empty, logs are printed to stdout. static std::string log_dir_; /// This flag is used to avoid calling UninstallSignalAction in ShutDownRayLog if /// InstallFailureSignalHandler was not called. static bool is_failure_signal_handler_installed_; - // Log format content. + /// Whether emit json logs. + static bool log_format_json_; + // Log format pattern. static std::string log_format_pattern_; // Log rotation file size limitation. static long log_rotation_max_size_; @@ -345,17 +375,22 @@ class RayLog : public RayLogBase { static std::string logger_name_; protected: - virtual std::ostream &Stream(); - virtual std::ostream &ExposeStream(); + virtual std::ostream &Stream() { return msg_osstream_; } }; +template <> +RayLog &RayLog::WithFieldJsonFormat(std::string_view key, + const std::string &value); +template <> +RayLog &RayLog::WithFieldJsonFormat(std::string_view key, const int &value); + // This class make RAY_CHECK compilation pass to change the << operator to void. class Voidify { public: Voidify() {} // This has to be an operator with a precedence lower than << but // higher than ?: - void operator&(RayLogBase &) {} + void operator&(RayLog &) {} }; } // namespace ray diff --git a/src/ray/util/tests/logging_test.cc b/src/ray/util/tests/logging_test.cc index 965e443eb466..b09197b64a0f 100644 --- a/src/ray/util/tests/logging_test.cc +++ b/src/ray/util/tests/logging_test.cc @@ -16,24 +16,22 @@ #include #include +#include #include #include "absl/strings/str_format.h" +#include "absl/strings/str_split.h" #include "gmock/gmock.h" #include "gtest/gtest.h" +#include "nlohmann/json.hpp" #include "ray/util/filesystem.h" +#include "ray/util/util.h" using namespace testing; +using json = nlohmann::json; namespace ray { -int64_t current_time_ms() { - std::chrono::milliseconds ms_since_epoch = - std::chrono::duration_cast( - std::chrono::steady_clock::now().time_since_epoch()); - return ms_since_epoch.count(); -} - // This is not really test. // This file just print some information using the logging macro. @@ -61,7 +59,9 @@ TEST(PrintLogTest, LogTestWithoutInit) { #if GTEST_HAS_STREAM_REDIRECTION using testing::internal::CaptureStderr; +using testing::internal::CaptureStdout; using testing::internal::GetCapturedStderr; +using testing::internal::GetCapturedStdout; namespace { void VerifyOnlyNthOccurenceLogged(bool fallback_to_debug) { @@ -186,6 +186,54 @@ TEST(PrintLogTest, TestRayLogEveryMs) { EXPECT_LT(occurrences, 15); } +TEST(PrintLogTest, TestTextLogging) { + setenv("RAY_BACKEND_LOG_JSON", "0", true); + RayLog::StartRayLog("/tmp/gcs", RayLogLevel::INFO, ""); + CaptureStdout(); + RAY_LOG(INFO).WithField("key1", "value1").WithField("key2", "value2") + << "contextual log"; + + std::vector log_lines = + absl::StrSplit(GetCapturedStdout(), '\n', absl::SkipEmpty()); + ASSERT_EQ(1, log_lines.size()); + ASSERT_NE(log_lines[0].find("contextual log key1=value1 key2=value2"), + std::string::npos); + + RayLog::ShutDownRayLog(); + unsetenv("RAY_BACKEND_LOG_JSON"); +} + +TEST(PrintLogTest, TestJSONLogging) { + setenv("RAY_BACKEND_LOG_JSON", "1", true); + RayLog::StartRayLog("/tmp/raylet", RayLogLevel::INFO, ""); + CaptureStdout(); + RAY_LOG(DEBUG) << "this is not logged"; + RAY_LOG(INFO) << "this is info logged"; + RAY_LOG(WARNING) << "this needs\nescape\""; + RAY_LOG(INFO).WithField("key1", "value1").WithField("key2", "value\n2") + << "contextual log"; + + std::vector log_lines = + absl::StrSplit(GetCapturedStdout(), '\n', absl::SkipEmpty()); + ASSERT_EQ(3, log_lines.size()); + json log1 = json::parse(log_lines[0]); + json log2 = json::parse(log_lines[1]); + json log3 = json::parse(log_lines[2]); + ASSERT_EQ(log1[std::string(kLogKeyMessage)], "this is info logged"); + ASSERT_EQ(log2[std::string(kLogKeyMessage)], "this needs\nescape\""); + ASSERT_EQ(log3[std::string(kLogKeyMessage)], "contextual log"); + ASSERT_TRUE(log3.contains(kLogKeyAsctime)); + ASSERT_TRUE(log3.contains(kLogKeyFilename)); + ASSERT_TRUE(log3.contains(kLogKeyLineno)); + ASSERT_EQ(log3[std::string(kLogKeyLevelname)], "I"); + ASSERT_EQ(log3[std::string(kLogKeyComponent)], "raylet"); + ASSERT_EQ(log3["key1"], "value1"); + ASSERT_EQ(log3["key2"], "value\n2"); + + RayLog::ShutDownRayLog(); + unsetenv("RAY_BACKEND_LOG_JSON"); +} + #endif /* GTEST_HAS_STREAM_REDIRECTION */ TEST(PrintLogTest, LogTestWithInit) { From 3c9edf1b790dc65c36ee27d4032ab75a89e41930 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Fri, 31 May 2024 13:43:49 -0700 Subject: [PATCH 63/65] [Data] Record more telemetry for newly added datasources (#45647) This PR is to add the telemetry recording for newly added datasources. Signed-off-by: Cheng Su --- python/ray/data/_internal/logical/util.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/python/ray/data/_internal/logical/util.py b/python/ray/data/_internal/logical/util.py index 084bb2c67aff..af6f2420a269 100644 --- a/python/ray/data/_internal/logical/util.py +++ b/python/ray/data/_internal/logical/util.py @@ -28,8 +28,14 @@ "ReadNumpy", "ReadTFRecord", "ReadBinary", - "ReadCustom", "ReadTorch", + "ReadAvro", + "ReadWebDataset", + "ReadSQL", + "ReadDatabricksUC", + "ReadLance", + "ReadHuggingFace", + "ReadCustom", # From "FromArrow", "FromItems", @@ -43,6 +49,8 @@ "WriteTFRecord", "WriteNumpy", "WriteMongo", + "WriteWebDataset", + "WriteSQL", "WriteCustom", # Map "Map", From fe191e6c81178de48a404bb279233651aad8a63e Mon Sep 17 00:00:00 2001 From: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> Date: Fri, 31 May 2024 14:06:59 -0700 Subject: [PATCH 64/65] [Core] Expose NodeDeathInfo in state CLI (#45644) Signed-off-by: Rui Qiao --- dashboard/datacenter.py | 3 ++- dashboard/state_aggregator.py | 5 +++++ dashboard/utils.py | 11 +++++++---- python/ray/tests/test_state_api.py | 18 +++++++++++------- python/ray/tests/test_task_events.py | 3 ++- python/ray/util/state/common.py | 3 +++ .../transport/direct_task_transport.cc | 17 ++++++++++------- 7 files changed, 40 insertions(+), 20 deletions(-) diff --git a/dashboard/datacenter.py b/dashboard/datacenter.py index 1e4627eadc86..f7702e64411e 100644 --- a/dashboard/datacenter.py +++ b/dashboard/datacenter.py @@ -138,8 +138,9 @@ async def get_node_info(cls, node_id, get_summary=False): # Merge GcsNodeInfo to node physical stats node_info["raylet"].update(node) + death_info = node.get("deathInfo", {}) node_info["raylet"]["stateMessage"] = compose_state_message( - node.get("deathInfo", {}) + death_info.get("reason", None), death_info.get("reasonMessage", None) ) if not get_summary: diff --git a/dashboard/state_aggregator.py b/dashboard/state_aggregator.py index e3aa7a9f8c6a..5c8591484d32 100644 --- a/dashboard/state_aggregator.py +++ b/dashboard/state_aggregator.py @@ -11,6 +11,7 @@ from ray._private.profiling import chrome_tracing_dump import ray.dashboard.memory_utils as memory_utils +from ray.dashboard.utils import compose_state_message from ray.util.state.common import ( protobuf_message_to_dict, @@ -323,6 +324,10 @@ async def list_nodes(self, *, option: ListApiOptions) -> ListApiResponse: data["node_ip"] = data["node_manager_address"] data["start_time_ms"] = int(data["start_time_ms"]) data["end_time_ms"] = int(data["end_time_ms"]) + death_info = data.get("death_info", {}) + data["state_message"] = compose_state_message( + death_info.get("reason", None), death_info.get("reason_message", None) + ) result.append(data) diff --git a/dashboard/utils.py b/dashboard/utils.py index 5d5c4a0284c7..99c624803bbe 100644 --- a/dashboard/utils.py +++ b/dashboard/utils.py @@ -656,13 +656,17 @@ def get_address_for_submission_client(address: Optional[str]) -> str: return address -def compose_state_message(death_info_dict: dict) -> Optional[str]: +def compose_state_message( + death_reason: Optional[str], death_reason_message: Optional[str] +) -> Optional[str]: """Compose node state message based on death information. Args: - death_info_dict: the node_death field in GcsNodeInfo, in dict type. + death_reason: The reason of node death. + This is a string representation of `gcs_pb2.NodeDeathInfo.Reason`. + death_reason_message: The message of node death. + This corresponds to `gcs_pb2.NodeDeathInfo.ReasonMessage`. """ - death_reason = death_info_dict.get("reason", None) if death_reason == "EXPECTED_TERMINATION": state_message = "Expected termination" elif death_reason == "UNEXPECTED_TERMINATION": @@ -674,7 +678,6 @@ def compose_state_message(death_info_dict: dict) -> Optional[str]: else: state_message = None - death_reason_message = death_info_dict.get("reasonMessage", None) if death_reason_message: if state_message: state_message += f": {death_reason_message}" diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index b6555e1e9ab1..f9f582547f9c 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -2154,19 +2154,23 @@ def test_list_get_nodes(ray_start_cluster): cluster = ray_start_cluster cluster.add_node(num_cpus=1, node_name="head_node") ray.init(address=cluster.address) - cluster.add_node(num_cpus=1, node_name="worker_node") + worker_node = cluster.add_node(num_cpus=1, node_name="worker_node") + + cluster.remove_node(worker_node) def verify(): nodes = list_nodes(detail=True) for node in nodes: - assert node["state"] == "ALIVE" assert is_hex(node["node_id"]) - assert ( - node["is_head_node"] - if node["node_name"] == "head_node" - else not node["is_head_node"] - ) assert node["labels"] == {"ray.io/node_id": node["node_id"]} + if node["node_name"] == "head_node": + assert node["is_head_node"] + assert node["state"] == "ALIVE" + assert node["state_message"] is None + else: + assert not node["is_head_node"] + assert node["state"] == "DEAD" + assert node["state_message"] == "Expected termination: received SIGTERM" # Check with legacy API check_nodes = ray.nodes() diff --git a/python/ray/tests/test_task_events.py b/python/ray/tests/test_task_events.py index 89434049349d..b1210b59aff3 100644 --- a/python/ray/tests/test_task_events.py +++ b/python/ray/tests/test_task_events.py @@ -204,7 +204,8 @@ def driver_running(): verify_failed_task, name="node-killed", error_type="NODE_DIED", - error_message="Task failed due to the node dying", + error_message="Task failed due to the node (where this task was running) " + " was dead or unavailable", ) diff --git a/python/ray/util/state/common.py b/python/ray/util/state/common.py index 366648674ee4..9dbb56286c61 100644 --- a/python/ray/util/state/common.py +++ b/python/ray/util/state/common.py @@ -507,6 +507,9 @@ class NodeState(StateSchema): #: ALIVE: The node is alive. #: DEAD: The node is dead. state: TypeNodeStatus = state_column(filterable=True) + #: The state message of the node. + #: This provides more detailed information about the node's state. + state_message: Optional[str] = state_column(filterable=False) #: The name of the node if it is given by the name argument. node_name: str = state_column(filterable=True) #: The total resources of the node. diff --git a/src/ray/core_worker/transport/direct_task_transport.cc b/src/ray/core_worker/transport/direct_task_transport.cc index 54799a1acba0..b419fae2ffea 100644 --- a/src/ray/core_worker/transport/direct_task_transport.cc +++ b/src/ray/core_worker/transport/direct_task_transport.cc @@ -728,13 +728,16 @@ void CoreWorkerDirectTaskSubmitter::HandleGetTaskFailureCause( << " ip: " << addr.ip_address(); task_error_type = rpc::ErrorType::NODE_DIED; std::stringstream buffer; - buffer << "Task failed due to the node dying.\n\nThe node (IP: " << addr.ip_address() - << ", node ID: " << NodeID::FromBinary(addr.raylet_id()) - << ") where this task was running crashed unexpectedly. " - << "This can happen if: (1) the instance where the node was running failed, " - "(2) raylet crashes unexpectedly (OOM, preempted node, etc).\n\n" - << "To see more information about the crash, use `ray logs raylet.out -ip " - << addr.ip_address() << "`"; + buffer << "Task failed due to the node (where this task was running) " + << " was dead or unavailable.\n\nThe node IP: " << addr.ip_address() + << ", node ID: " << NodeID::FromBinary(addr.raylet_id()) << "\n\n" + << "This can happen if the instance where the node was running failed, " + << "the node was preempted, or raylet crashed unexpectedly " + << "(e.g., due to OOM) etc.\n\n" + << "To see node death information, use `ray list nodes --filter \"node_id=" + << NodeID::FromBinary(addr.raylet_id()) << "\"`, " + << "or check Ray dashboard cluster page, or search the node ID in GCS log, " + << "or use `ray logs raylet.out -ip " << addr.ip_address() << "`"; error_info = std::make_unique(); error_info->set_error_message(buffer.str()); error_info->set_error_type(rpc::ErrorType::NODE_DIED); From 7021b10356069cf424556f1a5683c5f270a87e5b Mon Sep 17 00:00:00 2001 From: Rui Qiao <161574667+ruisearch42@users.noreply.github.com> Date: Fri, 31 May 2024 21:27:39 -0700 Subject: [PATCH 65/65] [Core] Expose NodeDeathInfo in ActorDiedError (#45497) Signed-off-by: Rui Qiao --- dashboard/tests/test_dashboard.py | 4 +- python/ray/exceptions.py | 14 +++--- python/ray/tests/test_actor_failures.py | 2 +- python/ray/tests/test_draining.py | 18 +++++--- python/ray/tests/test_node_death.py | 43 +++++++++++++++++-- python/ray/tests/test_runtime_env_agent.py | 4 +- .../transport/direct_actor_task_submitter.cc | 13 +++--- src/ray/gcs/gcs_server/gcs_actor_manager.cc | 34 +++++++++++---- src/ray/protobuf/common.proto | 18 +++++++- src/ray/protobuf/gcs.proto | 14 ------ 10 files changed, 114 insertions(+), 50 deletions(-) diff --git a/dashboard/tests/test_dashboard.py b/dashboard/tests/test_dashboard.py index a5254179d948..1fc3ad1bb630 100644 --- a/dashboard/tests/test_dashboard.py +++ b/dashboard/tests/test_dashboard.py @@ -36,7 +36,7 @@ wait_until_server_available, wait_until_succeeded_without_exception, ) -from ray.core.generated import gcs_pb2 +from ray.core.generated import common_pb2 import ray.scripts.scripts as scripts from ray.dashboard import dashboard from ray.dashboard.head import DashboardHead @@ -219,7 +219,7 @@ def test_raylet_and_agent_share_fate(shutdown_only): node for node in ray.nodes() if node["NodeID"] == worker_node_id ][0] assert not worker_node_info["Alive"] - assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node_info["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "UNEXPECTED_TERMINATION" ) assert ( diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index e35eee260d95..14c308e20eac 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -13,6 +13,7 @@ ActorDiedErrorContext, Address, Language, + NodeDeathInfo, RayException, ) from ray.util.annotations import DeveloperAPI, PublicAPI @@ -339,7 +340,9 @@ class ActorDiedError(RayActorError): BASE_ERROR_MSG = "The actor died unexpectedly before finishing this task." - def __init__(self, cause: Union[RayTaskError, ActorDiedErrorContext] = None): + def __init__( + self, cause: Optional[Union[RayTaskError, ActorDiedErrorContext]] = None + ): """ Construct a RayActorError by building the arguments. """ @@ -380,11 +383,12 @@ def __init__(self, cause: Union[RayTaskError, ActorDiedErrorContext] = None): error_msg_lines.append( "The actor never ran - it was cancelled before it started running." ) - if cause.preempted: + if ( + cause.node_death_info + and cause.node_death_info.reason + == NodeDeathInfo.AUTOSCALER_DRAIN_PREEMPTED + ): preempted = True - error_msg_lines.append( - "\tThe actor's node was killed by a spot preemption." - ) error_msg = "\n".join(error_msg_lines) actor_id = ActorID(cause.actor_id).hex() super().__init__(actor_id, error_msg, actor_init_failed, preempted) diff --git a/python/ray/tests/test_actor_failures.py b/python/ray/tests/test_actor_failures.py index 86e14eaee994..975b942d5f9f 100644 --- a/python/ray/tests/test_actor_failures.py +++ b/python/ray/tests/test_actor_failures.py @@ -748,7 +748,7 @@ def create_actor(self): cluster.remove_node(node_to_kill) with pytest.raises( ray.exceptions.RayActorError, - match="The actor is dead because its node has died.", + match="The actor died because its node has died.", ) as exc_info: ray.get(a.check_alive.remote()) assert exc_info.value.actor_id == a._actor_id.hex() diff --git a/python/ray/tests/test_draining.py b/python/ray/tests/test_draining.py index 360e5d5d34ad..ee942f4c13bf 100644 --- a/python/ray/tests/test_draining.py +++ b/python/ray/tests/test_draining.py @@ -4,7 +4,7 @@ import ray import time from ray._raylet import GcsClient -from ray.core.generated import autoscaler_pb2, gcs_pb2 +from ray.core.generated import autoscaler_pb2, common_pb2 from ray._private.test_utils import wait_for_condition from ray.util.scheduling_strategies import ( NodeAffinitySchedulingStrategy, @@ -69,7 +69,7 @@ def drain_until_accept(): ) worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] - assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "AUTOSCALER_DRAIN_IDLE" ) assert worker_node["DeathReasonMessage"] == "idle for long enough" @@ -137,7 +137,7 @@ def ping(self): ) worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] - assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "AUTOSCALER_DRAIN_PREEMPTED" ) assert worker_node["DeathReasonMessage"] == "preemption" @@ -195,7 +195,7 @@ def ping(self): ) worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] - assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "AUTOSCALER_DRAIN_PREEMPTED" ) assert worker_node["DeathReasonMessage"] == "preemption" @@ -251,7 +251,7 @@ def ping(self): # Since worker node failure is detected to be before the draining deadline, # this is considered as an unexpected termination. worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] - assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "UNEXPECTED_TERMINATION" ) assert ( @@ -412,11 +412,12 @@ def ping(self): actor = Actor.options(num_cpus=0, resources={"node2": 1}).remote() ray.get(actor.ping.remote()) + drain_reason_message = "testing node preemption." # Preemption is always accepted. is_accepted, _ = gcs_client.drain_node( node2_id, autoscaler_pb2.DrainNodeReason.Value("DRAIN_NODE_REASON_PREEMPTION"), - "preemption", + drain_reason_message, 1, ) assert is_accepted @@ -426,8 +427,11 @@ def ping(self): try: ray.get(actor.ping.remote()) raise - except ray.exceptions.RayActorError as e: + except ray.exceptions.ActorDiedError as e: assert e.preempted + if graceful: + assert "The actor died because its node has died." in str(e) + assert "the actor's node was preempted: " + drain_reason_message in str(e) if __name__ == "__main__": diff --git a/python/ray/tests/test_node_death.py b/python/ray/tests/test_node_death.py index de36091c4341..cc2ef747102b 100644 --- a/python/ray/tests/test_node_death.py +++ b/python/ray/tests/test_node_death.py @@ -4,7 +4,7 @@ import ray from ray._private.test_utils import wait_for_condition -from ray.core.generated import gcs_pb2 +from ray.core.generated import common_pb2 def test_normal_termination(ray_start_cluster): @@ -14,17 +14,35 @@ def test_normal_termination(ray_start_cluster): worker_node = cluster.add_node(resources={"worker": 1}) cluster.wait_for_nodes() worker_node_id = worker_node.node_id + + @ray.remote + class Actor: + def ping(self): + pass + + actor = Actor.options(num_cpus=0, resources={"worker": 1}).remote() + ray.get(actor.ping.remote()) + + # normal node termination cluster.remove_node(worker_node) worker_node_info = [ node for node in ray.nodes() if node["NodeID"] == worker_node_id ][0] assert not worker_node_info["Alive"] - assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node_info["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "EXPECTED_TERMINATION" ) assert worker_node_info["DeathReasonMessage"] == "received SIGTERM" + try: + ray.get(actor.ping.remote()) + raise + except ray.exceptions.ActorDiedError as e: + assert not e.preempted + assert "The actor died because its node has died." in str(e) + assert "the actor's node was terminated expectedly: received SIGTERM" in str(e) + def test_abnormal_termination(monkeypatch, ray_start_cluster): monkeypatch.setenv("RAY_health_check_failure_threshold", "3") @@ -46,6 +64,14 @@ def test_abnormal_termination(monkeypatch, ray_start_cluster): == {head_node_id, worker_node_id} ) + @ray.remote + class Actor: + def ping(self): + pass + + actor = Actor.options(num_cpus=0, resources={"worker": 1}).remote() + ray.get(actor.ping.remote()) + # Simulate the worker node crashes. cluster.remove_node(worker_node, False) @@ -55,7 +81,7 @@ def test_abnormal_termination(monkeypatch, ray_start_cluster): ) worker_node = [node for node in ray.nodes() if node["NodeID"] == worker_node_id][0] - assert worker_node["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "UNEXPECTED_TERMINATION" ) assert ( @@ -63,6 +89,17 @@ def test_abnormal_termination(monkeypatch, ray_start_cluster): == "health check failed due to missing too many heartbeats" ) + try: + ray.get(actor.ping.remote()) + raise + except ray.exceptions.ActorDiedError as e: + assert not e.preempted + assert "The actor died because its node has died." in str(e) + assert ( + "the actor's node was terminated unexpectedly: " + "health check failed due to missing too many heartbeats" in str(e) + ) + if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/test_runtime_env_agent.py b/python/ray/tests/test_runtime_env_agent.py index 4af9165754a3..0320bcba8b94 100644 --- a/python/ray/tests/test_runtime_env_agent.py +++ b/python/ray/tests/test_runtime_env_agent.py @@ -13,7 +13,7 @@ init_error_pubsub, wait_for_condition, ) -from ray.core.generated import gcs_pb2 +from ray.core.generated import common_pb2 from ray.runtime_env import RuntimeEnv import psutil @@ -160,7 +160,7 @@ def test_raylet_and_agent_share_fate(shutdown_only): node for node in ray.nodes() if node["NodeID"] == worker_node_id ][0] assert not worker_node_info["Alive"] - assert worker_node_info["DeathReason"] == gcs_pb2.NodeDeathInfo.Reason.Value( + assert worker_node_info["DeathReason"] == common_pb2.NodeDeathInfo.Reason.Value( "UNEXPECTED_TERMINATION" ) assert ( diff --git a/src/ray/core_worker/transport/direct_actor_task_submitter.cc b/src/ray/core_worker/transport/direct_actor_task_submitter.cc index 7c740f4cea70..0bda386195cc 100644 --- a/src/ray/core_worker/transport/direct_actor_task_submitter.cc +++ b/src/ray/core_worker/transport/direct_actor_task_submitter.cc @@ -347,12 +347,13 @@ void CoreWorkerDirectActorTaskSubmitter::FailTaskWithError( // Special error for preempted actor. The task "timed out" because the actor may // not have sent a notification to the gcs; regardless we already know it's // preempted and it's dead. - rpc::ActorDeathCause &actor_death_cause = *error_info.mutable_actor_died_error(); - actor_death_cause.mutable_actor_died_error_context()->set_actor_id( - task.task_spec.ActorId().Binary()); - actor_death_cause.mutable_actor_died_error_context()->set_preempted( - task.actor_preempted); - + auto actor_death_cause = error_info.mutable_actor_died_error(); + auto actor_died_error_context = actor_death_cause->mutable_actor_died_error_context(); + actor_died_error_context->set_actor_id(task.task_spec.ActorId().Binary()); + auto node_death_info = actor_died_error_context->mutable_node_death_info(); + node_death_info->set_reason(rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED); + node_death_info->set_reason_message( + "the node was inferred to be dead due to draining."); error_info.set_error_type(rpc::ErrorType::ACTOR_DIED); error_info.set_error_message("Actor died by preemption."); } diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.cc b/src/ray/gcs/gcs_server/gcs_actor_manager.cc index 008b6c13e3a2..fb89e092f256 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.cc @@ -205,17 +205,35 @@ const ray::rpc::ActorDeathCause GcsActorManager::GenNodeDiedCause( const std::string ip_address, std::shared_ptr node) { ray::rpc::ActorDeathCause death_cause; + auto actor_died_error_ctx = death_cause.mutable_actor_died_error_context(); AddActorInfo(actor, actor_died_error_ctx); - actor_died_error_ctx->set_error_message( - absl::StrCat("The actor is dead because its node has died. Node Id: ", - NodeID::FromBinary(node->node_id()).Hex())); - - // TODO(vitsai): Publish this information as well - if (auto death_info = node->death_info(); - death_info.reason() == rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED) { - actor_died_error_ctx->set_preempted(true); + auto node_death_info = actor_died_error_ctx->mutable_node_death_info(); + node_death_info->CopyFrom(node->death_info()); + + std::ostringstream oss; + oss << "The actor died because its node has died. Node Id: " + << NodeID::FromBinary(node->node_id()).Hex() << "\n"; + switch (node_death_info->reason()) { + case rpc::NodeDeathInfo::EXPECTED_TERMINATION: + oss << "\tthe actor's node was terminated expectedly: "; + break; + case rpc::NodeDeathInfo::UNEXPECTED_TERMINATION: + oss << "\tthe actor's node was terminated unexpectedly: "; + break; + case rpc::NodeDeathInfo::AUTOSCALER_DRAIN_PREEMPTED: + oss << "\tthe actor's node was preempted: "; + break; + default: + // Control should not reach here, but in case it happens in unexpected scenarios, + // log it and provide a generic message to the user. + RAY_LOG(ERROR) << "Actor death is not expected to be caused by " + << rpc::NodeDeathInfo_Reason_Name(node_death_info->reason()); + oss << "\tthe actor's node was terminated: "; } + oss << node_death_info->reason_message(); + actor_died_error_ctx->set_error_message(oss.str()); + return death_cause; } diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index 05c148b3fb2b..8f6d9c27cdee 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -308,6 +308,20 @@ message RayException { string formatted_exception_string = 3; } +message NodeDeathInfo { + // TODO(sang): Update drain reason + enum Reason { + UNSPECIFIED = 0; + EXPECTED_TERMINATION = 1; + UNEXPECTED_TERMINATION = 2; + AUTOSCALER_DRAIN_PREEMPTED = 3; + AUTOSCALER_DRAIN_IDLE = 4; + } + Reason reason = 1; + // A message describing the reason for the node death. + string reason_message = 2; +} + message ActorDeathCause { oneof context { // Indicates that this actor is marked as DEAD due to actor creation task failure. @@ -350,8 +364,8 @@ message ActorDiedErrorContext { // Whether the actor had never started running before it died, i.e. it was cancelled // before scheduling had completed. bool never_started = 10; - // Whether the actor was on a preempted node. - bool preempted = 11; + // The node death info, if node death is the cause of actor death. + optional NodeDeathInfo node_death_info = 11; } // Context for task OOM. diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index 652310c3b0bf..fd8310b9f566 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -318,20 +318,6 @@ message NodeSnapshot { repeated string node_activity = 3; } -message NodeDeathInfo { - // TODO(sang): Update drain reason - enum Reason { - UNSPECIFIED = 0; - EXPECTED_TERMINATION = 1; - UNEXPECTED_TERMINATION = 2; - AUTOSCALER_DRAIN_PREEMPTED = 3; - AUTOSCALER_DRAIN_IDLE = 4; - } - Reason reason = 1; - // A message describing the reason for the node death. - string reason_message = 2; -} - message GcsNodeInfo { // State of a node. enum GcsNodeState {